You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by jm...@apache.org on 2015/06/24 18:14:54 UTC

[01/32] cassandra git commit: Migrate CQL tests from dtest to unit tests

Repository: cassandra
Updated Branches:
  refs/heads/trunk 7392fb96d -> bc9b0be32


http://git-wip-us.apache.org/repos/asf/cassandra/blob/f797bfa4/test/unit/org/apache/cassandra/cql3/validation/operations/SelectOrderedPartitionerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/SelectOrderedPartitionerTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/SelectOrderedPartitionerTest.java
new file mode 100644
index 0000000..a2ba71e
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/SelectOrderedPartitionerTest.java
@@ -0,0 +1,325 @@
+package org.apache.cassandra.cql3.validation.operations;
+
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import static junit.framework.Assert.assertNull;
+import static org.junit.Assert.assertEquals;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.dht.ByteOrderedPartitioner;
+
+/**
+ * SELECT statement tests that require a ByteOrderedPartitioner
+ */
+public class SelectOrderedPartitionerTest extends CQLTester
+{
+    @BeforeClass
+    public static void setUp()
+    {
+        DatabaseDescriptor.setPartitioner(new ByteOrderedPartitioner());
+    }
+
+    @Test
+    public void testTokenFunctionWithSingleColumnPartitionKey() throws Throwable
+    {
+        createTable("CREATE TABLE IF NOT EXISTS %s (a int PRIMARY KEY, b text)");
+        execute("INSERT INTO %s (a, b) VALUES (0, 'a')");
+
+        assertRows(execute("SELECT * FROM %s WHERE token(a) >= token(?)", 0), row(0, "a"));
+        assertRows(execute("SELECT * FROM %s WHERE token(a) >= token(?) and token(a) < token(?)", 0, 1), row(0, "a"));
+        assertInvalid("SELECT * FROM %s WHERE token(a) > token(?)", "a");
+        assertInvalid("SELECT * FROM %s WHERE token(a, b) >= token(?, ?)", "b", 0);
+        assertInvalid("SELECT * FROM %s WHERE token(a) >= token(?) and token(a) >= token(?)", 0, 1);
+        assertInvalid("SELECT * FROM %s WHERE token(a) >= token(?) and token(a) = token(?)", 0, 1);
+        assertInvalidSyntax("SELECT * FROM %s WHERE token(a) = token(?) and token(a) IN (token(?))", 0, 1);
+    }
+
+    @Test
+    public void testTokenFunctionWithPartitionKeyAndClusteringKeyArguments() throws Throwable
+    {
+        createTable("CREATE TABLE IF NOT EXISTS %s (a int, b text, PRIMARY KEY (a, b))");
+        assertInvalid("SELECT * FROM %s WHERE token(a, b) > token(0, 'c')");
+    }
+
+    @Test
+    public void testTokenFunctionWithMultiColumnPartitionKey() throws Throwable
+    {
+        createTable("CREATE TABLE IF NOT EXISTS %s (a int, b text, PRIMARY KEY ((a, b)))");
+        execute("INSERT INTO %s (a, b) VALUES (0, 'a')");
+        execute("INSERT INTO %s (a, b) VALUES (0, 'b')");
+        execute("INSERT INTO %s (a, b) VALUES (0, 'c')");
+
+        assertRows(execute("SELECT * FROM %s WHERE token(a, b) > token(?, ?)", 0, "a"),
+                   row(0, "b"),
+                   row(0, "c"));
+        assertRows(execute("SELECT * FROM %s WHERE token(a, b) > token(?, ?) and token(a, b) < token(?, ?)",
+                           0, "a",
+                           0, "d"),
+                   row(0, "b"),
+                   row(0, "c"));
+        assertInvalid("SELECT * FROM %s WHERE token(a) > token(?) and token(b) > token(?)", 0, "a");
+        assertInvalid("SELECT * FROM %s WHERE token(a) > token(?, ?) and token(a) < token(?, ?) and token(b) > token(?, ?) ", 0, "a", 0, "d", 0, "a");
+        assertInvalid("SELECT * FROM %s WHERE token(b, a) > token(0, 'c')");
+    }
+
+    @Test
+    public void testTokenFunctionWithCompoundPartitionAndClusteringCols() throws Throwable
+    {
+        createTable("CREATE TABLE IF NOT EXISTS %s (a int, b int, c int, d int, PRIMARY KEY ((a, b), c, d))");
+        // just test that the queries don't error
+        execute("SELECT * FROM %s WHERE token(a, b) > token(0, 0) AND c > 10 ALLOW FILTERING;");
+        execute("SELECT * FROM %s WHERE c > 10 AND token(a, b) > token(0, 0) ALLOW FILTERING;");
+        execute("SELECT * FROM %s WHERE token(a, b) > token(0, 0) AND (c, d) > (0, 0) ALLOW FILTERING;");
+        execute("SELECT * FROM %s WHERE (c, d) > (0, 0) AND token(a, b) > token(0, 0) ALLOW FILTERING;");
+    }
+
+    /**
+     * Test undefined columns
+     * migrated from cql_tests.py:TestCQL.undefined_column_handling_test()
+     */
+    @Test
+    public void testUndefinedColumns() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, v1 int, v2 int,)");
+
+        execute("INSERT INTO %s (k, v1, v2) VALUES (0, 0, 0)");
+        execute("INSERT INTO %s (k, v1) VALUES (1, 1)");
+        execute("INSERT INTO %s (k, v1, v2) VALUES (2, 2, 2)");
+
+        Object[][] rows = getRows(execute("SELECT v2 FROM %s"));
+        assertEquals(0, rows[0][0]);
+        assertEquals(null, rows[1][0]);
+        assertEquals(2, rows[2][0]);
+
+        rows = getRows(execute("SELECT v2 FROM %s WHERE k = 1"));
+        assertEquals(1, rows.length);
+        assertNull(rows[0][0]);
+    }
+
+    /**
+     * Check table with only a PK (#4361),
+     * migrated from cql_tests.py:TestCQL.only_pk_test()
+     */
+    @Test
+    public void testPrimaryKeyOnly() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int, c int, PRIMARY KEY (k, c))");
+
+        for (int k = 0; k < 2; k++)
+            for (int c = 0; c < 2; c++)
+                execute("INSERT INTO %s (k, c) VALUES (?, ?)", k, c);
+
+        assertRows(execute("SELECT * FROM %s"),
+                   row(0, 0),
+                   row(0, 1),
+                   row(1, 0),
+                   row(1, 1));
+
+        // Check for dense tables too
+        createTable(" CREATE TABLE %s (k int, c int, PRIMARY KEY (k, c)) WITH COMPACT STORAGE");
+
+        for (int k = 0; k < 2; k++)
+            for (int c = 0; c < 2; c++)
+                execute("INSERT INTO %s (k, c) VALUES (?, ?)", k, c);
+
+        assertRows(execute("SELECT * FROM %s"),
+                   row(0, 0),
+                   row(0, 1),
+                   row(1, 0),
+                   row(1, 1));
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.composite_index_with_pk_test()
+     */
+    @Test
+    public void testCompositeIndexWithPK() throws Throwable
+    {
+        createTable("CREATE TABLE %s (blog_id int, time1 int, time2 int, author text, content text, PRIMARY KEY (blog_id, time1, time2))");
+
+        createIndex("CREATE INDEX ON %s(author)");
+
+        execute("INSERT INTO %s (blog_id, time1, time2, author, content) VALUES (?, ?, ?, ?, ?)", 1, 0, 0, "foo", "bar1");
+        execute("INSERT INTO %s (blog_id, time1, time2, author, content) VALUES (?, ?, ?, ?, ?)", 1, 0, 1, "foo", "bar2");
+        execute("INSERT INTO %s (blog_id, time1, time2, author, content) VALUES (?, ?, ?, ?, ?)", 2, 1, 0, "foo", "baz");
+        execute("INSERT INTO %s (blog_id, time1, time2, author, content) VALUES (?, ?, ?, ?, ?)", 3, 0, 1, "gux", "qux");
+
+        assertRows(execute("SELECT blog_id, content FROM %s WHERE author='foo'"),
+                   row(1, "bar1"),
+                   row(1, "bar2"),
+                   row(2, "baz"));
+
+        assertRows(execute("SELECT blog_id, content FROM %s WHERE time1 > 0 AND author='foo' ALLOW FILTERING"),
+                   row(2, "baz"));
+
+        assertRows(execute("SELECT blog_id, content FROM %s WHERE time1 = 1 AND author='foo' ALLOW FILTERING"),
+                   row(2, "baz"));
+
+        assertRows(execute("SELECT blog_id, content FROM %s WHERE time1 = 1 AND time2 = 0 AND author='foo' ALLOW FILTERING"),
+                   row(2, "baz"));
+
+        assertEmpty(execute("SELECT content FROM %s WHERE time1 = 1 AND time2 = 1 AND author='foo' ALLOW FILTERING"));
+
+        assertEmpty(execute("SELECT content FROM %s WHERE time1 = 1 AND time2 > 0 AND author='foo' ALLOW FILTERING"));
+
+        assertInvalid("SELECT content FROM %s WHERE time2 >= 0 AND author='foo'");
+
+        execute("SELECT blog_id, content FROM %s WHERE time1 > 0 AND author='foo'");
+        execute("SELECT blog_id, content FROM %s WHERE time1 = 1 AND author='foo'");
+        execute("SELECT blog_id, content FROM %s WHERE time1 = 1 AND time2 = 0 AND author='foo'");
+        execute("SELECT content FROM %s WHERE time1 = 1 AND time2 = 1 AND author='foo'");
+        execute("SELECT content FROM %s WHERE time1 = 1 AND time2 > 0 AND author='foo'");
+    }
+
+    /**
+     * Test for LIMIT bugs from 4579,
+     * migrated from cql_tests.py:TestCQL.limit_bugs_test()
+     */
+    @Test
+    public void testLimitBug() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int, b int, c int, d int, e int, PRIMARY KEY (a, b))");
+
+        execute("INSERT INTO %s (a, b, c, d, e) VALUES (1, 1, 1, 1, 1);");
+        execute("INSERT INTO %s (a, b, c, d, e) VALUES (2, 2, 2, 2, 2);");
+        execute("INSERT INTO %s (a, b, c, d, e) VALUES (3, 3, 3, 3, 3);");
+        execute("INSERT INTO %s (a, b, c, d, e) VALUES (4, 4, 4, 4, 4);");
+
+        assertRows(execute("SELECT * FROM %s"),
+                   row(1, 1, 1, 1, 1),
+                   row(2, 2, 2, 2, 2),
+                   row(3, 3, 3, 3, 3),
+                   row(4, 4, 4, 4, 4));
+
+        assertRows(execute("SELECT * FROM %s LIMIT 1"),
+                   row(1, 1, 1, 1, 1));
+
+        assertRows(execute("SELECT * FROM %s LIMIT 2"),
+                   row(1, 1, 1, 1, 1),
+                   row(2, 2, 2, 2, 2));
+
+        createTable("CREATE TABLE %s (a int primary key, b int, c int,)");
+
+        execute("INSERT INTO %s (a, b, c) VALUES (1, 1, 1)");
+        execute("INSERT INTO %s (a, b, c) VALUES (2, 2, 2)");
+        execute("INSERT INTO %s (a, b, c) VALUES (3, 3, 3)");
+        execute("INSERT INTO %s (a, b, c) VALUES (4, 4, 4)");
+
+        assertRows(execute("SELECT * FROM %s"),
+                   row(1, 1, 1),
+                   row(2, 2, 2),
+                   row(3, 3, 3),
+                   row(4, 4, 4));
+
+        assertRows(execute("SELECT * FROM %s LIMIT 1"),
+                   row(1, 1, 1));
+
+        assertRows(execute("SELECT * FROM %s LIMIT 2"),
+                   row(1, 1, 1),
+                   row(2, 2, 2));
+
+        assertRows(execute("SELECT * FROM %s LIMIT 3"),
+                   row(1, 1, 1),
+                   row(2, 2, 2),
+                   row(3, 3, 3));
+
+        assertRows(execute("SELECT * FROM %s LIMIT 4"),
+                   row(1, 1, 1),
+                   row(2, 2, 2),
+                   row(3, 3, 3),
+                   row(4, 4, 4));
+
+        assertRows(execute("SELECT * FROM %s LIMIT 5"),
+                   row(1, 1, 1),
+                   row(2, 2, 2),
+                   row(3, 3, 3),
+                   row(4, 4, 4));
+    }
+
+    /**
+     * Test for #4612 bug and more generally order by when multiple C* rows are queried
+     * migrated from cql_tests.py:TestCQL.order_by_multikey_test()
+     */
+    @Test
+    public void testOrderByMultikey() throws Throwable
+    {
+        createTable("CREATE TABLE %s (my_id varchar, col1 int, col2 int, value varchar, PRIMARY KEY (my_id, col1, col2))");
+
+        execute("INSERT INTO %s (my_id, col1, col2, value) VALUES ( 'key1', 1, 1, 'a');");
+        execute("INSERT INTO %s (my_id, col1, col2, value) VALUES ( 'key2', 3, 3, 'a');");
+        execute("INSERT INTO %s (my_id, col1, col2, value) VALUES ( 'key3', 2, 2, 'b');");
+        execute("INSERT INTO %s (my_id, col1, col2, value) VALUES ( 'key4', 2, 1, 'b');");
+
+        assertRows(execute("SELECT col1 FROM %s WHERE my_id in('key1', 'key2', 'key3') ORDER BY col1"),
+                   row(1), row(2), row(3));
+
+        assertRows(execute("SELECT col1, value, my_id, col2 FROM %s WHERE my_id in('key3', 'key4') ORDER BY col1, col2"),
+                   row(2, "b", "key4", 1), row(2, "b", "key3", 2));
+
+        assertInvalid("SELECT col1 FROM %s ORDER BY col1");
+        assertInvalid("SELECT col1 FROM %s WHERE my_id > 'key1' ORDER BY col1");
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.composite_index_collections_test()
+     */
+    @Test
+    public void testIndexOnCompositeWithCollections() throws Throwable
+    {
+        createTable("CREATE TABLE %s (blog_id int, time1 int, time2 int, author text, content set<text>, PRIMARY KEY (blog_id, time1, time2))");
+
+        createIndex("CREATE INDEX ON %s (author)");
+
+        execute("INSERT INTO %s (blog_id, time1, time2, author, content) VALUES (?, ?, ?, ?, { 'bar1', 'bar2' })", 1, 0, 0, "foo");
+        execute("INSERT INTO %s (blog_id, time1, time2, author, content) VALUES (?, ?, ?, ?, { 'bar2', 'bar3' })", 1, 0, 1, "foo");
+        execute("INSERT INTO %s (blog_id, time1, time2, author, content) VALUES (?, ?, ?, ?, { 'baz' })", 2, 1, 0, "foo");
+        execute("INSERT INTO %s (blog_id, time1, time2, author, content) VALUES (?, ?, ?, ?, { 'qux' })", 3, 0, 1, "gux");
+
+        assertRows(execute("SELECT blog_id, content FROM %s WHERE author='foo'"),
+                   row(1, set("bar1", "bar2")),
+                   row(1, set("bar2", "bar3")),
+                   row(2, set("baz")));
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.truncate_clean_cache_test()
+     */
+    @Test
+    public void testTruncateWithCaching() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, v1 int, v2 int,) WITH CACHING = ALL;");
+
+        for (int i = 0; i < 3; i++)
+            execute("INSERT INTO %s (k, v1, v2) VALUES (?, ?, ?)", i, i, i * 2);
+
+        assertRows(execute("SELECT v1, v2 FROM %s WHERE k IN (0, 1, 2)"),
+                   row(0, 0),
+                   row(1, 2),
+                   row(2, 4));
+
+        execute("TRUNCATE %s");
+
+        assertEmpty(execute("SELECT v1, v2 FROM %s WHERE k IN (0, 1, 2)"));
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.range_key_ordered_test()
+     */
+    @Test
+    public void testRangeKey() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY)");
+
+        execute("INSERT INTO %s (k) VALUES (-1)");
+        execute("INSERT INTO %s (k) VALUES ( 0)");
+        execute("INSERT INTO %s (k) VALUES ( 1)");
+
+        assertRows(execute("SELECT * FROM %s"),
+                   row(0),
+                   row(1),
+                   row(-1));
+
+        assertInvalid("SELECT * FROM %s WHERE k >= -1 AND k < 1");
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f797bfa4/test/unit/org/apache/cassandra/cql3/validation/operations/SelectSingleColumnRelationTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/SelectSingleColumnRelationTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/SelectSingleColumnRelationTest.java
new file mode 100644
index 0000000..e950e7a
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/SelectSingleColumnRelationTest.java
@@ -0,0 +1,109 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.cql3.validation.operations;
+
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.cassandra.cql3.CQLTester;
+
+public class SelectSingleColumnRelationTest extends CQLTester
+{
+    @Test
+    public void testInvalidCollectionEqualityRelation() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int PRIMARY KEY, b set<int>, c list<int>, d map<int, int>)");
+        createIndex("CREATE INDEX ON %s (b)");
+        createIndex("CREATE INDEX ON %s (c)");
+        createIndex("CREATE INDEX ON %s (d)");
+
+        assertInvalid("SELECT * FROM %s WHERE a = 0 AND b=?", set(0));
+        assertInvalid("SELECT * FROM %s WHERE a = 0 AND c=?", list(0));
+        assertInvalid("SELECT * FROM %s WHERE a = 0 AND d=?", map(0, 0));
+    }
+
+    @Test
+    public void testInvalidCollectionNonEQRelation() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int PRIMARY KEY, b set<int>, c int)");
+        createIndex("CREATE INDEX ON %s (c)");
+        execute("INSERT INTO %s (a, b, c) VALUES (0, {0}, 0)");
+
+        // non-EQ operators
+        assertInvalid("SELECT * FROM %s WHERE c = 0 AND b > ?", set(0));
+        assertInvalid("SELECT * FROM %s WHERE c = 0 AND b >= ?", set(0));
+        assertInvalid("SELECT * FROM %s WHERE c = 0 AND b < ?", set(0));
+        assertInvalid("SELECT * FROM %s WHERE c = 0 AND b <= ?", set(0));
+        assertInvalid("SELECT * FROM %s WHERE c = 0 AND b IN (?)", set(0));
+    }
+
+    @Test
+    public void testLargeClusteringINValues() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int, c int, v int, PRIMARY KEY (k, c))");
+        execute("INSERT INTO %s (k, c, v) VALUES (0, 0, 0)");
+        List<Integer> inValues = new ArrayList<>(10000);
+        for (int i = 0; i < 10000; i++)
+            inValues.add(i);
+        assertRows(execute("SELECT * FROM %s WHERE k=? AND c IN ?", 0, inValues),
+                row(0, 0, 0)
+        );
+    }
+
+    @Test
+    public void testMultiplePartitionKeyWithIndex() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int, b int, c int, d int, e int, f int, PRIMARY KEY ((a, b), c, d, e))");
+        createIndex("CREATE INDEX ON %s (c)");
+        createIndex("CREATE INDEX ON %s (f)");
+
+        execute("INSERT INTO %s (a, b, c, d, e, f) VALUES (?, ?, ?, ?, ?, ?)", 0, 0, 0, 0, 0, 0);
+        execute("INSERT INTO %s (a, b, c, d, e, f) VALUES (?, ?, ?, ?, ?, ?)", 0, 0, 0, 1, 0, 1);
+        execute("INSERT INTO %s (a, b, c, d, e, f) VALUES (?, ?, ?, ?, ?, ?)", 0, 0, 0, 1, 1, 2);
+
+        execute("INSERT INTO %s (a, b, c, d, e, f) VALUES (?, ?, ?, ?, ?, ?)", 0, 0, 1, 0, 0, 3);
+        execute("INSERT INTO %s (a, b, c, d, e, f) VALUES (?, ?, ?, ?, ?, ?)", 0, 0, 1, 1, 0, 4);
+        execute("INSERT INTO %s (a, b, c, d, e, f) VALUES (?, ?, ?, ?, ?, ?)", 0, 0, 1, 1, 1, 5);
+
+        execute("INSERT INTO %s (a, b, c, d, e, f) VALUES (?, ?, ?, ?, ?, ?)", 0, 0, 2, 0, 0, 5);
+
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND c = ? ALLOW FILTERING", 0, 1),
+                   row(0, 0, 1, 0, 0, 3),
+                   row(0, 0, 1, 1, 0, 4),
+                   row(0, 0, 1, 1, 1, 5));
+
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND c = ? AND d = ? ALLOW FILTERING", 0, 1, 1),
+                   row(0, 0, 1, 1, 0, 4),
+                   row(0, 0, 1, 1, 1, 5));
+
+        assertInvalidMessage("Partition key part b must be restricted since preceding part is",
+                             "SELECT * FROM %s WHERE a = ? AND c >= ? ALLOW FILTERING", 0, 1);
+
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND c >= ? AND f = ? ALLOW FILTERING", 0, 1, 5),
+                   row(0, 0, 1, 1, 1, 5),
+                   row(0, 0, 2, 0, 0, 5));
+
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND c = ? AND d >= ? AND f = ? ALLOW FILTERING", 0, 1, 1, 5),
+                   row(0, 0, 1, 1, 1, 5));
+
+        assertInvalidMessage("Cannot execute this query as it might involve data filtering and thus may have unpredictable performance. If you want to execute this query despite the performance unpredictability, use ALLOW FILTERING",
+                             "SELECT * FROM %s WHERE a = ? AND d >= ? AND f = ?", 0, 1, 5);
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f797bfa4/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
new file mode 100644
index 0000000..c8c1f1a
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/SelectTest.java
@@ -0,0 +1,1238 @@
+/*
+ * 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.validation.operations;
+
+import java.nio.ByteBuffer;
+import java.util.UUID;
+
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import junit.framework.Assert;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.cql3.UntypedResultSet;
+import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.dht.Murmur3Partitioner;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Test column ranges and ordering with static column in table
+ */
+public class SelectTest extends CQLTester
+{
+    @BeforeClass
+    public static void setUp()
+    {
+        DatabaseDescriptor.setPartitioner(new Murmur3Partitioner());
+    }
+
+    @Test
+    public void testSingleClustering() throws Throwable
+    {
+        createTable("CREATE TABLE %s (p text, c text, v text, s text static, PRIMARY KEY (p, c))");
+
+        execute("INSERT INTO %s(p, c, v, s) values (?, ?, ?, ?)", "p1", "k1", "v1", "sv1");
+        execute("INSERT INTO %s(p, c, v) values (?, ?, ?)", "p1", "k2", "v2");
+        execute("INSERT INTO %s(p, s) values (?, ?)", "p2", "sv2");
+
+        assertRows(execute("SELECT * FROM %s WHERE p=?", "p1"),
+            row("p1", "k1", "sv1", "v1"),
+            row("p1", "k2", "sv1", "v2")
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE p=?", "p2"),
+            row("p2", null, "sv2", null)
+        );
+
+        // Ascending order
+
+        assertRows(execute("SELECT * FROM %s WHERE p=? ORDER BY c ASC", "p1"),
+            row("p1", "k1", "sv1", "v1"),
+            row("p1", "k2", "sv1", "v2")
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE p=? ORDER BY c ASC", "p2"),
+            row("p2", null, "sv2", null)
+        );
+
+        // Descending order
+
+        assertRows(execute("SELECT * FROM %s WHERE p=? ORDER BY c DESC", "p1"),
+            row("p1", "k2", "sv1", "v2"),
+            row("p1", "k1", "sv1", "v1")
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE p=? ORDER BY c DESC", "p2"),
+            row("p2", null, "sv2", null)
+        );
+
+        // No order with one relation
+
+        assertRows(execute("SELECT * FROM %s WHERE p=? AND c>=?", "p1", "k1"),
+            row("p1", "k1", "sv1", "v1"),
+            row("p1", "k2", "sv1", "v2")
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE p=? AND c>=?", "p1", "k2"),
+            row("p1", "k2", "sv1", "v2")
+        );
+
+        assertEmpty(execute("SELECT * FROM %s WHERE p=? AND c>=?", "p1", "k3"));
+
+        assertRows(execute("SELECT * FROM %s WHERE p=? AND c =?", "p1", "k1"),
+            row("p1", "k1", "sv1", "v1")
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE p=? AND c<=?", "p1", "k1"),
+            row("p1", "k1", "sv1", "v1")
+        );
+
+        assertEmpty(execute("SELECT * FROM %s WHERE p=? AND c<=?", "p1", "k0"));
+
+        // Ascending with one relation
+
+        assertRows(execute("SELECT * FROM %s WHERE p=? AND c>=? ORDER BY c ASC", "p1", "k1"),
+            row("p1", "k1", "sv1", "v1"),
+            row("p1", "k2", "sv1", "v2")
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE p=? AND c>=? ORDER BY c ASC", "p1", "k2"),
+            row("p1", "k2", "sv1", "v2")
+        );
+
+        assertEmpty(execute("SELECT * FROM %s WHERE p=? AND c>=? ORDER BY c ASC", "p1", "k3"));
+
+        assertRows(execute("SELECT * FROM %s WHERE p=? AND c =? ORDER BY c ASC", "p1", "k1"),
+            row("p1", "k1", "sv1", "v1")
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE p=? AND c<=? ORDER BY c ASC", "p1", "k1"),
+            row("p1", "k1", "sv1", "v1")
+        );
+
+        assertEmpty(execute("SELECT * FROM %s WHERE p=? AND c<=? ORDER BY c ASC", "p1", "k0"));
+
+        // Descending with one relation
+
+        assertRows(execute("SELECT * FROM %s WHERE p=? AND c>=? ORDER BY c DESC", "p1", "k1"),
+            row("p1", "k2", "sv1", "v2"),
+            row("p1", "k1", "sv1", "v1")
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE p=? AND c>=? ORDER BY c DESC", "p1", "k2"),
+            row("p1", "k2", "sv1", "v2")
+        );
+
+        assertEmpty(execute("SELECT * FROM %s WHERE p=? AND c>=? ORDER BY c DESC", "p1", "k3"));
+
+        assertRows(execute("SELECT * FROM %s WHERE p=? AND c =? ORDER BY c DESC", "p1", "k1"),
+            row("p1", "k1", "sv1", "v1")
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE p=? AND c<=? ORDER BY c DESC", "p1", "k1"),
+            row("p1", "k1", "sv1", "v1")
+        );
+
+        assertEmpty(execute("SELECT * FROM %s WHERE p=? AND c<=? ORDER BY c DESC", "p1", "k0"));
+
+        // IN
+
+        assertRows(execute("SELECT * FROM %s WHERE p=? AND c IN (?, ?)", "p1", "k1", "k2"),
+            row("p1", "k1", "sv1", "v1"),
+            row("p1", "k2", "sv1", "v2")
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE p=? AND c IN (?, ?) ORDER BY c ASC", "p1", "k1", "k2"),
+            row("p1", "k1", "sv1", "v1"),
+            row("p1", "k2", "sv1", "v2")
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE p=? AND c IN (?, ?) ORDER BY c DESC", "p1", "k1", "k2"),
+            row("p1", "k2", "sv1", "v2"),
+            row("p1", "k1", "sv1", "v1")
+        );
+    }
+
+    @Test
+    public void testSingleClusteringReversed() throws Throwable
+    {
+        createTable("CREATE TABLE %s (p text, c text, v text, s text static, PRIMARY KEY (p, c)) WITH CLUSTERING ORDER BY (c DESC)");
+
+        execute("INSERT INTO %s(p, c, v, s) values (?, ?, ?, ?)", "p1", "k1", "v1", "sv1");
+        execute("INSERT INTO %s(p, c, v) values (?, ?, ?)", "p1", "k2", "v2");
+        execute("INSERT INTO %s(p, s) values (?, ?)", "p2", "sv2");
+
+        assertRows(execute("SELECT * FROM %s WHERE p=?", "p1"),
+            row("p1", "k2", "sv1", "v2"),
+            row("p1", "k1", "sv1", "v1")
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE p=?", "p2"),
+            row("p2", null, "sv2", null)
+        );
+
+        // Ascending order
+
+        assertRows(execute("SELECT * FROM %s WHERE p=? ORDER BY c ASC", "p1"),
+            row("p1", "k1", "sv1", "v1"),
+            row("p1", "k2", "sv1", "v2")
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE p=? ORDER BY c ASC", "p2"),
+            row("p2", null, "sv2", null)
+        );
+
+        // Descending order
+
+        assertRows(execute("SELECT * FROM %s WHERE p=? ORDER BY c DESC", "p1"),
+            row("p1", "k2", "sv1", "v2"),
+            row("p1", "k1", "sv1", "v1")
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE p=? ORDER BY c DESC", "p2"),
+            row("p2", null, "sv2", null)
+        );
+
+        // No order with one relation
+
+        assertRows(execute("SELECT * FROM %s WHERE p=? AND c>=?", "p1", "k1"),
+            row("p1", "k2", "sv1", "v2"),
+            row("p1", "k1", "sv1", "v1")
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE p=? AND c>=?", "p1", "k2"),
+            row("p1", "k2", "sv1", "v2")
+        );
+
+        assertEmpty(execute("SELECT * FROM %s WHERE p=? AND c>=?", "p1", "k3"));
+
+        assertRows(execute("SELECT * FROM %s WHERE p=? AND c=?", "p1", "k1"),
+            row("p1", "k1", "sv1", "v1")
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE p=? AND c<=?", "p1", "k1"),
+            row("p1", "k1", "sv1", "v1")
+        );
+
+        assertEmpty(execute("SELECT * FROM %s WHERE p=? AND c<=?", "p1", "k0"));
+
+        // Ascending with one relation
+
+        assertRows(execute("SELECT * FROM %s WHERE p=? AND c>=? ORDER BY c ASC", "p1", "k1"),
+            row("p1", "k1", "sv1", "v1"),
+            row("p1", "k2", "sv1", "v2")
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE p=? AND c>=? ORDER BY c ASC", "p1", "k2"),
+            row("p1", "k2", "sv1", "v2")
+        );
+
+        assertEmpty(execute("SELECT * FROM %s WHERE p=? AND c>=? ORDER BY c ASC", "p1", "k3"));
+
+        assertRows(execute("SELECT * FROM %s WHERE p=? AND c=? ORDER BY c ASC", "p1", "k1"),
+            row("p1", "k1", "sv1", "v1")
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE p=? AND c<=? ORDER BY c ASC", "p1", "k1"),
+            row("p1", "k1", "sv1", "v1")
+        );
+
+        assertEmpty(execute("SELECT * FROM %s WHERE p=? AND c<=? ORDER BY c ASC", "p1", "k0"));
+
+        // Descending with one relation
+
+        assertRows(execute("SELECT * FROM %s WHERE p=? AND c>=? ORDER BY c DESC", "p1", "k1"),
+            row("p1", "k2", "sv1", "v2"),
+            row("p1", "k1", "sv1", "v1")
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE p=? AND c>=? ORDER BY c DESC", "p1", "k2"),
+            row("p1", "k2", "sv1", "v2")
+        );
+
+        assertEmpty(execute("SELECT * FROM %s WHERE p=? AND c>=? ORDER BY c DESC", "p1", "k3"));
+
+        assertRows(execute("SELECT * FROM %s WHERE p=? AND c=? ORDER BY c DESC", "p1", "k1"),
+            row("p1", "k1", "sv1", "v1")
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE p=? AND c<=? ORDER BY c DESC", "p1", "k1"),
+            row("p1", "k1", "sv1", "v1")
+        );
+
+        assertEmpty(execute("SELECT * FROM %s WHERE p=? AND c<=? ORDER BY c DESC", "p1", "k0"));
+
+        // IN
+
+        assertRows(execute("SELECT * FROM %s WHERE p=? AND c IN (?, ?)", "p1", "k1", "k2"),
+            row("p1", "k2", "sv1", "v2"),
+            row("p1", "k1", "sv1", "v1")
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE p=? AND c IN (?, ?) ORDER BY c ASC", "p1", "k1", "k2"),
+            row("p1", "k1", "sv1", "v1"),
+            row("p1", "k2", "sv1", "v2")
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE p=? AND c IN (?, ?) ORDER BY c DESC", "p1", "k1", "k2"),
+            row("p1", "k2", "sv1", "v2"),
+            row("p1", "k1", "sv1", "v1")
+        );
+    }
+
+    /**
+     * Check query with KEY IN clause
+     * migrated from cql_tests.py:TestCQL.select_key_in_test()
+     */
+    @Test
+    public void testSelectKeyIn() throws Throwable
+    {
+        createTable("CREATE TABLE %s (userid uuid PRIMARY KEY, firstname text, lastname text, age int)");
+
+        UUID id1 = UUID.fromString("550e8400-e29b-41d4-a716-446655440000");
+        UUID id2 = UUID.fromString("f47ac10b-58cc-4372-a567-0e02b2c3d479");
+
+        execute("INSERT INTO %s (userid, firstname, lastname, age) VALUES (?, 'Frodo', 'Baggins', 32)", id1);
+        execute("INSERT INTO %s (userid, firstname, lastname, age) VALUES (?, 'Samwise', 'Gamgee', 33)", id2);
+
+        assertRowCount(execute("SELECT firstname, lastname FROM %s WHERE userid IN (?, ?)", id1, id2), 2);
+    }
+
+    /**
+     * Check query with KEY IN clause for wide row tables
+     * migrated from cql_tests.py:TestCQL.in_clause_wide_rows_test()
+     */
+    @Test
+    public void testSelectKeyInForWideRows() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int, c int, v int, PRIMARY KEY (k, c)) WITH COMPACT STORAGE");
+
+        for (int i = 0; i < 10; i++)
+            execute("INSERT INTO %s (k, c, v) VALUES (0, ?, ?)", i, i);
+
+        assertRows(execute("SELECT v FROM %s WHERE k = 0 AND c IN (5, 2, 8)"),
+                   row(2), row(5), row(8));
+
+        createTable("CREATE TABLE %s (k int, c1 int, c2 int, v int, PRIMARY KEY (k, c1, c2)) WITH COMPACT STORAGE");
+
+        for (int i = 0; i < 10; i++)
+            execute("INSERT INTO %s (k, c1, c2, v) VALUES (0, 0, ?, ?)", i, i);
+
+        assertInvalid("SELECT v FROM %s WHERE k = 0 AND c1 IN (5, 2, 8) AND c2 = 3");
+
+        assertRows(execute("SELECT v FROM %s WHERE k = 0 AND c1 = 0 AND c2 IN (5, 2, 8)"),
+                   row(2), row(5), row(8));
+    }
+
+    /**
+     * Check SELECT respects inclusive and exclusive bounds
+     * migrated from cql_tests.py:TestCQL.exclusive_slice_test()
+     */
+    @Test
+    public void testSelectBounds() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int, c int, v int, PRIMARY KEY (k, c)) WITH COMPACT STORAGE");
+
+        for (int i = 0; i < 10; i++)
+            execute("INSERT INTO %s (k, c, v) VALUES (0, ?, ?)", i, i);
+
+        assertRowCount(execute("SELECT v FROM %s WHERE k = 0"), 10);
+
+        assertRows(execute("SELECT v FROM %s WHERE k = 0 AND c >= 2 AND c <= 6"),
+                   row(2), row(3), row(4), row(5), row(6));
+
+        assertRows(execute("SELECT v FROM %s WHERE k = 0 AND c > 2 AND c <= 6"),
+                   row(3), row(4), row(5), row(6));
+
+        assertRows(execute("SELECT v FROM %s WHERE k = 0 AND c >= 2 AND c < 6"),
+                   row(2), row(3), row(4), row(5));
+
+        assertRows(execute("SELECT v FROM %s WHERE k = 0 AND c > 2 AND c < 6"),
+                   row(3), row(4), row(5));
+
+        assertRows(execute("SELECT v FROM %s WHERE k = 0 AND c > 2 AND c <= 6 LIMIT 2"),
+                   row(3), row(4));
+
+        assertRows(execute("SELECT v FROM %s WHERE k = 0 AND c >= 2 AND c < 6 ORDER BY c DESC LIMIT 2"),
+                   row(5), row(4));
+    }
+
+    @Test
+    public void testSetContains() throws Throwable
+    {
+        createTable("CREATE TABLE %s (account text, id int, categories set<text>, PRIMARY KEY (account, id))");
+        createIndex("CREATE INDEX ON %s(categories)");
+
+        execute("INSERT INTO %s (account, id , categories) VALUES (?, ?, ?)", "test", 5, set("lmn"));
+
+        assertEmpty(execute("SELECT * FROM %s WHERE account = ? AND categories CONTAINS ?", "xyz", "lmn"));
+
+        assertRows(execute("SELECT * FROM %s WHERE categories CONTAINS ?", "lmn"),
+                   row("test", 5, set("lmn"))
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE account = ? AND categories CONTAINS ?", "test", "lmn"),
+                   row("test", 5, set("lmn"))
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE account = ? AND id = ? AND categories CONTAINS ?", "test", 5, "lmn"),
+                   row("test", 5, set("lmn"))
+        );
+
+        assertInvalid("SELECT * FROM %s WHERE account = ? AND categories CONTAINS ? AND categories CONTAINS ?", "xyz", "lmn", "notPresent");
+        assertEmpty(execute("SELECT * FROM %s WHERE account = ? AND categories CONTAINS ? AND categories CONTAINS ? ALLOW FILTERING", "xyz", "lmn", "notPresent"));
+    }
+
+    @Test
+    public void testListContains() throws Throwable
+    {
+        createTable("CREATE TABLE %s (account text, id int, categories list<text>, PRIMARY KEY (account, id))");
+        createIndex("CREATE INDEX ON %s(categories)");
+
+        execute("INSERT INTO %s (account, id , categories) VALUES (?, ?, ?)", "test", 5, list("lmn"));
+
+        assertEmpty(execute("SELECT * FROM %s WHERE account = ? AND categories CONTAINS ?", "xyz", "lmn"));
+
+        assertRows(execute("SELECT * FROM %s WHERE account = ? AND categories CONTAINS ?;", "test", "lmn"),
+                   row("test", 5, list("lmn"))
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE categories CONTAINS ?", "lmn"),
+                   row("test", 5, list("lmn"))
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE account = ? AND id = ? AND categories CONTAINS ?;", "test", 5, "lmn"),
+                   row("test", 5, list("lmn"))
+        );
+
+        assertInvalid("SELECT * FROM %s WHERE account = ? AND id = ? AND categories CONTAINS ? AND categories CONTAINS ?",
+                      "test", 5, "lmn", "notPresent");
+        assertEmpty(execute("SELECT * FROM %s WHERE account = ? AND id = ? AND categories CONTAINS ? AND categories CONTAINS ? ALLOW FILTERING",
+                            "test", 5, "lmn", "notPresent"));
+    }
+
+    @Test
+    public void testListContainsWithFiltering() throws Throwable
+    {
+        createTable("CREATE TABLE %s (e int PRIMARY KEY, f list<text>, s int)");
+        createIndex("CREATE INDEX ON %s(f)");
+        for(int i = 0; i < 3; i++)
+        {
+            execute("INSERT INTO %s (e, f, s) VALUES (?, ?, ?)", i, list("Dubai"), 4);
+        }
+        for(int i = 3; i < 5; i++)
+        {
+            execute("INSERT INTO %s (e, f, s) VALUES (?, ?, ?)", i, list("Dubai"), 3);
+        }
+        assertRows(execute("SELECT * FROM %s WHERE f CONTAINS ? AND s=? allow filtering", "Dubai", 3),
+                   row(4, list("Dubai"), 3),
+                   row(3, list("Dubai"), 3));
+    }
+
+    @Test
+    public void testMapKeyContains() throws Throwable
+    {
+        createTable("CREATE TABLE %s (account text, id int, categories map<text,text>, PRIMARY KEY (account, id))");
+        createIndex("CREATE INDEX ON %s(keys(categories))");
+
+        execute("INSERT INTO %s (account, id , categories) VALUES (?, ?, ?)", "test", 5, map("lmn", "foo"));
+
+        assertEmpty(execute("SELECT * FROM %s WHERE account = ? AND categories CONTAINS KEY ?", "xyz", "lmn"));
+
+        assertRows(execute("SELECT * FROM %s WHERE account = ? AND categories CONTAINS KEY ?", "test", "lmn"),
+                   row("test", 5, map("lmn", "foo"))
+        );
+        assertRows(execute("SELECT * FROM %s WHERE categories CONTAINS KEY ?", "lmn"),
+                   row("test", 5, map("lmn", "foo"))
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE account = ? AND id = ? AND categories CONTAINS KEY ?", "test", 5, "lmn"),
+                   row("test", 5, map("lmn", "foo"))
+        );
+
+        assertInvalid("SELECT * FROM %s WHERE account = ? AND id = ? AND categories CONTAINS KEY ? AND categories CONTAINS KEY ?",
+                      "test", 5, "lmn", "notPresent");
+        assertEmpty(execute("SELECT * FROM %s WHERE account = ? AND id = ? AND categories CONTAINS KEY ? AND categories CONTAINS KEY ? ALLOW FILTERING",
+                            "test", 5, "lmn", "notPresent"));
+
+        assertInvalid("SELECT * FROM %s WHERE account = ? AND id = ? AND categories CONTAINS KEY ? AND categories CONTAINS ?",
+                      "test", 5, "lmn", "foo");
+    }
+
+    @Test
+    public void testMapValueContains() throws Throwable
+    {
+        createTable("CREATE TABLE %s (account text, id int, categories map<text,text>, PRIMARY KEY (account, id))");
+        createIndex("CREATE INDEX ON %s(categories)");
+
+        execute("INSERT INTO %s (account, id , categories) VALUES (?, ?, ?)", "test", 5, map("lmn", "foo"));
+
+        assertEmpty(execute("SELECT * FROM %s WHERE account = ? AND categories CONTAINS ?", "xyz", "foo"));
+
+        assertRows(execute("SELECT * FROM %s WHERE account = ? AND categories CONTAINS ?", "test", "foo"),
+                   row("test", 5, map("lmn", "foo"))
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE categories CONTAINS ?", "foo"),
+                   row("test", 5, map("lmn", "foo"))
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE account = ? AND id = ? AND categories CONTAINS ?", "test", 5, "foo"),
+                   row("test", 5, map("lmn", "foo"))
+        );
+
+        assertInvalid("SELECT * FROM %s WHERE account = ? AND id = ? AND categories CONTAINS ? AND categories CONTAINS ?"
+                     , "test", 5, "foo", "notPresent");
+
+        assertEmpty(execute("SELECT * FROM %s WHERE account = ? AND id = ? AND categories CONTAINS ? AND categories CONTAINS ? ALLOW FILTERING"
+                           , "test", 5, "foo", "notPresent"));
+    }
+
+    // See CASSANDRA-7525
+    @Test
+    public void testQueryMultipleIndexTypes() throws Throwable
+    {
+        createTable("CREATE TABLE %s (account text, id int, categories map<text,text>, PRIMARY KEY (account, id))");
+
+        // create an index on
+        createIndex("CREATE INDEX id_index ON %s(id)");
+        createIndex("CREATE INDEX categories_values_index ON %s(categories)");
+
+        execute("INSERT INTO %s (account, id , categories) VALUES (?, ?, ?)", "test", 5, map("lmn", "foo"));
+
+        assertRows(execute("SELECT * FROM %s WHERE categories CONTAINS ? AND id = ? ALLOW FILTERING", "foo", 5),
+                   row("test", 5, map("lmn", "foo"))
+        );
+
+        assertRows(
+                  execute("SELECT * FROM %s WHERE account = ? AND categories CONTAINS ? AND id = ? ALLOW FILTERING", "test", "foo", 5),
+                  row("test", 5, map("lmn", "foo"))
+        );
+    }
+
+    // See CASSANDRA-8033
+    @Test
+    public void testFilterForContains() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k1 int, k2 int, v set<int>, PRIMARY KEY ((k1, k2)))");
+        createIndex("CREATE INDEX ON %s(k2)");
+
+        execute("INSERT INTO %s (k1, k2, v) VALUES (?, ?, ?)", 0, 0, set(1, 2, 3));
+        execute("INSERT INTO %s (k1, k2, v) VALUES (?, ?, ?)", 0, 1, set(2, 3, 4));
+        execute("INSERT INTO %s (k1, k2, v) VALUES (?, ?, ?)", 1, 0, set(3, 4, 5));
+        execute("INSERT INTO %s (k1, k2, v) VALUES (?, ?, ?)", 1, 1, set(4, 5, 6));
+
+        assertRows(execute("SELECT * FROM %s WHERE k2 = ?", 1),
+                   row(0, 1, set(2, 3, 4)),
+                   row(1, 1, set(4, 5, 6))
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE k2 = ? AND v CONTAINS ? ALLOW FILTERING", 1, 6),
+                   row(1, 1, set(4, 5, 6))
+        );
+
+        assertEmpty(execute("SELECT * FROM %s WHERE k2 = ? AND v CONTAINS ? ALLOW FILTERING", 1, 7));
+    }
+
+    // See CASSANDRA-8073
+    @Test
+    public void testIndexLookupWithClusteringPrefix() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int, b int, c int, d set<int>, PRIMARY KEY (a, b, c))");
+        createIndex("CREATE INDEX ON %s(d)");
+        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 0, set(1, 2, 3));
+        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 1, set(3, 4, 5));
+        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 0, set(1, 2, 3));
+        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 1, set(3, 4, 5));
+
+        assertRows(execute("SELECT * FROM %s WHERE a=? AND b=? AND d CONTAINS ?", 0, 1, 3),
+                   row(0, 1, 0, set(1, 2, 3)),
+                   row(0, 1, 1, set(3, 4, 5))
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE a=? AND b=? AND d CONTAINS ?", 0, 1, 2),
+                   row(0, 1, 0, set(1, 2, 3))
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE a=? AND b=? AND d CONTAINS ?", 0, 1, 5),
+                   row(0, 1, 1, set(3, 4, 5))
+        );
+    }
+
+    @Test
+    public void testContainsKeyAndContainsWithIndexOnMapKey() throws Throwable
+    {
+        createTable("CREATE TABLE %s (account text, id int, categories map<text,text>, PRIMARY KEY (account, id))");
+        createIndex("CREATE INDEX ON %s(keys(categories))");
+
+        execute("INSERT INTO %s (account, id , categories) VALUES (?, ?, ?)", "test", 5, map("lmn", "foo"));
+        execute("INSERT INTO %s (account, id , categories) VALUES (?, ?, ?)", "test", 6, map("lmn", "foo2"));
+
+        assertInvalid("SELECT * FROM %s WHERE account = ? AND categories CONTAINS ?", "test", "foo");
+
+        assertRows(execute("SELECT * FROM %s WHERE account = ? AND categories CONTAINS KEY ?", "test", "lmn"),
+                   row("test", 5, map("lmn", "foo")),
+                   row("test", 6, map("lmn", "foo2")));
+        assertRows(execute("SELECT * FROM %s WHERE account = ? AND categories CONTAINS KEY ? AND categories CONTAINS ? ALLOW FILTERING",
+                           "test", "lmn", "foo"),
+                   row("test", 5, map("lmn", "foo")));
+        assertRows(execute("SELECT * FROM %s WHERE account = ? AND categories CONTAINS ? AND categories CONTAINS KEY ? ALLOW FILTERING",
+                           "test", "foo", "lmn"),
+                   row("test", 5, map("lmn", "foo")));
+    }
+
+    @Test
+    public void testContainsKeyAndContainsWithIndexOnMapValue() throws Throwable
+    {
+        createTable("CREATE TABLE %s (account text, id int, categories map<text,text>, PRIMARY KEY (account, id))");
+        createIndex("CREATE INDEX ON %s(categories)");
+
+        execute("INSERT INTO %s (account, id , categories) VALUES (?, ?, ?)", "test", 5, map("lmn", "foo"));
+        execute("INSERT INTO %s (account, id , categories) VALUES (?, ?, ?)", "test", 6, map("lmn2", "foo"));
+
+        assertInvalid("SELECT * FROM %s WHERE account = ? AND categories CONTAINS KEY ?", "test", "lmn");
+
+        assertRows(execute("SELECT * FROM %s WHERE account = ? AND categories CONTAINS ?", "test", "foo"),
+                   row("test", 5, map("lmn", "foo")),
+                   row("test", 6, map("lmn2", "foo")));
+        assertRows(execute("SELECT * FROM %s WHERE account = ? AND categories CONTAINS KEY ? AND categories CONTAINS ? ALLOW FILTERING",
+                           "test", "lmn", "foo"),
+                   row("test", 5, map("lmn", "foo")));
+        assertRows(execute("SELECT * FROM %s WHERE account = ? AND categories CONTAINS ? AND categories CONTAINS KEY ? ALLOW FILTERING",
+                           "test", "foo", "lmn"),
+                   row("test", 5, map("lmn", "foo")));
+    }
+
+    /**
+     * Test token ranges
+     * migrated from cql_tests.py:TestCQL.token_range_test()
+     */
+    @Test
+    public void testTokenRange() throws Throwable
+    {
+        createTable(" CREATE TABLE %s (k int PRIMARY KEY, c int, v int)");
+
+        int c = 100;
+        for (int i = 0; i < c; i++)
+            execute("INSERT INTO %s (k, c, v) VALUES (?, ?, ?)", i, i, i);
+
+        Object[][] res = getRows(execute("SELECT k FROM %s"));
+        assertEquals(c, res.length);
+
+        Object[] inOrder = new Object[res.length];
+        for (int i = 0; i < res.length; i++)
+            inOrder[i] = res[i][0];
+
+        Long min_token = Long.MIN_VALUE;
+
+        res = getRows(execute(String.format("SELECT k FROM %s.%s WHERE token(k) >= %d",
+                                            keyspace(), currentTable(), min_token)));
+        assertEquals(c, res.length);
+
+        res = getRows(execute(String.format("SELECT k FROM %s.%s WHERE token(k) >= token(%d) AND token(k) < token(%d)",
+                                            keyspace(), currentTable(), inOrder[32], inOrder[65])));
+
+        for (int i = 32; i < 65; i++)
+            Assert.assertEquals(inOrder[i], res[i - 32][0]);
+    }
+
+    /**
+     * Test select count
+     * migrated from cql_tests.py:TestCQL.count_test()
+     */
+    @Test
+    public void testSelectCount() throws Throwable
+    {
+        createTable(" CREATE TABLE %s (kind text, time int, value1 int, value2 int, PRIMARY KEY(kind, time))");
+
+        execute("INSERT INTO %s (kind, time, value1, value2) VALUES ('ev1', ?, ?, ?)", 0, 0, 0);
+        execute("INSERT INTO %s (kind, time, value1, value2) VALUES ('ev1', ?, ?, ?)", 1, 1, 1);
+        execute("INSERT INTO %s (kind, time, value1) VALUES ('ev1', ?, ?)", 2, 2);
+        execute("INSERT INTO %s (kind, time, value1, value2) VALUES ('ev1', ?, ?, ?)", 3, 3, 3);
+        execute("INSERT INTO %s (kind, time, value1) VALUES ('ev1', ?, ?)", 4, 4);
+        execute("INSERT INTO %s (kind, time, value1, value2) VALUES ('ev2', 0, 0, 0)");
+
+        assertRows(execute("SELECT COUNT(*) FROM %s WHERE kind = 'ev1'"),
+                   row(5L));
+
+        assertRows(execute("SELECT COUNT(1) FROM %s WHERE kind IN ('ev1', 'ev2') AND time=0"),
+                   row(2L));
+    }
+
+    /**
+     * Range test query from #4372
+     * migrated from cql_tests.py:TestCQL.range_query_test()
+     */
+    @Test
+    public void testRangeQuery() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int, b int, c int, d int, e int, f text, PRIMARY KEY (a, b, c, d, e) )");
+
+        execute("INSERT INTO %s (a, b, c, d, e, f) VALUES (1, 1, 1, 1, 2, '2')");
+        execute("INSERT INTO %s (a, b, c, d, e, f) VALUES (1, 1, 1, 1, 1, '1')");
+        execute("INSERT INTO %s (a, b, c, d, e, f) VALUES (1, 1, 1, 2, 1, '1')");
+        execute("INSERT INTO %s (a, b, c, d, e, f) VALUES (1, 1, 1, 1, 3, '3')");
+        execute("INSERT INTO %s (a, b, c, d, e, f) VALUES (1, 1, 1, 1, 5, '5')");
+
+        assertRows(execute("SELECT a, b, c, d, e, f FROM %s WHERE a = 1 AND b = 1 AND c = 1 AND d = 1 AND e >= 2"),
+                   row(1, 1, 1, 1, 2, "2"),
+                   row(1, 1, 1, 1, 3, "3"),
+                   row(1, 1, 1, 1, 5, "5"));
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.composite_row_key_test()
+     */
+    @Test
+    public void testCompositeRowKey() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k1 int, k2 int, c int, v int, PRIMARY KEY ((k1, k2), c))");
+
+        for (int i = 0; i < 4; i++)
+            execute("INSERT INTO %s (k1, k2, c, v) VALUES (?, ?, ?, ?)", 0, i, i, i);
+
+        assertRows(execute("SELECT * FROM %s"),
+                   row(0, 2, 2, 2),
+                   row(0, 3, 3, 3),
+                   row(0, 0, 0, 0),
+                   row(0, 1, 1, 1));
+
+        assertRows(execute("SELECT * FROM %s WHERE k1 = 0 and k2 IN (1, 3)"),
+                   row(0, 1, 1, 1),
+                   row(0, 3, 3, 3));
+
+        assertInvalid("SELECT * FROM %s WHERE k2 = 3");
+
+        assertRows(execute("SELECT * FROM %s WHERE token(k1, k2) = token(0, 1)"),
+                   row(0, 1, 1, 1));
+
+
+        assertRows(execute("SELECT * FROM %s WHERE token(k1, k2) > ?", Long.MIN_VALUE),
+                   row(0, 2, 2, 2),
+                   row(0, 3, 3, 3),
+                   row(0, 0, 0, 0),
+                   row(0, 1, 1, 1));
+    }
+
+    /**
+     * Test for #4532, NPE when trying to select a slice from a composite table
+     * migrated from cql_tests.py:TestCQL.bug_4532_test()
+     */
+    @Test
+    public void testSelectSliceFromComposite() throws Throwable
+    {
+        createTable("CREATE TABLE %s (status ascii, ctime bigint, key ascii, nil ascii, PRIMARY KEY (status, ctime, key))");
+
+        execute("INSERT INTO %s (status,ctime,key,nil) VALUES ('C',12345678,'key1','')");
+        execute("INSERT INTO %s (status,ctime,key,nil) VALUES ('C',12345678,'key2','')");
+        execute("INSERT INTO %s (status,ctime,key,nil) VALUES ('C',12345679,'key3','')");
+        execute("INSERT INTO %s (status,ctime,key,nil) VALUES ('C',12345679,'key4','')");
+        execute("INSERT INTO %s (status,ctime,key,nil) VALUES ('C',12345679,'key5','')");
+        execute("INSERT INTO %s (status,ctime,key,nil) VALUES ('C',12345680,'key6','')");
+
+        assertInvalid("SELECT * FROM %s WHERE ctime>=12345679 AND key='key3' AND ctime<=12345680 LIMIT 3;");
+        assertInvalid("SELECT * FROM %s WHERE ctime=12345679  AND key='key3' AND ctime<=12345680 LIMIT 3");
+    }
+
+    /**
+     * Test for #4716 bug and more generally for good behavior of ordering,
+     * migrated from cql_tests.py:TestCQL.reversed_compact_test()
+     */
+    @Test
+    public void testReverseCompact() throws Throwable
+    {
+        createTable("CREATE TABLE %s ( k text, c int, v int, PRIMARY KEY (k, c) ) WITH COMPACT STORAGE AND CLUSTERING ORDER BY (c DESC)");
+
+        for (int i = 0; i < 10; i++)
+            execute("INSERT INTO %s (k, c, v) VALUES ('foo', ?, ?)", i, i);
+
+        assertRows(execute("SELECT c FROM %s WHERE c > 2 AND c < 6 AND k = 'foo'"),
+                   row(5), row(4), row(3));
+
+        assertRows(execute("SELECT c FROM %s WHERE c >= 2 AND c <= 6 AND k = 'foo'"),
+                   row(6), row(5), row(4), row(3), row(2));
+
+        assertRows(execute("SELECT c FROM %s WHERE c > 2 AND c < 6 AND k = 'foo' ORDER BY c ASC"),
+                   row(3), row(4), row(5));
+
+        assertRows(execute("SELECT c FROM %s WHERE c >= 2 AND c <= 6 AND k = 'foo' ORDER BY c ASC"),
+                   row(2), row(3), row(4), row(5), row(6));
+
+        assertRows(execute("SELECT c FROM %s WHERE c > 2 AND c < 6 AND k = 'foo' ORDER BY c DESC"),
+                   row(5), row(4), row(3));
+
+        assertRows(execute("SELECT c FROM %s WHERE c >= 2 AND c <= 6 AND k = 'foo' ORDER BY c DESC"),
+                   row(6), row(5), row(4), row(3), row(2));
+
+        createTable("CREATE TABLE %s ( k text, c int, v int, PRIMARY KEY (k, c) ) WITH COMPACT STORAGE");
+
+        for (int i = 0; i < 10; i++)
+            execute("INSERT INTO %s(k, c, v) VALUES ('foo', ?, ?)", i, i);
+
+        assertRows(execute("SELECT c FROM %s WHERE c > 2 AND c < 6 AND k = 'foo'"),
+                   row(3), row(4), row(5));
+
+        assertRows(execute("SELECT c FROM %s WHERE c >= 2 AND c <= 6 AND k = 'foo'"),
+                   row(2), row(3), row(4), row(5), row(6));
+
+        assertRows(execute("SELECT c FROM %s WHERE c > 2 AND c < 6 AND k = 'foo' ORDER BY c ASC"),
+                   row(3), row(4), row(5));
+
+        assertRows(execute("SELECT c FROM %s WHERE c >= 2 AND c <= 6 AND k = 'foo' ORDER BY c ASC"),
+                   row(2), row(3), row(4), row(5), row(6));
+
+        assertRows(execute("SELECT c FROM %s WHERE c > 2 AND c < 6 AND k = 'foo' ORDER BY c DESC"),
+                   row(5), row(4), row(3));
+
+        assertRows(execute("SELECT c FROM %s WHERE c >= 2 AND c <= 6 AND k = 'foo' ORDER BY c DESC"),
+                   row(6), row(5), row(4), row(3), row(2));
+    }
+
+    /**
+     * Test for the bug from #4760 and #4759,
+     * migrated from cql_tests.py:TestCQL.reversed_compact_multikey_test()
+     */
+    @Test
+    public void testReversedCompactMultikey() throws Throwable
+    {
+        createTable("CREATE TABLE %s (key text, c1 int, c2 int, value text, PRIMARY KEY(key, c1, c2) ) WITH COMPACT STORAGE AND CLUSTERING ORDER BY(c1 DESC, c2 DESC)");
+
+        for (int i = 0; i < 3; i++)
+            for (int j = 0; j < 3; j++)
+                execute("INSERT INTO %s (key, c1, c2, value) VALUES ('foo', ?, ?, 'bar')", i, j);
+
+        // Equalities
+        assertRows(execute("SELECT c1, c2 FROM %s WHERE key='foo' AND c1 = 1"),
+                   row(1, 2), row(1, 1), row(1, 0));
+
+        assertRows(execute("SELECT c1, c2 FROM %s WHERE key='foo' AND c1 = 1 ORDER BY c1 ASC, c2 ASC"),
+                   row(1, 0), row(1, 1), row(1, 2));
+
+        assertRows(execute("SELECT c1, c2 FROM %s WHERE key='foo' AND c1 = 1 ORDER BY c1 DESC, c2 DESC"),
+                   row(1, 2), row(1, 1), row(1, 0));
+
+        // GT
+        assertRows(execute("SELECT c1, c2 FROM %s WHERE key='foo' AND c1 > 1"),
+                   row(2, 2), row(2, 1), row(2, 0));
+
+        assertRows(execute("SELECT c1, c2 FROM %s WHERE key='foo' AND c1 > 1 ORDER BY c1 ASC, c2 ASC"),
+                   row(2, 0), row(2, 1), row(2, 2));
+
+        assertRows(execute("SELECT c1, c2 FROM %s WHERE key='foo' AND c1 > 1 ORDER BY c1 DESC, c2 DESC"),
+                   row(2, 2), row(2, 1), row(2, 0));
+
+        assertRows(execute("SELECT c1, c2 FROM %s WHERE key='foo' AND c1 >= 1"),
+                   row(2, 2), row(2, 1), row(2, 0), row(1, 2), row(1, 1), row(1, 0));
+
+        assertRows(execute("SELECT c1, c2 FROM %s WHERE key='foo' AND c1 >= 1 ORDER BY c1 ASC, c2 ASC"),
+                   row(1, 0), row(1, 1), row(1, 2), row(2, 0), row(2, 1), row(2, 2));
+
+        assertRows(execute("SELECT c1, c2 FROM %s WHERE key='foo' AND c1 >= 1 ORDER BY c1 ASC"),
+                   row(1, 0), row(1, 1), row(1, 2), row(2, 0), row(2, 1), row(2, 2));
+
+        assertRows(execute("SELECT c1, c2 FROM %s WHERE key='foo' AND c1 >= 1 ORDER BY c1 DESC, c2 DESC"),
+                   row(2, 2), row(2, 1), row(2, 0), row(1, 2), row(1, 1), row(1, 0));
+
+        // LT
+        assertRows(execute("SELECT c1, c2 FROM %s WHERE key='foo' AND c1 < 1"),
+                   row(0, 2), row(0, 1), row(0, 0));
+
+        assertRows(execute("SELECT c1, c2 FROM %s WHERE key='foo' AND c1 < 1 ORDER BY c1 ASC, c2 ASC"),
+                   row(0, 0), row(0, 1), row(0, 2));
+
+        assertRows(execute("SELECT c1, c2 FROM %s WHERE key='foo' AND c1 < 1 ORDER BY c1 DESC, c2 DESC"),
+                   row(0, 2), row(0, 1), row(0, 0));
+
+        assertRows(execute("SELECT c1, c2 FROM %s WHERE key='foo' AND c1 <= 1"),
+                   row(1, 2), row(1, 1), row(1, 0), row(0, 2), row(0, 1), row(0, 0));
+
+        assertRows(execute("SELECT c1, c2 FROM %s WHERE key='foo' AND c1 <= 1 ORDER BY c1 ASC, c2 ASC"),
+                   row(0, 0), row(0, 1), row(0, 2), row(1, 0), row(1, 1), row(1, 2));
+
+        assertRows(execute("SELECT c1, c2 FROM %s WHERE key='foo' AND c1 <= 1 ORDER BY c1 ASC"),
+                   row(0, 0), row(0, 1), row(0, 2), row(1, 0), row(1, 1), row(1, 2));
+
+        assertRows(execute("SELECT c1, c2 FROM %s WHERE key='foo' AND c1 <= 1 ORDER BY c1 DESC, c2 DESC"),
+                   row(1, 2), row(1, 1), row(1, 0), row(0, 2), row(0, 1), row(0, 0));
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.bug_4882_test()
+     */
+    @Test
+    public void testDifferentOrdering() throws Throwable
+    {
+        createTable(" CREATE TABLE %s ( k int, c1 int, c2 int, v int, PRIMARY KEY (k, c1, c2) ) WITH CLUSTERING ORDER BY (c1 ASC, c2 DESC)");
+
+        execute("INSERT INTO %s (k, c1, c2, v) VALUES (0, 0, 0, 0)");
+        execute("INSERT INTO %s (k, c1, c2, v) VALUES (0, 1, 1, 1)");
+        execute("INSERT INTO %s (k, c1, c2, v) VALUES (0, 0, 2, 2)");
+        execute("INSERT INTO %s (k, c1, c2, v) VALUES (0, 1, 3, 3)");
+
+        assertRows(execute("select * from %s where k = 0 limit 1"),
+                   row(0, 0, 2, 2));
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.allow_filtering_test()
+     */
+    @Test
+    public void testAllowFiltering() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int, c int, v int, PRIMARY KEY (k, c))");
+
+        for (int i = 0; i < 3; i++)
+            for (int j = 0; j < 3; j++)
+                execute("INSERT INTO %s (k, c, v) VALUES (?, ?, ?)", i, j, j);
+
+        // Don't require filtering, always allowed
+        String[] queries = new String[]
+                           {
+                           "SELECT * FROM %s WHERE k = 1",
+                           "SELECT * FROM %s WHERE k = 1 AND c > 2",
+                           "SELECT * FROM %s WHERE k = 1 AND c = 2"
+                           };
+
+        for (String q : queries)
+        {
+            execute(q);
+            execute(q + " ALLOW FILTERING");
+        }
+
+        // Require filtering, allowed only with ALLOW FILTERING
+        queries = new String[]
+                  {
+                  "SELECT * FROM %s WHERE c = 2",
+                  "SELECT * FROM %s WHERE c > 2 AND c <= 4"
+                  };
+
+        for (String q : queries)
+        {
+            assertInvalid(q);
+            execute(q + " ALLOW FILTERING");
+        }
+
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, a int, b int,)");
+        createIndex("CREATE INDEX ON %s (a)");
+
+        for (int i = 0; i < 5; i++)
+            execute("INSERT INTO %s (k, a, b) VALUES (?, ?, ?)", i, i * 10, i * 100);
+
+        // Don't require filtering, always allowed
+        queries = new String[]
+                  {
+                  "SELECT * FROM %s WHERE k = 1",
+                  "SELECT * FROM %s WHERE a = 20"
+                  };
+
+        for (String q : queries)
+        {
+            execute(q);
+            execute(q + " ALLOW FILTERING");
+        }
+
+        // Require filtering, allowed only with ALLOW FILTERING
+        queries = new String[]
+                  {
+                  "SELECT * FROM %s WHERE a = 20 AND b = 200"
+                  };
+
+        for (String q : queries)
+        {
+            assertInvalid(q);
+            execute(q + " ALLOW FILTERING");
+        }
+    }
+
+    /**
+     * Test for bug from #5122,
+     * migrated from cql_tests.py:TestCQL.composite_partition_key_validation_test()
+     */
+    @Test
+    public void testSelectOnCompositeInvalid() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int, b text, c uuid, PRIMARY KEY ((a, b)))");
+
+        execute("INSERT INTO %s (a, b , c ) VALUES (1, 'aze', 4d481800-4c5f-11e1-82e0-3f484de45426)");
+        execute("INSERT INTO %s (a, b , c ) VALUES (1, 'ert', 693f5800-8acb-11e3-82e0-3f484de45426)");
+        execute("INSERT INTO %s (a, b , c ) VALUES (1, 'opl', d4815800-2d8d-11e0-82e0-3f484de45426)");
+
+        assertRowCount(execute("SELECT * FROM %s"), 3);
+        assertInvalid("SELECT * FROM %s WHERE a=1");
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.multi_in_compact_non_composite_test()
+     */
+    @Test
+    public void testMultiSelectsNonCompositeCompactStorage() throws Throwable
+    {
+        createTable("CREATE TABLE %s (key int, c int, v int, PRIMARY KEY (key, c)) WITH COMPACT STORAGE");
+
+        execute("INSERT INTO %s (key, c, v) VALUES (0, 0, 0)");
+        execute("INSERT INTO %s (key, c, v) VALUES (0, 1, 1)");
+        execute("INSERT INTO %s (key, c, v) VALUES (0, 2, 2)");
+
+        assertRows(execute("SELECT * FROM %s WHERE key=0 AND c IN (0, 2)"),
+                   row(0, 0, 0), row(0, 2, 2));
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.ticket_5230_test()
+     */
+    @Test
+    public void testMultipleClausesOnPrimaryKey() throws Throwable
+    {
+        createTable("CREATE TABLE %s (key text, c text, v text, PRIMARY KEY(key, c))");
+
+        execute("INSERT INTO %s (key, c, v) VALUES ('foo', '1', '1')");
+        execute("INSERT INTO %s(key, c, v) VALUES ('foo', '2', '2')");
+        execute("INSERT INTO %s(key, c, v) VALUES ('foo', '3', '3')");
+
+        assertRows(execute("SELECT c FROM %s WHERE key = 'foo' AND c IN ('1', '2')"),
+                   row("1"), row("2"));
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.bug_5404()
+     */
+    @Test
+    public void testSelectWithToken() throws Throwable
+    {
+        createTable("CREATE TABLE %s (key text PRIMARY KEY)");
+
+        // We just want to make sure this doesn 't NPE server side
+        assertInvalid("select * from %s where token(key) > token(int(3030343330393233)) limit 1");
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.clustering_order_and_functions_test()
+     */
+    @Test
+    public void testFunctionsWithClusteringDesc() throws Throwable
+    {
+        createTable("CREATE TABLE %s ( k int, t timeuuid, PRIMARY KEY (k, t) ) WITH CLUSTERING ORDER BY (t DESC)");
+
+        for (int i = 0; i < 5; i++)
+            execute("INSERT INTO %s (k, t) VALUES (?, now())", i);
+
+        execute("SELECT dateOf(t) FROM %s");
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.select_with_alias_test()
+     */
+    @Test
+    public void testSelectWithAlias() throws Throwable
+    {
+        createTable("CREATE TABLE %s (id int PRIMARY KEY, name text)");
+
+        for (int id = 0; id < 5; id++)
+            execute("INSERT INTO %s (id, name) VALUES (?, ?) USING TTL 10 AND TIMESTAMP 0", id, "name" + id);
+
+        // test aliasing count( *)
+        UntypedResultSet rs = execute("SELECT count(*) AS user_count FROM %s");
+        assertEquals("user_count", rs.metadata().get(0).name.toString());
+        assertEquals(5L, rs.one().getLong(rs.metadata().get(0).name.toString()));
+
+        // test aliasing regular value
+        rs = execute("SELECT name AS user_name FROM %s WHERE id = 0");
+        assertEquals("user_name", rs.metadata().get(0).name.toString());
+        assertEquals("name0", rs.one().getString(rs.metadata().get(0).name.toString()));
+
+        // test aliasing writetime
+        rs = execute("SELECT writeTime(name) AS name_writetime FROM %s WHERE id = 0");
+        assertEquals("name_writetime", rs.metadata().get(0).name.toString());
+        assertEquals(0, rs.one().getInt(rs.metadata().get(0).name.toString()));
+
+        // test aliasing ttl
+        rs = execute("SELECT ttl(name) AS name_ttl FROM %s WHERE id = 0");
+        assertEquals("name_ttl", rs.metadata().get(0).name.toString());
+        int ttl = rs.one().getInt(rs.metadata().get(0).name.toString());
+        assertTrue(ttl == 9 || ttl == 10);
+
+        // test aliasing a regular function
+        rs = execute("SELECT intAsBlob(id) AS id_blob FROM %s WHERE id = 0");
+        assertEquals("id_blob", rs.metadata().get(0).name.toString());
+        assertEquals(ByteBuffer.wrap(new byte[4]), rs.one().getBlob(rs.metadata().get(0).name.toString()));
+
+        // test that select throws a meaningful exception for aliases in where clause
+        assertInvalidMessage("Aliases aren't allowed in the where clause",
+                             "SELECT id AS user_id, name AS user_name FROM %s WHERE user_id = 0");
+
+        // test that select throws a meaningful exception for aliases in order by clause
+        assertInvalidMessage("Aliases are not allowed in order by clause",
+                             "SELECT id AS user_id, name AS user_name FROM %s WHERE id IN (0) ORDER BY user_name");
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.select_distinct_test()
+     */
+    @Test
+    public void testSelectDistinct() throws Throwable
+    {
+        // Test a regular(CQL3) table.
+        createTable("CREATE TABLE %s (pk0 int, pk1 int, ck0 int, val int, PRIMARY KEY((pk0, pk1), ck0))");
+
+        for (int i = 0; i < 3; i++)
+        {
+            execute("INSERT INTO %s (pk0, pk1, ck0, val) VALUES (?, ?, 0, 0)", i, i);
+            execute("INSERT INTO %s (pk0, pk1, ck0, val) VALUES (?, ?, 1, 1)", i, i);
+        }
+
+        assertRows(execute("SELECT DISTINCT pk0, pk1 FROM %s LIMIT 1"),
+                   row(0, 0));
+
+        assertRows(execute("SELECT DISTINCT pk0, pk1 FROM %s LIMIT 3"),
+                   row(0, 0),
+                   row(2, 2),
+                   row(1, 1));
+
+        // Test selection validation.
+        assertInvalidMessage("queries must request all the partition key columns", "SELECT DISTINCT pk0 FROM %s");
+        assertInvalidMessage("queries must only request partition key columns", "SELECT DISTINCT pk0, pk1, ck0 FROM %s");
+
+        //Test a 'compact storage' table.
+        createTable("CREATE TABLE %s (pk0 int, pk1 int, val int, PRIMARY KEY((pk0, pk1))) WITH COMPACT STORAGE");
+
+        for (int i = 0; i < 3; i++)
+            execute("INSERT INTO %s (pk0, pk1, val) VALUES (?, ?, ?)", i, i, i);
+
+        assertRows(execute("SELECT DISTINCT pk0, pk1 FROM %s LIMIT 1"),
+                   row(0, 0));
+
+        assertRows(execute("SELECT DISTINCT pk0, pk1 FROM %s LIMIT 3"),
+                   row(0, 0),
+                   row(2, 2),
+                   row(1, 1));
+
+        // Test a 'wide row' thrift table.
+        createTable("CREATE TABLE %s (pk int, name text, val int, PRIMARY KEY(pk, name)) WITH COMPACT STORAGE");
+
+        for (int i = 0; i < 3; i++)
+        {
+            execute("INSERT INTO %s (pk, name, val) VALUES (?, 'name0', 0)", i);
+            execute("INSERT INTO %s (pk, name, val) VALUES (?, 'name1', 1)", i);
+        }
+
+        assertRows(execute("SELECT DISTINCT pk FROM %s LIMIT 1"),
+                   row(1));
+
+        assertRows(execute("SELECT DISTINCT pk FROM %s LIMIT 3"),
+                   row(1),
+                   row(0),
+                   row(2));
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.select_distinct_with_deletions_test()
+     */
+    @Test
+    public void testSelectDistinctWithDeletions() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, c int, v int)");
+
+        for (int i = 0; i < 10; i++)
+            execute("INSERT INTO %s (k, c, v) VALUES (?, ?, ?)", i, i, i);
+
+        Object[][] rows = getRows(execute("SELECT DISTINCT k FROM %s"));
+        Assert.assertEquals(10, rows.length);
+        Object key_to_delete = rows[3][0];
+
+        execute("DELETE FROM %s WHERE k=?", key_to_delete);
+
+        rows = getRows(execute("SELECT DISTINCT k FROM %s"));
+        Assert.assertEquals(9, rows.length);
+
+        rows = getRows(execute("SELECT DISTINCT k FROM %s LIMIT 5"));
+        Assert.assertEquals(5, rows.length);
+
+        rows = getRows(execute("SELECT DISTINCT k FROM %s"));
+        Assert.assertEquals(9, rows.length);
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.bug_6327_test()
+     */
+    @Test
+    public void testSelectInClauseAtOne() throws Throwable
+    {
+        createTable("CREATE TABLE %s ( k int, v int, PRIMARY KEY (k, v))");
+
+        execute("INSERT INTO %s (k, v) VALUES (0, 0)");
+
+        flush();
+
+        assertRows(execute("SELECT v FROM %s WHERE k=0 AND v IN (1, 0)"),
+                   row(0));
+    }
+
+    /**
+     * Test for the #6579 'select count' paging bug,
+     * migrated from cql_tests.py:TestCQL.select_count_paging_test()
+     */
+    @Test
+    public void testSelectCountPaging() throws Throwable
+    {
+        createTable("create table %s (field1 text, field2 timeuuid, field3 boolean, primary key(field1, field2))");
+        createIndex("create index test_index on %s (field3)");
+
+        execute("insert into %s (field1, field2, field3) values ('hola', now(), false)");
+        execute("insert into %s (field1, field2, field3) values ('hola', now(), false)");
+
+        assertRows(execute("select count(*) from %s where field3 = false limit 1"),
+                   row(1L));
+    }
+
+    /**
+     * Test for #7105 bug,
+     * migrated from cql_tests.py:TestCQL.clustering_order_in_test()
+     */
+    @Test
+    public void testClusteringOrder() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int, b int, c int, PRIMARY KEY ((a, b), c) ) with clustering order by (c desc)");
+
+        execute("INSERT INTO %s (a, b, c) VALUES (1, 2, 3)");
+        execute("INSERT INTO %s (a, b, c) VALUES (4, 5, 6)");
+
+        assertRows(execute("SELECT * FROM %s WHERE a=1 AND b=2 AND c IN (3)"),
+                   row(1, 2, 3));
+        assertRows(execute("SELECT * FROM %s WHERE a=1 AND b=2 AND c IN (3, 4)"),
+                   row(1, 2, 3));
+    }
+
+    /**
+     * Test for #7105 bug,
+     * SELECT with IN on final column of composite and compound primary key fails
+     * migrated from cql_tests.py:TestCQL.bug7105_test()
+     */
+    @Test
+    public void testSelectInFinalColumn() 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 (1, 2, 3, 3)");
+        execute("INSERT INTO %s (a, b, c, d) VALUES (1, 4, 6, 5)");
+
+        assertRows(execute("SELECT * FROM %s WHERE a=1 AND b=2 ORDER BY b DESC"),
+                   row(1, 2, 3, 3));
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f797bfa4/test/unit/org/apache/cassandra/cql3/validation/operations/UpdateTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/UpdateTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/UpdateTest.java
new file mode 100644
index 0000000..19855d8
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/UpdateTest.java
@@ -0,0 +1,59 @@
+/*
+ * 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.validation.operations;
+
+import org.junit.Test;
+
+import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+public class UpdateTest extends CQLTester
+{
+    /**
+     * Test altering the type of a column, including the one in the primary key (#4041)
+     * migrated from cql_tests.py:TestCQL.update_type_test()
+     */
+    @Test
+    public void testUpdateColumnType() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k text, c text, s set <text>, v text, PRIMARY KEY(k, c))");
+
+        // using utf8 character so that we can see the transition to BytesType
+        execute("INSERT INTO %s (k, c, v, s) VALUES ('ɸ', 'ɸ', 'ɸ', {'ɸ'})");
+
+        assertRows(execute("SELECT * FROM %s"),
+                   row("ɸ", "ɸ", set("ɸ"), "ɸ"));
+
+        execute("ALTER TABLE %s ALTER v TYPE blob");
+        assertRows(execute("SELECT * FROM %s"),
+                   row("ɸ", "ɸ", set("ɸ"), ByteBufferUtil.bytes("ɸ")));
+
+        execute("ALTER TABLE %s ALTER k TYPE blob");
+        assertRows(execute("SELECT * FROM %s"),
+                   row(ByteBufferUtil.bytes("ɸ"), "ɸ", set("ɸ"), ByteBufferUtil.bytes("ɸ")));
+
+        execute("ALTER TABLE %s ALTER c TYPE blob");
+        assertRows(execute("SELECT * FROM %s"),
+                   row(ByteBufferUtil.bytes("ɸ"), ByteBufferUtil.bytes("ɸ"), set("ɸ"), ByteBufferUtil.bytes("ɸ")));
+
+        execute("ALTER TABLE %s ALTER s TYPE set<blob>");
+        assertRows(execute("SELECT * FROM %s"),
+                   row(ByteBufferUtil.bytes("ɸ"), ByteBufferUtil.bytes("ɸ"), set(ByteBufferUtil.bytes("ɸ")), ByteBufferUtil.bytes("ɸ")));
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f797bfa4/test/unit/org/apache/cassandra/cql3/validation/operations/UseTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/UseTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/UseTest.java
new file mode 100644
index 0000000..e1498b6
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/UseTest.java
@@ -0,0 +1,31 @@
+/*
+ * 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.validation.operations;
+
+import org.junit.Test;
+
+import org.apache.cassandra.cql3.CQLTester;
+
+public class UseTest extends CQLTester
+{
+    @Test
+    public void testUseStatementWithBindVariable() throws Throwable
+    {
+        assertInvalidSyntaxMessage("Bind variables cannot be used for keyspace names", "USE ?");
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f797bfa4/tools/stress/src/org/apache/cassandra/stress/generate/values/TimeUUIDs.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/generate/values/TimeUUIDs.java b/tools/stress/src/org/apache/cassandra/stress/generate/values/TimeUUIDs.java
index 7bfabf5..537d54e 100644
--- a/tools/stress/src/org/apache/cassandra/stress/generate/values/TimeUUIDs.java
+++ b/tools/stress/src/org/apache/cassandra/stress/generate/values/TimeUUIDs.java
@@ -46,6 +46,6 @@ public class TimeUUIDs extends Generator<UUID>
     @Override
     public UUID generate()
     {
-        return UUIDGen.getTimeUUID(dateGen.generate().getTime(), clockSeqAndNode);
+        return UUIDGen.getTimeUUID(dateGen.generate().getTime(), 0L, clockSeqAndNode);
     }
 }


[30/32] cassandra git commit: 2.2 commit for CASSANDRA-9160

Posted by jm...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/01115f72/test/unit/org/apache/cassandra/cql3/AggregationTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/AggregationTest.java b/test/unit/org/apache/cassandra/cql3/AggregationTest.java
deleted file mode 100644
index 38c4759..0000000
--- a/test/unit/org/apache/cassandra/cql3/AggregationTest.java
+++ /dev/null
@@ -1,1479 +0,0 @@
-/*
- * 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;
-
-import java.math.BigDecimal;
-import java.text.SimpleDateFormat;
-import java.util.Calendar;
-import java.util.Date;
-import java.util.TimeZone;
-
-import org.apache.commons.lang3.time.DateUtils;
-import org.junit.Assert;
-import org.junit.Test;
-
-import org.apache.cassandra.cql3.functions.Functions;
-import org.apache.cassandra.cql3.functions.UDAggregate;
-import org.apache.cassandra.exceptions.FunctionExecutionException;
-import org.apache.cassandra.exceptions.InvalidRequestException;
-import org.apache.cassandra.serializers.Int32Serializer;
-import org.apache.cassandra.service.ClientState;
-import org.apache.cassandra.transport.Event;
-import org.apache.cassandra.transport.messages.ResultMessage;
-
-public class AggregationTest extends CQLTester
-{
-    @Test
-    public void testFunctions() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a int, b int, c double, d decimal, primary key (a, b))");
-
-        // Test with empty table
-        assertColumnNames(execute("SELECT COUNT(*) FROM %s"), "count");
-        assertRows(execute("SELECT COUNT(*) FROM %s"), row(0L));
-        assertColumnNames(execute("SELECT max(b), min(b), sum(b), avg(b) , max(c), sum(c), avg(c), sum(d), avg(d) FROM %s"),
-                          "system.max(b)", "system.min(b)", "system.sum(b)", "system.avg(b)", "system.max(c)", "system.sum(c)", "system.avg(c)", "system.sum(d)", "system.avg(d)");
-        assertRows(execute("SELECT max(b), min(b), sum(b), avg(b) , max(c), sum(c), avg(c), sum(d), avg(d) FROM %s"),
-                   row(null, null, 0, 0, null, 0.0, 0.0, new BigDecimal("0"), new BigDecimal("0")));
-
-        execute("INSERT INTO %s (a, b, c, d) VALUES (1, 1, 11.5, 11.5)");
-        execute("INSERT INTO %s (a, b, c, d) VALUES (1, 2, 9.5, 1.5)");
-        execute("INSERT INTO %s (a, b, c, d) VALUES (1, 3, 9.0, 2.0)");
-
-        assertRows(execute("SELECT max(b), min(b), sum(b), avg(b) , max(c), sum(c), avg(c), sum(d), avg(d) FROM %s"),
-                   row(3, 1, 6, 2, 11.5, 30.0, 10.0, new BigDecimal("15.0"), new BigDecimal("5.0")));
-
-        execute("INSERT INTO %s (a, b, d) VALUES (1, 5, 1.0)");
-        assertRows(execute("SELECT COUNT(*) FROM %s"), row(4L));
-        assertRows(execute("SELECT COUNT(1) FROM %s"), row(4L));
-        assertRows(execute("SELECT COUNT(b), count(c) FROM %s"), row(4L, 3L));
-    }
-
-    @Test
-    public void testFunctionsWithCompactStorage() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a int , b int, c double, primary key(a, b) ) WITH COMPACT STORAGE");
-
-        execute("INSERT INTO %s (a, b, c) VALUES (1, 1, 11.5)");
-        execute("INSERT INTO %s (a, b, c) VALUES (1, 2, 9.5)");
-        execute("INSERT INTO %s (a, b, c) VALUES (1, 3, 9.0)");
-
-        assertRows(execute("SELECT max(b), min(b), sum(b), avg(b) , max(c), sum(c), avg(c) FROM %s"),
-                   row(3, 1, 6, 2, 11.5, 30.0, 10.0));
-
-        assertRows(execute("SELECT COUNT(*) FROM %s"), row(3L));
-        assertRows(execute("SELECT COUNT(1) FROM %s"), row(3L));
-        assertRows(execute("SELECT COUNT(*) FROM %s WHERE a = 1 AND b > 1"), row(2L));
-        assertRows(execute("SELECT COUNT(1) FROM %s WHERE a = 1 AND b > 1"), row(2L));
-        assertRows(execute("SELECT max(b), min(b), sum(b), avg(b) , max(c), sum(c), avg(c) FROM %s WHERE a = 1 AND b > 1"),
-                   row(3, 2, 5, 2, 9.5, 18.5, 9.25));
-    }
-
-    @Test
-    public void testInvalidCalls() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a int, b int, c int, primary key (a, b))");
-        execute("INSERT INTO %s (a, b, c) VALUES (1, 1, 10)");
-        execute("INSERT INTO %s (a, b, c) VALUES (1, 2, 9)");
-        execute("INSERT INTO %s (a, b, c) VALUES (1, 3, 8)");
-
-        assertInvalidSyntax("SELECT max(b), max(c) FROM %s WHERE max(a) = 1");
-        assertInvalidMessage("only aggregates or no aggregate", "SELECT max(b), c FROM %s");
-        assertInvalidMessage("only aggregates or no aggregate", "SELECT b, max(c) FROM %s");
-        assertInvalidMessage("aggregate functions cannot be used as arguments of aggregate functions", "SELECT max(sum(c)) FROM %s");
-        assertInvalidSyntax("SELECT COUNT(2) FROM %s");
-    }
-
-    @Test
-    public void testNestedFunctions() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a int primary key, b timeuuid, c double, d double)");
-
-        String copySign = createFunction(KEYSPACE,
-                                         "double, double",
-                                         "CREATE OR REPLACE FUNCTION %s(magnitude double, sign double) " +
-                                         "RETURNS NULL ON NULL INPUT " +
-                                         "RETURNS double " +
-                                         "LANGUAGE JAVA " +
-                                         "AS 'return Double.valueOf(Math.copySign(magnitude, sign));';");
-
-        assertColumnNames(execute("SELECT max(a), max(toUnixTimestamp(b)) FROM %s"), "system.max(a)", "system.max(system.tounixtimestamp(b))");
-        assertRows(execute("SELECT max(a), max(toUnixTimestamp(b)) FROM %s"), row(null, null));
-        assertColumnNames(execute("SELECT max(a), toUnixTimestamp(max(b)) FROM %s"), "system.max(a)", "system.tounixtimestamp(system.max(b))");
-        assertRows(execute("SELECT max(a), toUnixTimestamp(max(b)) FROM %s"), row(null, null));
-
-        assertColumnNames(execute("SELECT max(" + copySign + "(c, d)) FROM %s"), "system.max(" + copySign + "(c, d))");
-        assertRows(execute("SELECT max(" + copySign + "(c, d)) FROM %s"), row((Object) null));
-
-        execute("INSERT INTO %s (a, b, c, d) VALUES (1, maxTimeuuid('2011-02-03 04:05:00+0000'), -1.2, 2.1)");
-        execute("INSERT INTO %s (a, b, c, d) VALUES (2, maxTimeuuid('2011-02-03 04:06:00+0000'), 1.3, -3.4)");
-        execute("INSERT INTO %s (a, b, c, d) VALUES (3, maxTimeuuid('2011-02-03 04:10:00+0000'), 1.4, 1.2)");
-
-        SimpleDateFormat format = new SimpleDateFormat("yyyy-MM-dd hh:mm:ss");
-        format.setTimeZone(TimeZone.getTimeZone("GMT"));
-        Date date = format.parse("2011-02-03 04:10:00");
-        date = DateUtils.truncate(date, Calendar.MILLISECOND);
-
-        assertRows(execute("SELECT max(a), max(toUnixTimestamp(b)) FROM %s"), row(3, date.getTime()));
-        assertRows(execute("SELECT max(a), toUnixTimestamp(max(b)) FROM %s"), row(3, date.getTime()));
-
-        assertRows(execute("SELECT " + copySign + "(max(c), min(c)) FROM %s"), row(-1.4));
-        assertRows(execute("SELECT " + copySign + "(c, d) FROM %s"), row(1.2), row(-1.3), row(1.4));
-        assertRows(execute("SELECT max(" + copySign + "(c, d)) FROM %s"), row(1.4));
-        assertInvalidMessage("must be either all aggregates or no aggregates", "SELECT " + copySign + "(c, max(c)) FROM %s");
-        assertInvalidMessage("must be either all aggregates or no aggregates", "SELECT " + copySign + "(max(c), c) FROM %s");
-    }
-
-    @Test
-    public void testSchemaChange() throws Throwable
-    {
-        String f = createFunction(KEYSPACE,
-                                  "double, double",
-                                  "CREATE OR REPLACE FUNCTION %s(state double, val double) " +
-                                  "RETURNS NULL ON NULL INPUT " +
-                                  "RETURNS double " +
-                                  "LANGUAGE javascript " +
-                                  "AS '\"string\";';");
-
-        createFunctionOverload(f,
-                               "double, double",
-                               "CREATE OR REPLACE FUNCTION %s(state int, val int) " +
-                               "RETURNS NULL ON NULL INPUT " +
-                               "RETURNS int " +
-                               "LANGUAGE javascript " +
-                               "AS '\"string\";';");
-
-        String a = createAggregate(KEYSPACE,
-                                   "double",
-                                   "CREATE OR REPLACE AGGREGATE %s(double) " +
-                                   "SFUNC " + shortFunctionName(f) + " " +
-                                   "STYPE double " +
-                                   "INITCOND 0");
-
-        assertLastSchemaChange(Event.SchemaChange.Change.CREATED, Event.SchemaChange.Target.AGGREGATE,
-                               KEYSPACE, parseFunctionName(a).name,
-                               "double");
-
-        schemaChange("CREATE OR REPLACE AGGREGATE " + a + "(double) " +
-                     "SFUNC " + shortFunctionName(f) + " " +
-                     "STYPE double " +
-                     "INITCOND 0");
-
-        assertLastSchemaChange(Event.SchemaChange.Change.UPDATED, Event.SchemaChange.Target.AGGREGATE,
-                               KEYSPACE, parseFunctionName(a).name,
-                               "double");
-
-        createAggregateOverload(a,
-                                "int",
-                                "CREATE OR REPLACE AGGREGATE %s(int) " +
-                                "SFUNC " + shortFunctionName(f) + " " +
-                                "STYPE int " +
-                                "INITCOND 0");
-
-        assertLastSchemaChange(Event.SchemaChange.Change.CREATED, Event.SchemaChange.Target.AGGREGATE,
-                               KEYSPACE, parseFunctionName(a).name,
-                               "int");
-
-        schemaChange("DROP AGGREGATE " + a + "(double)");
-
-        assertLastSchemaChange(Event.SchemaChange.Change.DROPPED, Event.SchemaChange.Target.AGGREGATE,
-                               KEYSPACE, parseFunctionName(a).name,
-                               "double");
-    }
-
-    @Test
-    public void testDropStatements() throws Throwable
-    {
-        String f = createFunction(KEYSPACE,
-                                  "double, double",
-                                  "CREATE OR REPLACE FUNCTION %s(state double, val double) " +
-                                  "RETURNS NULL ON NULL INPUT " +
-                                  "RETURNS double " +
-                                  "LANGUAGE javascript " +
-                                  "AS '\"string\";';");
-
-        createFunctionOverload(f,
-                               "double, double",
-                               "CREATE OR REPLACE FUNCTION %s(state int, val int) " +
-                               "RETURNS NULL ON NULL INPUT " +
-                               "RETURNS int " +
-                               "LANGUAGE javascript " +
-                               "AS '\"string\";';");
-
-        // DROP AGGREGATE must not succeed against a scalar
-        assertInvalidMessage("matches multiple function definitions", "DROP AGGREGATE " + f);
-        assertInvalidMessage("non existing", "DROP AGGREGATE " + f + "(double, double)");
-
-        String a = createAggregate(KEYSPACE,
-                                   "double",
-                                   "CREATE OR REPLACE AGGREGATE %s(double) " +
-                                   "SFUNC " + shortFunctionName(f) + " " +
-                                   "STYPE double " +
-                                   "INITCOND 0");
-        createAggregateOverload(a,
-                                "int",
-                                "CREATE OR REPLACE AGGREGATE %s(int) " +
-                                "SFUNC " + shortFunctionName(f) + " " +
-                                "STYPE int " +
-                                "INITCOND 0");
-
-        // DROP FUNCTION must not succeed against an aggregate
-        assertInvalidMessage("matches multiple function definitions", "DROP FUNCTION " + a);
-        assertInvalidMessage("non existing function", "DROP FUNCTION " + a + "(double)");
-
-        // ambigious
-        assertInvalidMessage("matches multiple function definitions", "DROP AGGREGATE " + a);
-        assertInvalidMessage("matches multiple function definitions", "DROP AGGREGATE IF EXISTS " + a);
-
-        execute("DROP AGGREGATE IF EXISTS " + KEYSPACE + ".non_existing");
-        execute("DROP AGGREGATE IF EXISTS " + a + "(int, text)");
-
-        execute("DROP AGGREGATE " + a + "(double)");
-
-        execute("DROP AGGREGATE IF EXISTS " + a + "(double)");
-    }
-
-    @Test
-    public void testDropReferenced() throws Throwable
-    {
-        String f = createFunction(KEYSPACE,
-                                  "double, double",
-                                  "CREATE OR REPLACE FUNCTION %s(state double, val double) " +
-                                  "RETURNS NULL ON NULL INPUT " +
-                                  "RETURNS double " +
-                                  "LANGUAGE javascript " +
-                                  "AS '\"string\";';");
-
-        String a = createAggregate(KEYSPACE,
-                                   "double",
-                                   "CREATE OR REPLACE AGGREGATE %s(double) " +
-                                   "SFUNC " + shortFunctionName(f) + " " +
-                                   "STYPE double " +
-                                   "INITCOND 0");
-
-        // DROP FUNCTION must not succeed because the function is still referenced by the aggregate
-        assertInvalidMessage("still referenced by", "DROP FUNCTION " + f);
-
-        execute("DROP AGGREGATE " + a + "(double)");
-    }
-
-    @Test
-    public void testJavaAggregateNoInit() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a int primary key, b int)");
-        execute("INSERT INTO %s (a, b) VALUES (1, 1)");
-        execute("INSERT INTO %s (a, b) VALUES (2, 2)");
-        execute("INSERT INTO %s (a, b) VALUES (3, 3)");
-
-        String fState = createFunction(KEYSPACE,
-                                       "int, int",
-                                       "CREATE FUNCTION %s(a int, b int) " +
-                                       "CALLED ON NULL INPUT " +
-                                       "RETURNS int " +
-                                       "LANGUAGE java " +
-                                       "AS 'return Integer.valueOf((a!=null?a.intValue():0) + b.intValue());'");
-
-        String fFinal = createFunction(KEYSPACE,
-                                       "int",
-                                       "CREATE FUNCTION %s(a int) " +
-                                       "CALLED ON NULL INPUT " +
-                                       "RETURNS text " +
-                                       "LANGUAGE java " +
-                                       "AS 'return a.toString();'");
-
-        String a = createAggregate(KEYSPACE,
-                                   "int",
-                                   "CREATE AGGREGATE %s(int) " +
-                                   "SFUNC " + shortFunctionName(fState) + " " +
-                                   "STYPE int " +
-                                   "FINALFUNC " + shortFunctionName(fFinal));
-
-        // 1 + 2 + 3 = 6
-        assertRows(execute("SELECT " + a + "(b) FROM %s"), row("6"));
-
-        execute("DROP AGGREGATE " + a + "(int)");
-
-        assertInvalidMessage("Unknown function", "SELECT " + a + "(b) FROM %s");
-    }
-
-    @Test
-    public void testJavaAggregateNullInitcond() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a int primary key, b int)");
-        execute("INSERT INTO %s (a, b) VALUES (1, 1)");
-        execute("INSERT INTO %s (a, b) VALUES (2, 2)");
-        execute("INSERT INTO %s (a, b) VALUES (3, 3)");
-
-        String fState = createFunction(KEYSPACE,
-                                       "int, int",
-                                       "CREATE FUNCTION %s(a int, b int) " +
-                                       "CALLED ON NULL INPUT " +
-                                       "RETURNS int " +
-                                       "LANGUAGE java " +
-                                       "AS 'return Integer.valueOf((a!=null?a.intValue():0) + b.intValue());'");
-
-        String fFinal = createFunction(KEYSPACE,
-                                       "int",
-                                       "CREATE FUNCTION %s(a int) " +
-                                       "CALLED ON NULL INPUT " +
-                                       "RETURNS text " +
-                                       "LANGUAGE java " +
-                                       "AS 'return a.toString();'");
-
-        String a = createAggregate(KEYSPACE,
-                                   "int",
-                                   "CREATE AGGREGATE %s(int) " +
-                                   "SFUNC " + shortFunctionName(fState) + " " +
-                                   "STYPE int " +
-                                   "FINALFUNC " + shortFunctionName(fFinal) + " " +
-                                   "INITCOND null");
-
-        // 1 + 2 + 3 = 6
-        assertRows(execute("SELECT " + a + "(b) FROM %s"), row("6"));
-
-        execute("DROP AGGREGATE " + a + "(int)");
-
-        assertInvalidMessage("Unknown function", "SELECT " + a + "(b) FROM %s");
-    }
-
-    @Test
-    public void testJavaAggregateInvalidInitcond() throws Throwable
-    {
-        String fState = createFunction(KEYSPACE,
-                                       "int, int",
-                                       "CREATE FUNCTION %s(a int, b int) " +
-                                       "CALLED ON NULL INPUT " +
-                                       "RETURNS int " +
-                                       "LANGUAGE java " +
-                                       "AS 'return Integer.valueOf((a!=null?a.intValue():0) + b.intValue());'");
-
-        String fFinal = createFunction(KEYSPACE,
-                                       "int",
-                                       "CREATE FUNCTION %s(a int) " +
-                                       "CALLED ON NULL INPUT " +
-                                       "RETURNS text " +
-                                       "LANGUAGE java " +
-                                       "AS 'return a.toString();'");
-
-        assertInvalidMessage("Invalid STRING constant (foobar)",
-                             "CREATE AGGREGATE " + KEYSPACE + ".aggrInvalid(int)" +
-                             "SFUNC " + shortFunctionName(fState) + " " +
-                             "STYPE int " +
-                             "FINALFUNC " + shortFunctionName(fFinal) + " " +
-                             "INITCOND 'foobar'");
-    }
-
-    @Test
-    public void testJavaAggregateIncompatibleTypes() throws Throwable
-    {
-        String fState = createFunction(KEYSPACE,
-                                       "int, int",
-                                       "CREATE FUNCTION %s(a int, b int) " +
-                                       "CALLED ON NULL INPUT " +
-                                       "RETURNS int " +
-                                       "LANGUAGE java " +
-                                       "AS 'return Integer.valueOf((a!=null?a.intValue():0) + b.intValue());'");
-
-        String fFinal = createFunction(KEYSPACE,
-                                       "int",
-                                       "CREATE FUNCTION %s(a int) " +
-                                       "CALLED ON NULL INPUT " +
-                                       "RETURNS text " +
-                                       "LANGUAGE java " +
-                                       "AS 'return a.toString();'");
-
-        String fState2 = createFunction(KEYSPACE,
-                                        "int, int",
-                                        "CREATE FUNCTION %s(a double, b double) " +
-                                        "CALLED ON NULL INPUT " +
-                                        "RETURNS double " +
-                                        "LANGUAGE java " +
-                                        "AS 'return Double.valueOf((a!=null?a.doubleValue():0d) + b.doubleValue());'");
-
-        String fFinal2 = createFunction(KEYSPACE,
-                                        "int",
-                                        "CREATE FUNCTION %s(a double) " +
-                                        "CALLED ON NULL INPUT " +
-                                        "RETURNS text " +
-                                        "LANGUAGE java " +
-                                        "AS 'return a.toString();'");
-
-        assertInvalidMessage("does not exist or is not a scalar function",
-                             "CREATE AGGREGATE " + KEYSPACE + ".aggrInvalid(double)" +
-                             "SFUNC " + shortFunctionName(fState) + " " +
-                             "STYPE double " +
-                             "FINALFUNC " + shortFunctionName(fFinal));
-        assertInvalidMessage("does not exist or is not a scalar function",
-                             "CREATE AGGREGATE " + KEYSPACE + ".aggrInvalid(int)" +
-                             "SFUNC " + shortFunctionName(fState) + " " +
-                             "STYPE double " +
-                             "FINALFUNC " + shortFunctionName(fFinal));
-        assertInvalidMessage("does not exist or is not a scalar function",
-                             "CREATE AGGREGATE " + KEYSPACE + ".aggrInvalid(double)" +
-                             "SFUNC " + shortFunctionName(fState) + " " +
-                             "STYPE int " +
-                             "FINALFUNC " + shortFunctionName(fFinal));
-        assertInvalidMessage("does not exist or is not a scalar function",
-                             "CREATE AGGREGATE " + KEYSPACE + ".aggrInvalid(double)" +
-                             "SFUNC " + shortFunctionName(fState) + " " +
-                             "STYPE int");
-        assertInvalidMessage("does not exist or is not a scalar function",
-                             "CREATE AGGREGATE " + KEYSPACE + ".aggrInvalid(int)" +
-                             "SFUNC " + shortFunctionName(fState) + " " +
-                             "STYPE double");
-
-        assertInvalidMessage("does not exist or is not a scalar function",
-                             "CREATE AGGREGATE " + KEYSPACE + ".aggrInvalid(double)" +
-                             "SFUNC " + shortFunctionName(fState2) + " " +
-                             "STYPE double " +
-                             "FINALFUNC " + shortFunctionName(fFinal));
-
-        assertInvalidMessage("does not exist or is not a scalar function",
-                             "CREATE AGGREGATE " + KEYSPACE + ".aggrInvalid(double)" +
-                             "SFUNC " + shortFunctionName(fState) + " " +
-                             "STYPE double " +
-                             "FINALFUNC " + shortFunctionName(fFinal2));
-    }
-
-    @Test
-    public void testJavaAggregateNonExistingFuncs() throws Throwable
-    {
-        String fState = createFunction(KEYSPACE,
-                                       "int, int",
-                                       "CREATE FUNCTION %s(a int, b int) " +
-                                       "CALLED ON NULL INPUT " +
-                                       "RETURNS int " +
-                                       "LANGUAGE java " +
-                                       "AS 'return Integer.valueOf((a!=null?a.intValue():0) + b.intValue());'");
-
-        String fFinal = createFunction(KEYSPACE,
-                                       "int",
-                                       "CREATE FUNCTION %s(a int) " +
-                                       "CALLED ON NULL INPUT " +
-                                       "RETURNS text " +
-                                       "LANGUAGE java " +
-                                       "AS 'return a.toString();'");
-
-        assertInvalidMessage("does not exist or is not a scalar function",
-                             "CREATE AGGREGATE " + KEYSPACE + ".aggrInvalid(int)" +
-                             "SFUNC " + shortFunctionName(fState) + "_not_there " +
-                             "STYPE int " +
-                             "FINALFUNC " + shortFunctionName(fFinal));
-
-        assertInvalidMessage("does not exist or is not a scalar function",
-                             "CREATE AGGREGATE " + KEYSPACE + ".aggrInvalid(int)" +
-                             "SFUNC " + shortFunctionName(fState) + " " +
-                             "STYPE int " +
-                             "FINALFUNC " + shortFunctionName(fFinal) + "_not_there");
-
-        execute("CREATE AGGREGATE " + KEYSPACE + ".aggrInvalid(int)" +
-                "SFUNC " + shortFunctionName(fState) + " " +
-                "STYPE int " +
-                "FINALFUNC " + shortFunctionName(fFinal));
-        execute("DROP AGGREGATE " + KEYSPACE + ".aggrInvalid(int)");
-    }
-
-    @Test
-    public void testJavaAggregateFailingFuncs() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a int primary key, b int)");
-        execute("INSERT INTO %s (a, b) VALUES (1, 1)");
-        execute("INSERT INTO %s (a, b) VALUES (2, 2)");
-        execute("INSERT INTO %s (a, b) VALUES (3, 3)");
-
-        String fState = createFunction(KEYSPACE,
-                                       "int, int",
-                                       "CREATE FUNCTION %s(a int, b int) " +
-                                       "CALLED ON NULL INPUT " +
-                                       "RETURNS int " +
-                                       "LANGUAGE java " +
-                                       "AS 'throw new RuntimeException(\"thrown to unit test - not a bug\");'");
-
-        String fStateOK = createFunction(KEYSPACE,
-                                       "int, int",
-                                       "CREATE FUNCTION %s(a int, b int) " +
-                                       "CALLED ON NULL INPUT " +
-                                       "RETURNS int " +
-                                       "LANGUAGE java " +
-                                       "AS 'return Integer.valueOf(42);'");
-
-        String fFinal = createFunction(KEYSPACE,
-                                       "int",
-                                       "CREATE FUNCTION %s(a int) " +
-                                       "CALLED ON NULL INPUT " +
-                                       "RETURNS text " +
-                                       "LANGUAGE java " +
-                                       "AS 'throw new RuntimeException(\"thrown to unit test - not a bug\");'");
-
-        String fFinalOK = createFunction(KEYSPACE,
-                                       "int",
-                                       "CREATE FUNCTION %s(a int) " +
-                                       "CALLED ON NULL INPUT " +
-                                       "RETURNS text " +
-                                       "LANGUAGE java " +
-                                       "AS 'return \"foobar\";'");
-
-        String a0 = createAggregate(KEYSPACE,
-                                    "int",
-                                    "CREATE AGGREGATE %s(int) " +
-                                    "SFUNC " + shortFunctionName(fState) + " " +
-                                    "STYPE int " +
-                                    "FINALFUNC " + shortFunctionName(fFinal) + " " +
-                                    "INITCOND null");
-        String a1 = createAggregate(KEYSPACE,
-                                    "int",
-                                    "CREATE AGGREGATE %s(int) " +
-                                    "SFUNC " + shortFunctionName(fStateOK) + " " +
-                                    "STYPE int " +
-                                    "FINALFUNC " + shortFunctionName(fFinal) + " " +
-                                    "INITCOND null");
-        String a2 = createAggregate(KEYSPACE,
-                                    "int",
-                                    "CREATE AGGREGATE %s(int) " +
-                                    "SFUNC " + shortFunctionName(fStateOK) + " " +
-                                    "STYPE int " +
-                                    "FINALFUNC " + shortFunctionName(fFinalOK) + " " +
-                                    "INITCOND null");
-
-        assertInvalidThrowMessage("java.lang.RuntimeException", FunctionExecutionException.class, "SELECT " + a0 + "(b) FROM %s");
-        assertInvalidThrowMessage("java.lang.RuntimeException", FunctionExecutionException.class, "SELECT " + a1 + "(b) FROM %s");
-        assertRows(execute("SELECT " + a2 + "(b) FROM %s"), row("foobar"));
-    }
-
-    @Test
-    public void testJavaAggregateWithoutStateOrFinal() throws Throwable
-    {
-        assertInvalidMessage("does not exist or is not a scalar function",
-                             "CREATE AGGREGATE " + KEYSPACE + ".jSumFooNE1(int) " +
-                             "SFUNC jSumFooNEstate " +
-                             "STYPE int");
-
-        String f = createFunction(KEYSPACE,
-                                  "int, int",
-                                  "CREATE FUNCTION %s(a int, b int) " +
-                                  "RETURNS NULL ON NULL INPUT " +
-                                  "RETURNS int " +
-                                  "LANGUAGE java " +
-                                  "AS 'return Integer.valueOf(a + b);'");
-
-        assertInvalidMessage("does not exist or is not a scalar function",
-                             "CREATE AGGREGATE " + KEYSPACE + ".jSumFooNE2(int) " +
-                             "SFUNC " + shortFunctionName(f) + " " +
-                             "STYPE int " +
-                             "FINALFUNC jSumFooNEfinal");
-
-        execute("DROP FUNCTION " + f + "(int, int)");
-    }
-
-    @Test
-    public void testJavaAggregate() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a int primary key, b int)");
-        execute("INSERT INTO %s (a, b) VALUES (1, 1)");
-        execute("INSERT INTO %s (a, b) VALUES (2, 2)");
-        execute("INSERT INTO %s (a, b) VALUES (3, 3)");
-
-        String fState = createFunction(KEYSPACE,
-                                       "int, int",
-                                       "CREATE FUNCTION %s(a int, b int) " +
-                                       "CALLED ON NULL INPUT " +
-                                       "RETURNS int " +
-                                       "LANGUAGE java " +
-                                       "AS 'return Integer.valueOf((a!=null?a.intValue():0) + b.intValue());'");
-
-        String fFinal = createFunction(KEYSPACE,
-                                       "int",
-                                       "CREATE FUNCTION %s(a int) " +
-                                       "CALLED ON NULL INPUT " +
-                                       "RETURNS text " +
-                                       "LANGUAGE java " +
-                                       "AS 'return a.toString();'");
-
-        String a = createAggregate(KEYSPACE,
-                                   "int",
-                                   "CREATE AGGREGATE %s(int) " +
-                                   "SFUNC " + shortFunctionName(fState) + " " +
-                                   "STYPE int " +
-                                   "FINALFUNC " + shortFunctionName(fFinal) + " " +
-                                   "INITCOND 42");
-
-        // 42 + 1 + 2 + 3 = 48
-        assertRows(execute("SELECT " + a + "(b) FROM %s"), row("48"));
-
-        execute("DROP AGGREGATE " + a + "(int)");
-
-        execute("DROP FUNCTION " + fFinal + "(int)");
-        execute("DROP FUNCTION " + fState + "(int, int)");
-
-        assertInvalidMessage("Unknown function", "SELECT " + a + "(b) FROM %s");
-    }
-
-    @Test
-    public void testJavaAggregateSimple() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a int primary key, b int)");
-        execute("INSERT INTO %s (a, b) VALUES (1, 1)");
-        execute("INSERT INTO %s (a, b) VALUES (2, 2)");
-        execute("INSERT INTO %s (a, b) VALUES (3, 3)");
-
-        String fState = createFunction(KEYSPACE,
-                                       "int, int",
-                                       "CREATE FUNCTION %s(a int, b int) " +
-                                       "CALLED ON NULL INPUT " +
-                                       "RETURNS int " +
-                                       "LANGUAGE java " +
-                                       "AS 'return Integer.valueOf((a!=null?a.intValue():0) + b.intValue());'");
-
-        String a = createAggregate(KEYSPACE,
-                                   "int, int",
-                                   "CREATE AGGREGATE %s(int) " +
-                                   "SFUNC " + shortFunctionName(fState) + " " +
-                                   "STYPE int");
-
-        // 1 + 2 + 3 = 6
-        assertRows(execute("SELECT " + a + "(b) FROM %s"), row(6));
-
-        execute("DROP AGGREGATE " + a + "(int)");
-
-        execute("DROP FUNCTION " + fState + "(int, int)");
-
-        assertInvalidMessage("Unknown function", "SELECT " + a + "(b) FROM %s");
-    }
-
-    @Test
-    public void testJavaAggregateComplex() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a int primary key, b int)");
-        execute("INSERT INTO %s (a, b) VALUES (1, 1)");
-        execute("INSERT INTO %s (a, b) VALUES (2, 2)");
-        execute("INSERT INTO %s (a, b) VALUES (3, 3)");
-
-        // build an average aggregation function using
-        // tuple<bigint,int> as state
-        // double as finaltype
-
-        String fState = createFunction(KEYSPACE,
-                                       "tuple<bigint, int>, int",
-                                       "CREATE FUNCTION %s(a tuple<bigint, int>, b int) " +
-                                       "CALLED ON NULL INPUT " +
-                                       "RETURNS tuple<bigint, int> " +
-                                       "LANGUAGE java " +
-                                       "AS '" +
-                                       "a.setLong(0, a.getLong(0) + b.intValue());" +
-                                       "a.setInt(1, a.getInt(1) + 1);" +
-                                       "return a;" +
-                                       "'");
-
-        String fFinal = createFunction(KEYSPACE,
-                                       "tuple<bigint, int>",
-                                       "CREATE FUNCTION %s(a tuple<bigint, int>) " +
-                                       "RETURNS NULL ON NULL INPUT " +
-                                       "RETURNS double " +
-                                       "LANGUAGE java " +
-                                       "AS '" +
-                                       "double r = a.getLong(0);" +
-                                       "r /= a.getInt(1);" +
-                                       "return Double.valueOf(r);" +
-                                       "'");
-
-        String a = createAggregate(KEYSPACE,
-                                   "int",
-                                   "CREATE AGGREGATE %s(int) " +
-                                   "SFUNC " + shortFunctionName(fState) + " " +
-                                   "STYPE tuple<bigint, int> "+
-                                   "FINALFUNC " + shortFunctionName(fFinal) + " " +
-                                   "INITCOND (0, 0)");
-
-        // 1 + 2 + 3 = 6 / 3 = 2
-        assertRows(execute("SELECT " + a + "(b) FROM %s"), row(2d));
-
-    }
-
-    @Test
-    public void testJavascriptAggregate() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a int primary key, b int)");
-        execute("INSERT INTO %s (a, b) VALUES (1, 1)");
-        execute("INSERT INTO %s (a, b) VALUES (2, 2)");
-        execute("INSERT INTO %s (a, b) VALUES (3, 3)");
-
-        String fState = createFunction(KEYSPACE,
-                                       "int, int",
-                                       "CREATE FUNCTION %s(a int, b int) " +
-                                       "RETURNS NULL ON NULL INPUT " +
-                                       "RETURNS int " +
-                                       "LANGUAGE javascript " +
-                                       "AS 'a + b;'");
-
-        String fFinal = createFunction(KEYSPACE,
-                                       "int",
-                                       "CREATE FUNCTION %s(a int) " +
-                                       "RETURNS NULL ON NULL INPUT " +
-                                       "RETURNS text " +
-                                       "LANGUAGE javascript " +
-                                       "AS '\"\"+a'");
-
-        String a = createFunction(KEYSPACE,
-                                  "int",
-                                  "CREATE AGGREGATE %s(int) " +
-                                  "SFUNC " + shortFunctionName(fState) + " " +
-                                  "STYPE int " +
-                                  "FINALFUNC " + shortFunctionName(fFinal) + " " +
-                                  "INITCOND 42");
-
-        // 42 + 1 + 2 + 3 = 48
-        assertRows(execute("SELECT " + a + "(b) FROM %s"), row("48"));
-
-        execute("DROP AGGREGATE " + a + "(int)");
-
-        execute("DROP FUNCTION " + fFinal + "(int)");
-        execute("DROP FUNCTION " + fState + "(int, int)");
-
-        assertInvalidMessage("Unknown function", "SELECT " + a + "(b) FROM %s");
-    }
-
-    @Test
-    public void testJavascriptAggregateSimple() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a int primary key, b int)");
-        execute("INSERT INTO %s (a, b) VALUES (1, 1)");
-        execute("INSERT INTO %s (a, b) VALUES (2, 2)");
-        execute("INSERT INTO %s (a, b) VALUES (3, 3)");
-
-        String fState = createFunction(KEYSPACE,
-                                       "int, int",
-                                       "CREATE FUNCTION %s(a int, b int) " +
-                                       "CALLED ON NULL INPUT " +
-                                       "RETURNS int " +
-                                       "LANGUAGE javascript " +
-                                       "AS 'a + b;'");
-
-        String a = createAggregate(KEYSPACE,
-                                   "int, int",
-                                   "CREATE AGGREGATE %s(int) " +
-                                   "SFUNC " + shortFunctionName(fState) + " " +
-                                   "STYPE int ");
-
-        // 1 + 2 + 3 = 6
-        assertRows(execute("SELECT " + a + "(b) FROM %s"), row(6));
-
-        execute("DROP AGGREGATE " + a + "(int)");
-
-        execute("DROP FUNCTION " + fState + "(int, int)");
-
-        assertInvalidMessage("Unknown function", "SELECT " + a + "(b) FROM %s");
-    }
-
-    @Test
-    public void testFunctionDropPreparedStatement() throws Throwable
-    {
-        String otherKS = "cqltest_foo";
-
-        execute("CREATE KEYSPACE IF NOT EXISTS " + otherKS + " WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 3};");
-        try
-        {
-            execute("CREATE TABLE " + otherKS + ".jsdp (a int primary key, b int)");
-
-            String fState = createFunction(otherKS,
-                                           "int, int",
-                                           "CREATE FUNCTION %s(a int, b int) " +
-                                           "CALLED ON NULL INPUT " +
-                                           "RETURNS int " +
-                                           "LANGUAGE javascript " +
-                                           "AS 'a + b;'");
-
-            String a = createAggregate(otherKS,
-                                       "int",
-                                       "CREATE AGGREGATE %s(int) " +
-                                       "SFUNC " + shortFunctionName(fState) + " " +
-                                       "STYPE int");
-
-            ResultMessage.Prepared prepared = QueryProcessor.prepare("SELECT " + a + "(b) FROM " + otherKS + ".jsdp", ClientState.forInternalCalls(), false);
-            Assert.assertNotNull(QueryProcessor.instance.getPrepared(prepared.statementId));
-
-            execute("DROP AGGREGATE " + a + "(int)");
-            Assert.assertNull(QueryProcessor.instance.getPrepared(prepared.statementId));
-
-            //
-
-            execute("CREATE AGGREGATE " + a + "(int) " +
-                    "SFUNC " + shortFunctionName(fState) + " " +
-                    "STYPE int");
-
-            prepared = QueryProcessor.prepare("SELECT " + a + "(b) FROM " + otherKS + ".jsdp", ClientState.forInternalCalls(), false);
-            Assert.assertNotNull(QueryProcessor.instance.getPrepared(prepared.statementId));
-
-            execute("DROP KEYSPACE " + otherKS + ";");
-
-            Assert.assertNull(QueryProcessor.instance.getPrepared(prepared.statementId));
-        }
-        finally
-        {
-            execute("DROP KEYSPACE IF EXISTS " + otherKS + ";");
-        }
-    }
-
-    @Test
-    public void testAggregatesReferencedInAggregates() throws Throwable
-    {
-
-        String fState = createFunction(KEYSPACE,
-                                       "int, int",
-                                       "CREATE FUNCTION %s(a int, b int) " +
-                                       "CALLED ON NULL INPUT " +
-                                       "RETURNS int " +
-                                       "LANGUAGE javascript " +
-                                       "AS 'a + b;'");
-
-        String a = createAggregate(KEYSPACE,
-                                   "int",
-                                   "CREATE AGGREGATE %s(int) " +
-                                   "SFUNC " + shortFunctionName(fState) + " " +
-                                   "STYPE int ");
-
-        assertInvalidMessage("does not exist or is not a scalar function",
-                             "CREATE AGGREGATE " + KEYSPACE + ".aggInv(int) " +
-                             "SFUNC " + shortFunctionName(a) + " " +
-                             "STYPE int ");
-
-        assertInvalidMessage("does not exist or is not a scalar function",
-                             "CREATE AGGREGATE " + KEYSPACE + ".aggInv(int) " +
-                             "SFUNC " + shortFunctionName(fState) + " " +
-                             "STYPE int " +
-                             "FINALFUNC " + shortFunctionName(a));
-    }
-
-    @Test
-    public void testCalledOnNullInput() throws Throwable
-    {
-        String fStateNonNull = createFunction(KEYSPACE,
-                                              "int, int",
-                                              "CREATE OR REPLACE FUNCTION %s(state int, val int) " +
-                                              "RETURNS NULL ON NULL INPUT " +
-                                              "RETURNS int " +
-                                              "LANGUAGE java\n" +
-                                              "AS 'return Integer.valueOf(state + val);';");
-        String fStateNull = createFunction(KEYSPACE,
-                                           "int, int",
-                                           "CREATE OR REPLACE FUNCTION %s(state int, val int) " +
-                                           "CALLED ON NULL INPUT " +
-                                           "RETURNS int " +
-                                           "LANGUAGE java\n" +
-                                           "AS 'return Integer.valueOf(" +
-                                           "   (state != null ? state.intValue() : 0) " +
-                                           "   + (val != null ? val.intValue() : 0));';");
-        String fStateAlwaysNull = createFunction(KEYSPACE,
-                                           "int, int",
-                                           "CREATE OR REPLACE FUNCTION %s(state int, val int) " +
-                                           "CALLED ON NULL INPUT " +
-                                           "RETURNS int " +
-                                           "LANGUAGE java\n" +
-                                           "AS 'return null;';");
-        String fFinalNonNull = createFunction(KEYSPACE,
-                                              "int",
-                                              "CREATE OR REPLACE FUNCTION %s(state int) " +
-                                              "RETURNS NULL ON NULL INPUT " +
-                                              "RETURNS int " +
-                                              "LANGUAGE java\n" +
-                                              "AS 'return Integer.valueOf(state);';");
-        String fFinalNull = createFunction(KEYSPACE,
-                                           "int",
-                                           "CREATE OR REPLACE FUNCTION %s(state int) " +
-                                           "CALLED ON NULL INPUT " +
-                                           "RETURNS int " +
-                                           "LANGUAGE java\n" +
-                                           "AS 'return state;';");
-
-        assertInvalid("CREATE AGGREGATE " + KEYSPACE + ".invAggr(int) " +
-                      "SFUNC " + shortFunctionName(fStateNonNull) + " " +
-                      "STYPE int");
-        assertInvalid("CREATE AGGREGATE " + KEYSPACE + ".invAggr(int) " +
-                      "SFUNC " + shortFunctionName(fStateNonNull) + " " +
-                      "STYPE int " +
-                      "FINALFUNC " + shortFunctionName(fFinalNonNull));
-
-        String aStateNull = createAggregate(KEYSPACE,
-                                               "int",
-                                               "CREATE AGGREGATE %s(int) " +
-                                               "SFUNC " + shortFunctionName(fStateNull) + " " +
-                                               "STYPE int");
-        String aStateNullFinalNull = createAggregate(KEYSPACE,
-                                                        "int",
-                                                        "CREATE AGGREGATE %s(int) " +
-                                                        "SFUNC " + shortFunctionName(fStateNull) + " " +
-                                                        "STYPE int " +
-                                                        "FINALFUNC " + shortFunctionName(fFinalNull));
-        String aStateNullFinalNonNull = createAggregate(KEYSPACE,
-                                                        "int",
-                                                        "CREATE AGGREGATE %s(int) " +
-                                                        "SFUNC " + shortFunctionName(fStateNull) + " " +
-                                                        "STYPE int " +
-                                                        "FINALFUNC " + shortFunctionName(fFinalNonNull));
-        String aStateNonNull = createAggregate(KEYSPACE,
-                                               "int",
-                                               "CREATE AGGREGATE %s(int) " +
-                                               "SFUNC " + shortFunctionName(fStateNonNull) + " " +
-                                               "STYPE int " +
-                                               "INITCOND 0");
-        String aStateNonNullFinalNull = createAggregate(KEYSPACE,
-                                                        "int",
-                                                        "CREATE AGGREGATE %s(int) " +
-                                                        "SFUNC " + shortFunctionName(fStateNonNull) + " " +
-                                                        "STYPE int " +
-                                                        "FINALFUNC " + shortFunctionName(fFinalNull) + " " +
-                                                        "INITCOND 0");
-        String aStateNonNullFinalNonNull = createAggregate(KEYSPACE,
-                                                           "int",
-                                                           "CREATE AGGREGATE %s(int) " +
-                                                           "SFUNC " + shortFunctionName(fStateNonNull) + " " +
-                                                           "STYPE int " +
-                                                           "FINALFUNC " + shortFunctionName(fFinalNonNull) + " " +
-                                                           "INITCOND 0");
-        String aStateAlwaysNullFinalNull = createAggregate(KEYSPACE,
-                                                           "int",
-                                                           "CREATE AGGREGATE %s(int) " +
-                                                           "SFUNC " + shortFunctionName(fStateAlwaysNull) + " " +
-                                                           "STYPE int " +
-                                                           "FINALFUNC " + shortFunctionName(fFinalNull));
-        String aStateAlwaysNullFinalNonNull = createAggregate(KEYSPACE,
-                                                           "int",
-                                                           "CREATE AGGREGATE %s(int) " +
-                                                           "SFUNC " + shortFunctionName(fStateAlwaysNull) + " " +
-                                                           "STYPE int " +
-                                                           "FINALFUNC " + shortFunctionName(fFinalNonNull));
-
-        createTable("CREATE TABLE %s (key int PRIMARY KEY, i int)");
-
-        execute("INSERT INTO %s (key, i) VALUES (0, null)");
-        execute("INSERT INTO %s (key, i) VALUES (1, 1)");
-        execute("INSERT INTO %s (key, i) VALUES (2, 2)");
-        execute("INSERT INTO %s (key, i) VALUES (3, 3)");
-
-        assertRows(execute("SELECT " + aStateNull + "(i) FROM %s WHERE key = 0"), row(0));
-        assertRows(execute("SELECT " + aStateNullFinalNull + "(i) FROM %s WHERE key = 0"), row(0));
-        assertRows(execute("SELECT " + aStateNullFinalNonNull + "(i) FROM %s WHERE key = 0"), row(0));
-        assertRows(execute("SELECT " + aStateNonNull + "(i) FROM %s WHERE key = 0"), row(0));
-        assertRows(execute("SELECT " + aStateNonNullFinalNull + "(i) FROM %s WHERE key = 0"), row(0));
-        assertRows(execute("SELECT " + aStateNonNullFinalNonNull + "(i) FROM %s WHERE key = 0"), row(0));
-        assertRows(execute("SELECT " + aStateAlwaysNullFinalNull + "(i) FROM %s WHERE key = 0"), row(new Object[]{null}));
-        assertRows(execute("SELECT " + aStateAlwaysNullFinalNonNull + "(i) FROM %s WHERE key = 0"), row(new Object[]{null}));
-
-        assertRows(execute("SELECT " + aStateNull + "(i) FROM %s WHERE key = 1"), row(1));
-        assertRows(execute("SELECT " + aStateNullFinalNull + "(i) FROM %s WHERE key = 1"), row(1));
-        assertRows(execute("SELECT " + aStateNullFinalNonNull + "(i) FROM %s WHERE key = 1"), row(1));
-        assertRows(execute("SELECT " + aStateNonNull + "(i) FROM %s WHERE key = 1"), row(1));
-        assertRows(execute("SELECT " + aStateNonNullFinalNull + "(i) FROM %s WHERE key = 1"), row(1));
-        assertRows(execute("SELECT " + aStateNonNullFinalNonNull + "(i) FROM %s WHERE key = 1"), row(1));
-        assertRows(execute("SELECT " + aStateAlwaysNullFinalNull + "(i) FROM %s WHERE key = 1"), row(new Object[]{null}));
-        assertRows(execute("SELECT " + aStateAlwaysNullFinalNonNull + "(i) FROM %s WHERE key = 1"), row(new Object[]{null}));
-
-        assertRows(execute("SELECT " + aStateNull + "(i) FROM %s WHERE key IN (1, 2, 3)"), row(6));
-        assertRows(execute("SELECT " + aStateNullFinalNull + "(i) FROM %s WHERE key IN (1, 2, 3)"), row(6));
-        assertRows(execute("SELECT " + aStateNullFinalNonNull + "(i) FROM %s WHERE key IN (1, 2, 3)"), row(6));
-        assertRows(execute("SELECT " + aStateNonNull + "(i) FROM %s WHERE key IN (1, 2, 3)"), row(6));
-        assertRows(execute("SELECT " + aStateNonNullFinalNull + "(i) FROM %s WHERE key IN (1, 2, 3)"), row(6));
-        assertRows(execute("SELECT " + aStateNonNullFinalNonNull + "(i) FROM %s WHERE key IN (1, 2, 3)"), row(6));
-        assertRows(execute("SELECT " + aStateAlwaysNullFinalNull + "(i) FROM %s WHERE key IN (1, 2, 3)"), row(new Object[]{null}));
-        assertRows(execute("SELECT " + aStateAlwaysNullFinalNonNull + "(i) FROM %s WHERE key IN (1, 2, 3)"), row(new Object[]{null}));
-    }
-
-    @Test
-    public void testBrokenAggregate() throws Throwable
-    {
-        createTable("CREATE TABLE %s (key int primary key, val int)");
-        execute("INSERT INTO %s (key, val) VALUES (?, ?)", 1, 1);
-
-        String fState = createFunction(KEYSPACE,
-                                       "int, int",
-                                       "CREATE FUNCTION %s(a int, b int) " +
-                                       "CALLED ON NULL INPUT " +
-                                       "RETURNS int " +
-                                       "LANGUAGE javascript " +
-                                       "AS 'a + b;'");
-
-        String a = createAggregate(KEYSPACE,
-                                   "int",
-                                   "CREATE AGGREGATE %s(int) " +
-                                   "SFUNC " + shortFunctionName(fState) + " " +
-                                   "STYPE int ");
-
-        UDAggregate f = (UDAggregate) Functions.find(parseFunctionName(a)).get(0);
-
-        Functions.addOrReplaceFunction(UDAggregate.createBroken(f.name(), f.argTypes(), f.returnType(),
-                                                                null, new InvalidRequestException("foo bar is broken")));
-
-        assertInvalidThrowMessage("foo bar is broken", InvalidRequestException.class,
-                                  "SELECT " + a + "(val) FROM %s");
-    }
-
-    @Test
-    public void testWrongStateType() throws Throwable
-    {
-        createTable("CREATE TABLE %s (key int primary key, val int)");
-        execute("INSERT INTO %s (key, val) VALUES (?, ?)", 1, 1);
-
-        String fState = createFunction(KEYSPACE,
-                                       "int, int",
-                                       "CREATE FUNCTION %s(a int, b int) " +
-                                       "CALLED ON NULL INPUT " +
-                                       "RETURNS double " +
-                                       "LANGUAGE java " +
-                                       "AS 'return Double.valueOf(1.0);'");
-
-        String fFinal = createFunction(KEYSPACE,
-                                       "int",
-                                       "CREATE FUNCTION %s(a int) " +
-                                       "CALLED ON NULL INPUT " +
-                                       "RETURNS int " +
-                                       "LANGUAGE java " +
-                                       "AS 'return Integer.valueOf(1);';");
-
-        assertInvalidMessage("return type must be the same as the first argument type - check STYPE, argument and return types",
-                             "CREATE AGGREGATE %s(int) " +
-                             "SFUNC " + shortFunctionName(fState) + ' ' +
-                             "STYPE int " +
-                             "FINALFUNC " + shortFunctionName(fFinal) + ' ' +
-                             "INITCOND 1");
-    }
-
-    @Test
-    public void testWrongKeyspace() throws Throwable
-    {
-        String typeName = createType("CREATE TYPE %s (txt text, i int)");
-        String type = KEYSPACE + '.' + typeName;
-
-        String fState = createFunction(KEYSPACE_PER_TEST,
-                                       "int, int",
-                                       "CREATE FUNCTION %s(a int, b int) " +
-                                       "CALLED ON NULL INPUT " +
-                                       "RETURNS double " +
-                                       "LANGUAGE java " +
-                                       "AS 'return Double.valueOf(1.0);'");
-
-        String fFinal = createFunction(KEYSPACE_PER_TEST,
-                                       "int",
-                                       "CREATE FUNCTION %s(a int) " +
-                                       "CALLED ON NULL INPUT " +
-                                       "RETURNS int " +
-                                       "LANGUAGE java " +
-                                       "AS 'return Integer.valueOf(1);';");
-
-        String fStateWrong = createFunction(KEYSPACE,
-                                       "int, int",
-                                       "CREATE FUNCTION %s(a int, b int) " +
-                                       "CALLED ON NULL INPUT " +
-                                       "RETURNS double " +
-                                       "LANGUAGE java " +
-                                       "AS 'return Double.valueOf(1.0);'");
-
-        String fFinalWrong = createFunction(KEYSPACE,
-                                       "int",
-                                       "CREATE FUNCTION %s(a int) " +
-                                       "CALLED ON NULL INPUT " +
-                                       "RETURNS int " +
-                                       "LANGUAGE java " +
-                                       "AS 'return Integer.valueOf(1);';");
-
-        assertInvalidMessage(String.format("Statement on keyspace %s cannot refer to a user type in keyspace %s; user types can only be used in the keyspace they are defined in",
-                                           KEYSPACE_PER_TEST, KEYSPACE),
-                             "CREATE AGGREGATE " + KEYSPACE_PER_TEST + ".test_wrong_ks(int) " +
-                             "SFUNC " + shortFunctionName(fState) + ' ' +
-                             "STYPE " + type + " " +
-                             "FINALFUNC " + shortFunctionName(fFinal) + ' ' +
-                             "INITCOND 1");
-
-        assertInvalidMessage(String.format("Statement on keyspace %s cannot refer to a user type in keyspace %s; user types can only be used in the keyspace they are defined in",
-                                           KEYSPACE_PER_TEST, KEYSPACE),
-                             "CREATE AGGREGATE " + KEYSPACE_PER_TEST + ".test_wrong_ks(int) " +
-                             "SFUNC " + fStateWrong + ' ' +
-                             "STYPE " + type + " " +
-                             "FINALFUNC " + shortFunctionName(fFinal) + ' ' +
-                             "INITCOND 1");
-
-        assertInvalidMessage(String.format("Statement on keyspace %s cannot refer to a user type in keyspace %s; user types can only be used in the keyspace they are defined in",
-                                           KEYSPACE_PER_TEST, KEYSPACE),
-                             "CREATE AGGREGATE " + KEYSPACE_PER_TEST + ".test_wrong_ks(int) " +
-                             "SFUNC " + shortFunctionName(fState) + ' ' +
-                             "STYPE " + type + " " +
-                             "FINALFUNC " + fFinalWrong + ' ' +
-                             "INITCOND 1");
-    }
-
-    @Test
-    public void testSystemKeyspace() throws Throwable
-    {
-        String fState = createFunction(KEYSPACE,
-                                       "text, text",
-                                       "CREATE FUNCTION %s(a text, b text) " +
-                                       "CALLED ON NULL INPUT " +
-                                       "RETURNS text " +
-                                       "LANGUAGE java " +
-                                       "AS 'return \"foobar\";'");
-
-        createAggregate(KEYSPACE,
-                        "text",
-                        "CREATE AGGREGATE %s(text) " +
-                        "SFUNC " + shortFunctionName(fState) + ' ' +
-                        "STYPE text " +
-                        "FINALFUNC system.varcharasblob " +
-                        "INITCOND 'foobar'");
-    }
-
-    @Test
-    public void testFunctionWithFrozenSetType() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a int PRIMARY KEY, b frozen<set<int>>)");
-        createIndex("CREATE INDEX ON %s (FULL(b))");
-
-        execute("INSERT INTO %s (a, b) VALUES (?, ?)", 0, set());
-        execute("INSERT INTO %s (a, b) VALUES (?, ?)", 1, set(1, 2, 3));
-        execute("INSERT INTO %s (a, b) VALUES (?, ?)", 2, set(4, 5, 6));
-        execute("INSERT INTO %s (a, b) VALUES (?, ?)", 3, set(7, 8, 9));
-
-        String fState = createFunction(KEYSPACE,
-                                       "set<int>",
-                                       "CREATE FUNCTION %s (state set<int>, values set<int>) " +
-                                       "CALLED ON NULL INPUT " +
-                                       "RETURNS set<int> " +
-                                       "LANGUAGE java\n" +
-                                       "AS 'return values;';");
-
-        String fFinal = createFunction(KEYSPACE,
-                                       "set<int>",
-                                       "CREATE FUNCTION %s(state set<int>) " +
-                                       "CALLED ON NULL INPUT " +
-                                       "RETURNS set<int> " +
-                                       "LANGUAGE java " +
-                                       "AS 'return state;'");
-
-        assertInvalidMessage("The function state type should not be frozen",
-                             "CREATE AGGREGATE %s(set<int>) " +
-                             "SFUNC " + fState + " " +
-                             "STYPE frozen<set<int>> " +
-                             "FINALFUNC " + fFinal + " " +
-                             "INITCOND null");
-
-        String aggregation = createAggregate(KEYSPACE,
-                                             "set<int>",
-                                             "CREATE AGGREGATE %s(set<int>) " +
-                                             "SFUNC " + fState + " " +
-                                             "STYPE set<int> " +
-                                             "FINALFUNC " + fFinal + " " +
-                                             "INITCOND null");
-
-        assertRows(execute("SELECT " + aggregation + "(b) FROM %s"),
-                   row(set(7, 8, 9)));
-
-        assertInvalidMessage("The function arguments should not be frozen",
-                             "DROP AGGREGATE %s (frozen<set<int>>);");
-    }
-
-    @Test
-    public void testFunctionWithFrozenListType() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a int PRIMARY KEY, b frozen<list<int>>)");
-        createIndex("CREATE INDEX ON %s (FULL(b))");
-
-        execute("INSERT INTO %s (a, b) VALUES (?, ?)", 0, list());
-        execute("INSERT INTO %s (a, b) VALUES (?, ?)", 1, list(1, 2, 3));
-        execute("INSERT INTO %s (a, b) VALUES (?, ?)", 2, list(4, 5, 6));
-        execute("INSERT INTO %s (a, b) VALUES (?, ?)", 3, list(7, 8, 9));
-
-        String fState = createFunction(KEYSPACE,
-                                       "list<int>",
-                                       "CREATE FUNCTION %s (state list<int>, values list<int>) " +
-                                       "CALLED ON NULL INPUT " +
-                                       "RETURNS list<int> " +
-                                       "LANGUAGE java\n" +
-                                       "AS 'return values;';");
-
-        String fFinal = createFunction(KEYSPACE,
-                                       "list<int>",
-                                       "CREATE FUNCTION %s(state list<int>) " +
-                                       "CALLED ON NULL INPUT " +
-                                       "RETURNS list<int> " +
-                                       "LANGUAGE java " +
-                                       "AS 'return state;'");
-
-        assertInvalidMessage("The function state type should not be frozen",
-                             "CREATE AGGREGATE %s(list<int>) " +
-                             "SFUNC " + fState + " " +
-                             "STYPE frozen<list<int>> " +
-                             "FINALFUNC " + fFinal + " " +
-                             "INITCOND null");
-
-        String aggregation = createAggregate(KEYSPACE,
-                                             "list<int>",
-                                             "CREATE AGGREGATE %s(list<int>) " +
-                                             "SFUNC " + fState + " " +
-                                             "STYPE list<int> " +
-                                             "FINALFUNC " + fFinal + " " +
-                                             "INITCOND null");
-
-        assertRows(execute("SELECT " + aggregation + "(b) FROM %s"),
-                   row(list(7, 8, 9)));
-
-        assertInvalidMessage("The function arguments should not be frozen",
-                             "DROP AGGREGATE %s (frozen<list<int>>);");
-    }
-
-    @Test
-    public void testFunctionWithFrozenMapType() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a int PRIMARY KEY, b frozen<map<int, int>>)");
-        createIndex("CREATE INDEX ON %s (FULL(b))");
-
-        execute("INSERT INTO %s (a, b) VALUES (?, ?)", 0, map());
-        execute("INSERT INTO %s (a, b) VALUES (?, ?)", 1, map(1, 2, 3, 4));
-        execute("INSERT INTO %s (a, b) VALUES (?, ?)", 2, map(4, 5, 6, 7));
-        execute("INSERT INTO %s (a, b) VALUES (?, ?)", 3, map(7, 8, 9, 10));
-
-        String fState = createFunction(KEYSPACE,
-                                       "map<int, int>",
-                                       "CREATE FUNCTION %s (state map<int, int>, values map<int, int>) " +
-                                       "CALLED ON NULL INPUT " +
-                                       "RETURNS map<int, int> " +
-                                       "LANGUAGE java\n" +
-                                       "AS 'return values;';");
-
-        String fFinal = createFunction(KEYSPACE,
-                                       "map<int, int>",
-                                       "CREATE FUNCTION %s(state map<int, int>) " +
-                                       "CALLED ON NULL INPUT " +
-                                       "RETURNS map<int, int> " +
-                                       "LANGUAGE java " +
-                                       "AS 'return state;'");
-
-        assertInvalidMessage("The function state type should not be frozen",
-                             "CREATE AGGREGATE %s(map<int, int>) " +
-                             "SFUNC " + fState + " " +
-                             "STYPE frozen<map<int, int>> " +
-                             "FINALFUNC " + fFinal + " " +
-                             "INITCOND null");
-
-        String aggregation = createAggregate(KEYSPACE,
-                                             "map<int, int>",
-                                             "CREATE AGGREGATE %s(map<int, int>) " +
-                                             "SFUNC " + fState + " " +
-                                             "STYPE map<int, int> " +
-                                             "FINALFUNC " + fFinal + " " +
-                                             "INITCOND null");
-
-        assertRows(execute("SELECT " + aggregation + "(b) FROM %s"),
-                   row(map(7, 8, 9, 10)));
-
-        assertInvalidMessage("The function arguments should not be frozen",
-                             "DROP AGGREGATE %s (frozen<map<int, int>>);");
-    }
-
-    @Test
-    public void testFunctionWithFrozenTupleType() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a int PRIMARY KEY, b frozen<tuple<int, int>>)");
-        createIndex("CREATE INDEX ON %s (b)");
-
-        execute("INSERT INTO %s (a, b) VALUES (?, ?)", 0, tuple());
-        execute("INSERT INTO %s (a, b) VALUES (?, ?)", 1, tuple(1, 2));
-        execute("INSERT INTO %s (a, b) VALUES (?, ?)", 2, tuple(4, 5));
-        execute("INSERT INTO %s (a, b) VALUES (?, ?)", 3, tuple(7, 8));
-
-        String fState = createFunction(KEYSPACE,
-                                       "tuple<int, int>",
-                                       "CREATE FUNCTION %s (state tuple<int, int>, values tuple<int, int>) " +
-                                       "CALLED ON NULL INPUT " +
-                                       "RETURNS tuple<int, int> " +
-                                       "LANGUAGE java\n" +
-                                       "AS 'return values;';");
-
-        String fFinal = createFunction(KEYSPACE,
-                                       "tuple<int, int>",
-                                       "CREATE FUNCTION %s(state tuple<int, int>) " +
-                                       "CALLED ON NULL INPUT " +
-                                       "RETURNS tuple<int, int> " +
-                                       "LANGUAGE java " +
-                                       "AS 'return state;'");
-
-        assertInvalidMessage("The function state type should not be frozen",
-                             "CREATE AGGREGATE %s(tuple<int, int>) " +
-                             "SFUNC " + fState + " " +
-                             "STYPE frozen<tuple<int, int>> " +
-                             "FINALFUNC " + fFinal + " " +
-                             "INITCOND null");
-
-        String aggregation = createAggregate(KEYSPACE,
-                                             "tuple<int, int>",
-                                             "CREATE AGGREGATE %s(tuple<int, int>) " +
-                                             "SFUNC " + fState + " " +
-                                             "STYPE tuple<int, int> " +
-                                             "FINALFUNC " + fFinal + " " +
-                                             "INITCOND null");
-
-        assertRows(execute("SELECT " + aggregation + "(b) FROM %s"),
-                   row(tuple(7, 8)));
-
-        assertInvalidMessage("The function arguments should not be frozen",
-                             "DROP AGGREGATE %s (frozen<tuple<int, int>>);");
-    }
-
-    @Test
-    public void testFunctionWithFrozenUDFType() throws Throwable
-    {
-        String myType = createType("CREATE TYPE %s (f int)");
-        createTable("CREATE TABLE %s (a int PRIMARY KEY, b frozen<" + myType + ">)");
-        createIndex("CREATE INDEX ON %s (b)");
-
-        execute("INSERT INTO %s (a, b) VALUES (?, {f : ?})", 0, 1);
-        execute("INSERT INTO %s (a, b) VALUES (?, {f : ?})", 1, 2);
-        execute("INSERT INTO %s (a, b) VALUES (?, {f : ?})", 2, 4);
-        execute("INSERT INTO %s (a, b) VALUES (?, {f : ?})", 3, 7);
-
-        String fState = createFunction(KEYSPACE,
-                                       "tuple<int, int>",
-                                       "CREATE FUNCTION %s (state " + myType + ", values " + myType + ") " +
-                                       "CALLED ON NULL INPUT " +
-                                       "RETURNS " + myType + " " +
-                                       "LANGUAGE java\n" +
-                                       "AS 'return values;';");
-
-        String fFinal = createFunction(KEYSPACE,
-                                       myType,
-                                       "CREATE FUNCTION %s(state " + myType + ") " +
-                                       "CALLED ON NULL INPUT " +
-                                       "RETURNS " + myType + " " +
-                                       "LANGUAGE java " +
-                                       "AS 'return state;'");
-
-        assertInvalidMessage("The function state type should not be frozen",
-                             "CREATE AGGREGATE %s(" + myType + ") " +
-                             "SFUNC " + fState + " " +
-                             "STYPE frozen<" + myType + "> " +
-                             "FINALFUNC " + fFinal + " " +
-                             "INITCOND null");
-
-        String aggregation = createAggregate(KEYSPACE,
-                                             myType,
-                                             "CREATE AGGREGATE %s(" + myType + ") " +
-                                             "SFUNC " + fState + " " +
-                                             "STYPE " + myType + " " +
-                                             "FINALFUNC " + fFinal + " " +
-                                             "INITCOND null");
-
-        assertRows(execute("SELECT " + aggregation + "(b).f FROM %s"),
-                   row(7));
-
-        assertInvalidMessage("The function arguments should not be frozen",
-                             "DROP AGGREGATE %s (frozen<" + myType + ">);");
-    }
-
-    @Test
-    public void testEmptyValues() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a int primary key, b text)");
-        execute("INSERT INTO %s (a, b) VALUES (1, '')");
-        execute("INSERT INTO %s (a, b) VALUES (2, '')");
-        execute("INSERT INTO %s (a, b) VALUES (3, '')");
-
-        String fCON = createFunction(KEYSPACE,
-                                     "text, text",
-                                     "CREATE FUNCTION %s(a text, b text) " +
-                                     "CALLED ON NULL INPUT " +
-                                     "RETURNS text " +
-                                     "LANGUAGE java " +
-                                     "AS 'return a + \"x\" + b + \"y\";'");
-
-        String fCONf = createFunction(KEYSPACE,
-                                     "text",
-                                     "CREATE FUNCTION %s(a text) " +
-                                     "CALLED ON NULL INPUT " +
-                                     "RETURNS text " +
-                                     "LANGUAGE java " +
-                                     "AS 'return \"fin\" + a;'");
-
-        String aCON = createAggregate(KEYSPACE,
-                                      "text",
-                                      "CREATE AGGREGATE %s(text) " +
-                                      "SFUNC " + shortFunctionName(fCON) + ' ' +
-                                      "STYPE text " +
-                                      "FINALFUNC " + shortFunctionName(fCONf) + ' ' +
-                                      "INITCOND ''");
-
-        String fRNON = createFunction(KEYSPACE,
-                                      "text",
-                                      "CREATE FUNCTION %s(a text, b text) " +
-                                      "RETURNS NULL ON NULL INPUT " +
-                                      "RETURNS text " +
-                                      "LANGUAGE java " +
-                                      "AS 'return a + \"x\" + b + \"y\";'");
-
-        String fRNONf = createFunction(KEYSPACE,
-                                      "text",
-                                      "CREATE FUNCTION %s(a text) " +
-                                      "RETURNS NULL ON NULL INPUT " +
-                                      "RETURNS text " +
-                                      "LANGUAGE java " +
-                                      "AS 'return \"fin\" + a;'");
-
-        String aRNON = createAggregate(KEYSPACE,
-                                      "int",
-                                      "CREATE AGGREGATE %s(text) " +
-                                      "SFUNC " + shortFunctionName(fRNON) + ' ' +
-                                      "STYPE text " +
-                                      "FINALFUNC " + shortFunctionName(fRNONf) + ' ' +
-                                      "INITCOND ''");
-
-        assertRows(execute("SELECT " + aCON + "(b) FROM %s"), row("finxyxyxy"));
-        assertRows(execute("SELECT " + aRNON + "(b) FROM %s"), row("finxyxyxy"));
-
-        createTable("CREATE TABLE %s (a int primary key, b text)");
-        execute("INSERT INTO %s (a, b) VALUES (1, null)");
-        execute("INSERT INTO %s (a, b) VALUES (2, null)");
-        execute("INSERT INTO %s (a, b) VALUES (3, null)");
-
-        assertRows(execute("SELECT " + aCON + "(b) FROM %s"), row("finxnullyxnullyxnully"));
-        assertRows(execute("SELECT " + aRNON + "(b) FROM %s"), row("fin"));
-
-    }
-
-    @Test
-    public void testSystemKsFuncs() throws Throwable
-    {
-
-        String fAdder = createFunction(KEYSPACE,
-                                      "int, int",
-                                      "CREATE FUNCTION %s(a int, b int) " +
-                                      "CALLED ON NULL INPUT " +
-                                      "RETURNS int " +
-                                      "LANGUAGE java " +
-                                      "AS 'return (a != null ? a : 0) + (b != null ? b : 0);'");
-
-        String aAggr = createAggregate(KEYSPACE,
-                                      "int",
-                                      "CREATE AGGREGATE %s(int) " +
-                                      "SFUNC " + shortFunctionName(fAdder) + ' ' +
-                                      "STYPE int " +
-                                      "FINALFUNC intasblob");
-
-        createTable("CREATE TABLE %s (a int primary key, b int)");
-        execute("INSERT INTO %s (a, b) VALUES (1, 1)");
-        execute("INSERT INTO %s (a, b) VALUES (2, 2)");
-        execute("INSERT INTO %s (a, b) VALUES (3, 3)");
-
-        assertRows(execute("SELECT " + aAggr + "(b) FROM %s"), row(Int32Serializer.instance.serialize(6)));
-
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/01115f72/test/unit/org/apache/cassandra/cql3/AliasTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/AliasTest.java b/test/unit/org/apache/cassandra/cql3/AliasTest.java
deleted file mode 100644
index 132aa04..0000000
--- a/test/unit/org/apache/cassandra/cql3/AliasTest.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/*
- * 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;
-
-import org.junit.Test;
-
-public class AliasTest extends CQLTester
-{
-    @Test
-    public void testAlias() throws Throwable
-    {
-        createTable("CREATE TABLE %s (id int PRIMARY KEY, name text)");
-
-        for (int i = 0; i < 5; i++)
-            execute("INSERT INTO %s (id, name) VALUES (?, ?) USING TTL 10 AND TIMESTAMP 0", i, Integer.toString(i));
-
-        assertInvalidMessage("Aliases aren't allowed in the where clause" ,
-                             "SELECT id AS user_id, name AS user_name FROM %s WHERE user_id = 0");
-
-        // test that select throws a meaningful exception for aliases in order by clause
-        assertInvalidMessage("Aliases are not allowed in order by clause",
-                             "SELECT id AS user_id, name AS user_name FROM %s WHERE id IN (0) ORDER BY user_name");
-
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/01115f72/test/unit/org/apache/cassandra/cql3/AlterTableTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/AlterTableTest.java b/test/unit/org/apache/cassandra/cql3/AlterTableTest.java
deleted file mode 100644
index 4540f3c..0000000
--- a/test/unit/org/apache/cassandra/cql3/AlterTableTest.java
+++ /dev/null
@@ -1,113 +0,0 @@
-/*
- * 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;
-
-import org.apache.cassandra.db.ColumnFamilyStore;
-import org.apache.cassandra.db.Keyspace;
-import org.junit.Test;
-
-import static org.junit.Assert.assertEquals;
-
-public class AlterTableTest extends CQLTester
-{
-    @Test
-    public void testAddList() throws Throwable
-    {
-        createTable("CREATE TABLE %s (id text PRIMARY KEY, content text);");
-        execute("ALTER TABLE %s ADD myCollection list<text>;");
-        execute("INSERT INTO %s (id, content , myCollection) VALUES ('test', 'first test', ['first element']);");
-
-        assertRows(execute("SELECT * FROM %s;"), row("test", "first test", list("first element")));
-    }
-
-    @Test
-    public void testDropList() throws Throwable
-    {
-        createTable("CREATE TABLE %s (id text PRIMARY KEY, content text, myCollection list<text>);");
-        execute("INSERT INTO %s (id, content , myCollection) VALUES ('test', 'first test', ['first element']);");
-        execute("ALTER TABLE %s DROP myCollection;");
-
-        assertRows(execute("SELECT * FROM %s;"), row("test", "first test"));
-    }
-    @Test
-    public void testAddMap() throws Throwable
-    {
-        createTable("CREATE TABLE %s (id text PRIMARY KEY, content text);");
-        execute("ALTER TABLE %s ADD myCollection map<text, text>;");
-        execute("INSERT INTO %s (id, content , myCollection) VALUES ('test', 'first test', { '1' : 'first element'});");
-
-        assertRows(execute("SELECT * FROM %s;"), row("test", "first test", map("1", "first element")));
-    }
-
-    @Test
-    public void testDropMap() throws Throwable
-    {
-        createTable("CREATE TABLE %s (id text PRIMARY KEY, content text, myCollection map<text, text>);");
-        execute("INSERT INTO %s (id, content , myCollection) VALUES ('test', 'first test', { '1' : 'first element'});");
-        execute("ALTER TABLE %s DROP myCollection;");
-
-        assertRows(execute("SELECT * FROM %s;"), row("test", "first test"));
-    }
-
-    @Test
-    public void testDropListAndAddListWithSameName() throws Throwable
-    {
-        createTable("CREATE TABLE %s (id text PRIMARY KEY, content text, myCollection list<text>);");
-        execute("INSERT INTO %s (id, content , myCollection) VALUES ('test', 'first test', ['first element']);");
-        execute("ALTER TABLE %s DROP myCollection;");
-        execute("ALTER TABLE %s ADD myCollection list<text>;");
-
-        assertRows(execute("SELECT * FROM %s;"), row("test", "first test", null));
-        execute("UPDATE %s set myCollection = ['second element'] WHERE id = 'test';");
-        assertRows(execute("SELECT * FROM %s;"), row("test", "first test", list("second element")));
-    }
-    @Test
-    public void testDropListAndAddMapWithSameName() throws Throwable
-    {
-        createTable("CREATE TABLE %s (id text PRIMARY KEY, content text, myCollection list<text>);");
-        execute("INSERT INTO %s (id, content , myCollection) VALUES ('test', 'first test', ['first element']);");
-        execute("ALTER TABLE %s DROP myCollection;");
-
-        assertInvalid("ALTER TABLE %s ADD myCollection map<int, int>;");
-    }
-
-    @Test
-    public void testChangeStrategyWithUnquotedAgrument() throws Throwable
-    {
-        createTable("CREATE TABLE %s (id text PRIMARY KEY);");
-
-        assertInvalidSyntaxMessage("no viable alternative at input '}'",
-                                   "ALTER TABLE %s WITH caching = {'keys' : 'all', 'rows_per_partition' : ALL};");
-    }
-
-    @Test
-    // tests CASSANDRA-7976
-    public void testAlterIndexInterval() throws Throwable
-    {
-        String tableName = createTable("CREATE TABLE IF NOT EXISTS %s (id uuid, album text, artist text, data blob, PRIMARY KEY (id))");
-        ColumnFamilyStore cfs = Keyspace.open(KEYSPACE).getColumnFamilyStore(tableName);
-
-        alterTable("ALTER TABLE %s WITH min_index_interval=256 AND max_index_interval=512");
-        assertEquals(256, cfs.metadata.getMinIndexInterval());
-        assertEquals(512, cfs.metadata.getMaxIndexInterval());
-
-        alterTable("ALTER TABLE %s WITH caching = 'none'");
-        assertEquals(256, cfs.metadata.getMinIndexInterval());
-        assertEquals(512, cfs.metadata.getMaxIndexInterval());
-    }
-}


[23/32] cassandra git commit: 2.2 commit for CASSANDRA-9160

Posted by jm...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/01115f72/test/unit/org/apache/cassandra/cql3/UFTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/UFTest.java b/test/unit/org/apache/cassandra/cql3/UFTest.java
deleted file mode 100644
index 40f2dd3..0000000
--- a/test/unit/org/apache/cassandra/cql3/UFTest.java
+++ /dev/null
@@ -1,2585 +0,0 @@
-/*
- * 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;
-
-import java.math.BigDecimal;
-import java.math.BigInteger;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Date;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.TreeMap;
-import java.util.TreeSet;
-import java.util.UUID;
-
-import org.junit.Assert;
-import org.junit.Test;
-
-import com.datastax.driver.core.*;
-import com.datastax.driver.core.exceptions.InvalidQueryException;
-import org.apache.cassandra.cql3.functions.FunctionName;
-import org.apache.cassandra.cql3.functions.Functions;
-import org.apache.cassandra.cql3.functions.UDFunction;
-import org.apache.cassandra.db.marshal.CollectionType;
-import org.apache.cassandra.exceptions.FunctionExecutionException;
-import org.apache.cassandra.exceptions.InvalidRequestException;
-import org.apache.cassandra.service.ClientState;
-import org.apache.cassandra.transport.Event;
-import org.apache.cassandra.transport.Server;
-import org.apache.cassandra.transport.messages.ResultMessage;
-import org.apache.cassandra.utils.ByteBufferUtil;
-import org.apache.cassandra.utils.UUIDGen;
-
-public class UFTest extends CQLTester
-{
-
-    @Test
-    public void testSchemaChange() throws Throwable
-    {
-        String f = createFunction(KEYSPACE,
-                                  "double, double",
-                                  "CREATE OR REPLACE FUNCTION %s(state double, val double) " +
-                                  "RETURNS NULL ON NULL INPUT " +
-                                  "RETURNS double " +
-                                  "LANGUAGE javascript " +
-                                  "AS '\"string\";';");
-
-        assertLastSchemaChange(Event.SchemaChange.Change.CREATED, Event.SchemaChange.Target.FUNCTION,
-                               KEYSPACE, parseFunctionName(f).name,
-                               "double", "double");
-
-        createFunctionOverload(f,
-                               "double, double",
-                               "CREATE OR REPLACE FUNCTION %s(state int, val int) " +
-                               "RETURNS NULL ON NULL INPUT " +
-                               "RETURNS int " +
-                               "LANGUAGE javascript " +
-                               "AS '\"string\";';");
-
-        assertLastSchemaChange(Event.SchemaChange.Change.CREATED, Event.SchemaChange.Target.FUNCTION,
-                               KEYSPACE, parseFunctionName(f).name,
-                               "int", "int");
-
-        schemaChange("CREATE OR REPLACE FUNCTION " + f + "(state int, val int) " +
-                     "RETURNS NULL ON NULL INPUT " +
-                     "RETURNS int " +
-                     "LANGUAGE javascript " +
-                     "AS '\"string\";';");
-
-        assertLastSchemaChange(Event.SchemaChange.Change.UPDATED, Event.SchemaChange.Target.FUNCTION,
-                               KEYSPACE, parseFunctionName(f).name,
-                               "int", "int");
-
-        schemaChange("DROP FUNCTION " + f + "(double, double)");
-
-        assertLastSchemaChange(Event.SchemaChange.Change.DROPPED, Event.SchemaChange.Target.FUNCTION,
-                               KEYSPACE, parseFunctionName(f).name,
-                               "double", "double");
-    }
-
-    @Test
-    public void testFunctionDropOnKeyspaceDrop() throws Throwable
-    {
-        String fSin = createFunction(KEYSPACE_PER_TEST, "double",
-                                     "CREATE FUNCTION %s ( input double ) " +
-                                     "CALLED ON NULL INPUT " +
-                                     "RETURNS double " +
-                                     "LANGUAGE java " +
-                                     "AS 'return Double.valueOf(Math.sin(input.doubleValue()));'");
-
-        FunctionName fSinName = parseFunctionName(fSin);
-
-        Assert.assertEquals(1, Functions.find(parseFunctionName(fSin)).size());
-
-        assertRows(execute("SELECT function_name, language FROM system.schema_functions WHERE keyspace_name=?", KEYSPACE_PER_TEST),
-                   row(fSinName.name, "java"));
-
-        dropPerTestKeyspace();
-
-        assertRows(execute("SELECT function_name, language FROM system.schema_functions WHERE keyspace_name=?", KEYSPACE_PER_TEST));
-
-        Assert.assertEquals(0, Functions.find(fSinName).size());
-    }
-
-    @Test
-    public void testFunctionDropPreparedStatement() throws Throwable
-    {
-        createTable("CREATE TABLE %s (key int PRIMARY KEY, d double)");
-
-        String fSin = createFunction(KEYSPACE_PER_TEST, "double",
-                                     "CREATE FUNCTION %s ( input double ) " +
-                                     "CALLED ON NULL INPUT " +
-                                     "RETURNS double " +
-                                     "LANGUAGE java " +
-                                     "AS 'return Double.valueOf(Math.sin(input.doubleValue()));'");
-
-        FunctionName fSinName = parseFunctionName(fSin);
-
-        Assert.assertEquals(1, Functions.find(parseFunctionName(fSin)).size());
-
-        // create a pairs of Select and Inserts. One statement in each pair uses the function so when we
-        // drop it those statements should be removed from the cache in QueryProcessor. The other statements
-        // should be unaffected.
-
-        ResultMessage.Prepared preparedSelect1 = QueryProcessor.prepare(
-                                                    String.format("SELECT key, %s(d) FROM %s.%s", fSin, KEYSPACE, currentTable()),
-                                                    ClientState.forInternalCalls(), false);
-        ResultMessage.Prepared preparedSelect2 = QueryProcessor.prepare(
-                                                    String.format("SELECT key FROM %s.%s", KEYSPACE, currentTable()),
-                                                    ClientState.forInternalCalls(), false);
-        ResultMessage.Prepared preparedInsert1 = QueryProcessor.prepare(
-                                                      String.format("INSERT INTO %s.%s (key, d) VALUES (?, %s(?))", KEYSPACE, currentTable(), fSin),
-                                                      ClientState.forInternalCalls(), false);
-        ResultMessage.Prepared preparedInsert2 = QueryProcessor.prepare(
-                                                      String.format("INSERT INTO %s.%s (key, d) VALUES (?, ?)", KEYSPACE, currentTable()),
-                                                      ClientState.forInternalCalls(), false);
-
-        Assert.assertNotNull(QueryProcessor.instance.getPrepared(preparedSelect1.statementId));
-        Assert.assertNotNull(QueryProcessor.instance.getPrepared(preparedSelect2.statementId));
-        Assert.assertNotNull(QueryProcessor.instance.getPrepared(preparedInsert1.statementId));
-        Assert.assertNotNull(QueryProcessor.instance.getPrepared(preparedInsert2.statementId));
-
-        execute("DROP FUNCTION " + fSin + "(double);");
-
-        // the statements which use the dropped function should be removed from cache, with the others remaining
-        Assert.assertNull(QueryProcessor.instance.getPrepared(preparedSelect1.statementId));
-        Assert.assertNotNull(QueryProcessor.instance.getPrepared(preparedSelect2.statementId));
-        Assert.assertNull(QueryProcessor.instance.getPrepared(preparedInsert1.statementId));
-        Assert.assertNotNull(QueryProcessor.instance.getPrepared(preparedInsert2.statementId));
-
-        execute("CREATE FUNCTION " + fSin + " ( input double ) " +
-                "RETURNS NULL ON NULL INPUT " +
-                "RETURNS double " +
-                "LANGUAGE java " +
-                "AS 'return Double.valueOf(Math.sin(input));'");
-
-        Assert.assertEquals(1, Functions.find(fSinName).size());
-
-        preparedSelect1= QueryProcessor.prepare(
-                                         String.format("SELECT key, %s(d) FROM %s.%s", fSin, KEYSPACE, currentTable()),
-                                         ClientState.forInternalCalls(), false);
-        preparedInsert1 = QueryProcessor.prepare(
-                                         String.format("INSERT INTO %s.%s (key, d) VALUES (?, %s(?))", KEYSPACE, currentTable(), fSin),
-                                         ClientState.forInternalCalls(), false);
-        Assert.assertNotNull(QueryProcessor.instance.getPrepared(preparedSelect1.statementId));
-        Assert.assertNotNull(QueryProcessor.instance.getPrepared(preparedInsert1.statementId));
-
-        dropPerTestKeyspace();
-
-        // again, only the 2 statements referencing the function should be removed from cache
-        // this time because the statements select from tables in KEYSPACE, only the function
-        // is scoped to KEYSPACE_PER_TEST
-        Assert.assertNull(QueryProcessor.instance.getPrepared(preparedSelect1.statementId));
-        Assert.assertNotNull(QueryProcessor.instance.getPrepared(preparedSelect2.statementId));
-        Assert.assertNull(QueryProcessor.instance.getPrepared(preparedInsert1.statementId));
-        Assert.assertNotNull(QueryProcessor.instance.getPrepared(preparedInsert2.statementId));
-    }
-
-    @Test
-    public void testDropFunctionDropsPreparedStatementsWithDelayedValues() throws Throwable
-    {
-        // test that dropping a function removes stmts which use
-        // it to provide a DelayedValue collection from the
-        // cache in QueryProcessor
-        checkDelayedValuesCorrectlyIdentifyFunctionsInUse(false);
-    }
-
-    @Test
-    public void testDropKeyspaceContainingFunctionDropsPreparedStatementsWithDelayedValues() throws Throwable
-    {
-        // test that dropping a function removes stmts which use
-        // it to provide a DelayedValue collection from the
-        // cache in QueryProcessor
-        checkDelayedValuesCorrectlyIdentifyFunctionsInUse(true);
-    }
-
-    private ResultMessage.Prepared prepareStatementWithDelayedValue(CollectionType.Kind kind, String function)
-    {
-        String collectionType;
-        String literalArgs;
-        switch (kind)
-        {
-            case LIST:
-                collectionType = "list<double>";
-                literalArgs = String.format("[%s(0.0)]", function);
-                break;
-            case SET:
-                collectionType = "set<double>";
-                literalArgs = String.format("{%s(0.0)}", function);
-                break;
-            case MAP:
-                collectionType = "map<double, double>";
-                literalArgs = String.format("{%s(0.0):0.0}", function);
-                break;
-            default:
-                Assert.fail("Unsupported collection type " + kind);
-                collectionType = null;
-                literalArgs = null;
-        }
-
-        createTable("CREATE TABLE %s (" +
-                    " key int PRIMARY KEY," +
-                    " val " + collectionType + ')');
-
-        ResultMessage.Prepared prepared = QueryProcessor.prepare(
-                                                                String.format("INSERT INTO %s.%s (key, val) VALUES (?, %s)",
-                                                                             KEYSPACE,
-                                                                             currentTable(),
-                                                                             literalArgs),
-                                                                ClientState.forInternalCalls(), false);
-        Assert.assertNotNull(QueryProcessor.instance.getPrepared(prepared.statementId));
-        return prepared;
-    }
-
-    private ResultMessage.Prepared prepareStatementWithDelayedValueTuple(String function)
-    {
-        createTable("CREATE TABLE %s (" +
-                    " key int PRIMARY KEY," +
-                    " val tuple<double> )");
-
-        ResultMessage.Prepared prepared = QueryProcessor.prepare(
-                                                                String.format("INSERT INTO %s.%s (key, val) VALUES (?, (%s(0.0)))",
-                                                                             KEYSPACE,
-                                                                             currentTable(),
-                                                                             function),
-                                                                ClientState.forInternalCalls(), false);
-        Assert.assertNotNull(QueryProcessor.instance.getPrepared(prepared.statementId));
-        return prepared;
-    }
-
-    public void checkDelayedValuesCorrectlyIdentifyFunctionsInUse(boolean dropKeyspace) throws Throwable
-    {
-        // prepare a statement which doesn't use any function for a control
-        createTable("CREATE TABLE %s (" +
-                    " key int PRIMARY KEY," +
-                    " val double)");
-        ResultMessage.Prepared control = QueryProcessor.prepare(
-                                                               String.format("INSERT INTO %s.%s (key, val) VALUES (?, ?)",
-                                                                            KEYSPACE,
-                                                                            currentTable()),
-                                                               ClientState.forInternalCalls(), false);
-        Assert.assertNotNull(QueryProcessor.instance.getPrepared(control.statementId));
-
-        // a function that we'll drop and verify that statements which use it to
-        // provide a DelayedValue are removed from the cache in QueryProcessor
-        String function = createFunction(KEYSPACE_PER_TEST, "double",
-                                        "CREATE FUNCTION %s ( input double ) " +
-                                        "CALLED ON NULL INPUT " +
-                                        "RETURNS double " +
-                                        "LANGUAGE javascript " +
-                                        "AS 'input'");
-        Assert.assertEquals(1, Functions.find(parseFunctionName(function)).size());
-
-        List<ResultMessage.Prepared> prepared = new ArrayList<>();
-        // prepare statements which use the function to provide a DelayedValue
-        prepared.add(prepareStatementWithDelayedValue(CollectionType.Kind.LIST, function));
-        prepared.add(prepareStatementWithDelayedValue(CollectionType.Kind.SET, function));
-        prepared.add(prepareStatementWithDelayedValue(CollectionType.Kind.MAP, function));
-        prepared.add(prepareStatementWithDelayedValueTuple(function));
-
-        // what to drop - the function is scoped to the per-test keyspace, but the prepared statements
-        // select from the per-fixture keyspace. So if we drop the per-test keyspace, the function
-        // should be removed along with the statements that reference it. The control statement should
-        // remain present in the cache. Likewise, if we actually drop the function itself the control
-        // statement should not be removed, but the others should be
-        if (dropKeyspace)
-            dropPerTestKeyspace();
-        else
-            execute("DROP FUNCTION " + function);
-
-        Assert.assertNotNull(QueryProcessor.instance.getPrepared(control.statementId));
-        for (ResultMessage.Prepared removed : prepared)
-            Assert.assertNull(QueryProcessor.instance.getPrepared(removed.statementId));
-    }
-
-    @Test
-    public void testFunctionCreationAndDrop() throws Throwable
-    {
-        createTable("CREATE TABLE %s (key int PRIMARY KEY, d double)");
-
-        execute("INSERT INTO %s(key, d) VALUES (?, ?)", 1, 1d);
-        execute("INSERT INTO %s(key, d) VALUES (?, ?)", 2, 2d);
-        execute("INSERT INTO %s(key, d) VALUES (?, ?)", 3, 3d);
-
-        // simple creation
-        String fSin = createFunction(KEYSPACE_PER_TEST, "double",
-                                     "CREATE FUNCTION %s ( input double ) " +
-                                     "CALLED ON NULL INPUT " +
-                                     "RETURNS double " +
-                                     "LANGUAGE java " +
-                                     "AS 'return Math.sin(input);'");
-        // check we can't recreate the same function
-        assertInvalidMessage("already exists",
-                             "CREATE FUNCTION " + fSin + " ( input double ) " +
-                             "CALLED ON NULL INPUT " +
-                             "RETURNS double " +
-                             "LANGUAGE java AS 'return Double.valueOf(Math.sin(input.doubleValue()));'");
-
-        // but that it doesn't comply with "IF NOT EXISTS"
-        execute("CREATE FUNCTION IF NOT EXISTS " + fSin + " ( input double ) " +
-                "CALLED ON NULL INPUT " +
-                "RETURNS double " +
-                "LANGUAGE java AS 'return Double.valueOf(Math.sin(input.doubleValue()));'");
-
-        // Validate that it works as expected
-        assertRows(execute("SELECT key, " + fSin + "(d) FROM %s"),
-            row(1, Math.sin(1d)),
-            row(2, Math.sin(2d)),
-            row(3, Math.sin(3d))
-        );
-
-        // Replace the method with incompatible return type
-        assertInvalidMessage("the new return type text is not compatible with the return type double of existing function",
-                             "CREATE OR REPLACE FUNCTION " + fSin + " ( input double ) " +
-                             "CALLED ON NULL INPUT " +
-                             "RETURNS text " +
-                             "LANGUAGE java AS 'return Double.valueOf(42d);'");
-
-        // proper replacement
-        execute("CREATE OR REPLACE FUNCTION " + fSin + " ( input double ) " +
-                "CALLED ON NULL INPUT " +
-                "RETURNS double " +
-                "LANGUAGE java AS 'return Double.valueOf(42d);'");
-
-        // Validate the method as been replaced
-        assertRows(execute("SELECT key, " + fSin + "(d) FROM %s"),
-            row(1, 42.0),
-            row(2, 42.0),
-            row(3, 42.0)
-        );
-
-        // same function but other keyspace
-        String fSin2 = createFunction(KEYSPACE, "double",
-                                      "CREATE FUNCTION %s ( input double ) " +
-                                      "RETURNS NULL ON NULL INPUT " +
-                                      "RETURNS double " +
-                                      "LANGUAGE java " +
-                                      "AS 'return Math.sin(input);'");
-        assertRows(execute("SELECT key, " + fSin2 + "(d) FROM %s"),
-            row(1, Math.sin(1d)),
-            row(2, Math.sin(2d)),
-            row(3, Math.sin(3d))
-        );
-
-        // Drop
-        execute("DROP FUNCTION " + fSin);
-        execute("DROP FUNCTION " + fSin2);
-
-        // Drop unexisting function
-        assertInvalidMessage("Cannot drop non existing function", "DROP FUNCTION " + fSin);
-        // but don't complain with "IF EXISTS"
-        execute("DROP FUNCTION IF EXISTS " + fSin);
-
-        // can't drop native functions
-        assertInvalidMessage("system keyspace is not user-modifiable", "DROP FUNCTION totimestamp");
-        assertInvalidMessage("system keyspace is not user-modifiable", "DROP FUNCTION uuid");
-
-        // sin() no longer exists
-        assertInvalidMessage("Unknown function", "SELECT key, sin(d) FROM %s");
-    }
-
-    @Test
-    public void testFunctionExecution() throws Throwable
-    {
-        createTable("CREATE TABLE %s (v text PRIMARY KEY)");
-
-        execute("INSERT INTO %s(v) VALUES (?)", "aaa");
-
-        String fRepeat = createFunction(KEYSPACE_PER_TEST, "text,int",
-                                        "CREATE FUNCTION %s(v text, n int) " +
-                                        "RETURNS NULL ON NULL INPUT " +
-                                        "RETURNS text " +
-                                        "LANGUAGE java " +
-                                        "AS 'StringBuilder sb = new StringBuilder();\n" +
-                                        "    for (int i = 0; i < n; i++)\n" +
-                                        "        sb.append(v);\n" +
-                                        "    return sb.toString();'");
-
-        assertRows(execute("SELECT v FROM %s WHERE v=" + fRepeat + "(?, ?)", "a", 3), row("aaa"));
-        assertEmpty(execute("SELECT v FROM %s WHERE v=" + fRepeat + "(?, ?)", "a", 2));
-    }
-
-    @Test
-    public void testFunctionOverloading() throws Throwable
-    {
-        createTable("CREATE TABLE %s (k text PRIMARY KEY, v int)");
-
-        execute("INSERT INTO %s(k, v) VALUES (?, ?)", "f2", 1);
-
-        String fOverload = createFunction(KEYSPACE_PER_TEST, "varchar",
-                                          "CREATE FUNCTION %s ( input varchar ) " +
-                                          "RETURNS NULL ON NULL INPUT " +
-                                          "RETURNS text " +
-                                          "LANGUAGE java " +
-                                          "AS 'return \"f1\";'");
-        createFunctionOverload(fOverload,
-                               "int",
-                               "CREATE OR REPLACE FUNCTION %s(i int) " +
-                               "RETURNS NULL ON NULL INPUT " +
-                               "RETURNS text " +
-                               "LANGUAGE java " +
-                               "AS 'return \"f2\";'");
-        createFunctionOverload(fOverload,
-                               "text,text",
-                               "CREATE OR REPLACE FUNCTION %s(v1 text, v2 text) " +
-                               "RETURNS NULL ON NULL INPUT " +
-                               "RETURNS text " +
-                               "LANGUAGE java " +
-                               "AS 'return \"f3\";'");
-        createFunctionOverload(fOverload,
-                               "ascii",
-                               "CREATE OR REPLACE FUNCTION %s(v ascii) " +
-                               "RETURNS NULL ON NULL INPUT " +
-                               "RETURNS text " +
-                               "LANGUAGE java " +
-                               "AS 'return \"f1\";'");
-
-        // text == varchar, so this should be considered as a duplicate
-        assertInvalidMessage("already exists",
-                             "CREATE FUNCTION " + fOverload + "(v varchar) " +
-                             "RETURNS NULL ON NULL INPUT " +
-                             "RETURNS text " +
-                             "LANGUAGE java AS 'return \"f1\";'");
-
-        assertRows(execute("SELECT " + fOverload + "(k), " + fOverload + "(v), " + fOverload + "(k, k) FROM %s"),
-            row("f1", "f2", "f3")
-        );
-
-        forcePreparedValues();
-        // This shouldn't work if we use preparation since there no way to know which overload to use
-        assertInvalidMessage("Ambiguous call to function", "SELECT v FROM %s WHERE k = " + fOverload + "(?)", "foo");
-        stopForcingPreparedValues();
-
-        // but those should since we specifically cast
-        assertEmpty(execute("SELECT v FROM %s WHERE k = " + fOverload + "((text)?)", "foo"));
-        assertRows(execute("SELECT v FROM %s WHERE k = " + fOverload + "((int)?)", 3), row(1));
-        assertEmpty(execute("SELECT v FROM %s WHERE k = " + fOverload + "((ascii)?)", "foo"));
-        // And since varchar == text, this should work too
-        assertEmpty(execute("SELECT v FROM %s WHERE k = " + fOverload + "((varchar)?)", "foo"));
-
-        // no such functions exist...
-        assertInvalidMessage("non existing function", "DROP FUNCTION " + fOverload + "(boolean)");
-        assertInvalidMessage("non existing function", "DROP FUNCTION " + fOverload + "(bigint)");
-
-        // 'overloaded' has multiple overloads - so it has to fail (CASSANDRA-7812)
-        assertInvalidMessage("matches multiple function definitions", "DROP FUNCTION " + fOverload);
-        execute("DROP FUNCTION " + fOverload + "(varchar)");
-        assertInvalidMessage("none of its type signatures match", "SELECT v FROM %s WHERE k = " + fOverload + "((text)?)", "foo");
-        execute("DROP FUNCTION " + fOverload + "(text, text)");
-        assertInvalidMessage("none of its type signatures match", "SELECT v FROM %s WHERE k = " + fOverload + "((text)?,(text)?)", "foo", "bar");
-        execute("DROP FUNCTION " + fOverload + "(ascii)");
-        assertInvalidMessage("cannot be passed as argument 0 of function", "SELECT v FROM %s WHERE k = " + fOverload + "((ascii)?)", "foo");
-        // single-int-overload must still work
-        assertRows(execute("SELECT v FROM %s WHERE k = " + fOverload + "((int)?)", 3), row(1));
-        // overloaded has just one overload now - so the following DROP FUNCTION is not ambigious (CASSANDRA-7812)
-        execute("DROP FUNCTION " + fOverload);
-    }
-
-    @Test
-    public void testCreateOrReplaceJavaFunction() throws Throwable
-    {
-        createTable("CREATE TABLE %s (key int primary key, val double)");
-        execute("INSERT INTO %s (key, val) VALUES (?, ?)", 1, 1d);
-        execute("INSERT INTO %s (key, val) VALUES (?, ?)", 2, 2d);
-        execute("INSERT INTO %s (key, val) VALUES (?, ?)", 3, 3d);
-
-        String fName = createFunction(KEYSPACE_PER_TEST, "double",
-                "CREATE FUNCTION %s( input double ) " +
-                "CALLED ON NULL INPUT " +
-                "RETURNS double " +
-                "LANGUAGE java " +
-                "AS '\n" +
-                "  // parameter val is of type java.lang.Double\n" +
-                "  /* return type is of type java.lang.Double */\n" +
-                "  if (input == null) {\n" +
-                "    return null;\n" +
-                "  }\n" +
-                "  return Math.sin( input );\n" +
-                "';");
-
-        // just check created function
-        assertRows(execute("SELECT key, val, " + fName + "(val) FROM %s"),
-                   row(1, 1d, Math.sin(1d)),
-                   row(2, 2d, Math.sin(2d)),
-                   row(3, 3d, Math.sin(3d))
-        );
-
-        execute("CREATE OR REPLACE FUNCTION " + fName + "( input double ) " +
-                "CALLED ON NULL INPUT " +
-                "RETURNS double " +
-                "LANGUAGE java\n" +
-                "AS '\n" +
-                "  return input;\n" +
-                "';");
-
-        // check if replaced function returns correct result
-        assertRows(execute("SELECT key, val, " + fName + "(val) FROM %s"),
-                   row(1, 1d, 1d),
-                   row(2, 2d, 2d),
-                   row(3, 3d, 3d)
-        );
-    }
-
-    @Test
-    public void testJavaFunctionNoParameters() throws Throwable
-    {
-        createTable("CREATE TABLE %s (key int primary key, val double)");
-
-        String functionBody = "\n  return 1L;\n";
-
-        String fName = createFunction(KEYSPACE, "",
-                                      "CREATE OR REPLACE FUNCTION %s() " +
-                                      "RETURNS NULL ON NULL INPUT " +
-                                      "RETURNS bigint " +
-                                      "LANGUAGE JAVA\n" +
-                                      "AS '" +functionBody + "';");
-
-        assertRows(execute("SELECT language, body FROM system.schema_functions WHERE keyspace_name=? AND function_name=?",
-                           KEYSPACE, parseFunctionName(fName).name),
-                   row("java", functionBody));
-
-        execute("INSERT INTO %s (key, val) VALUES (?, ?)", 1, 1d);
-        execute("INSERT INTO %s (key, val) VALUES (?, ?)", 2, 2d);
-        execute("INSERT INTO %s (key, val) VALUES (?, ?)", 3, 3d);
-        assertRows(execute("SELECT key, val, " + fName + "() FROM %s"),
-                   row(1, 1d, 1L),
-                   row(2, 2d, 1L),
-                   row(3, 3d, 1L)
-        );
-    }
-
-    @Test
-    public void testJavaFunctionInvalidBodies() throws Throwable
-    {
-        try
-        {
-            execute("CREATE OR REPLACE FUNCTION " + KEYSPACE + ".jfinv() " +
-                    "RETURNS NULL ON NULL INPUT " +
-                    "RETURNS bigint " +
-                    "LANGUAGE JAVA\n" +
-                    "AS '\n" +
-                    "foobarbaz" +
-                    "\n';");
-            Assert.fail();
-        }
-        catch (InvalidRequestException e)
-        {
-            Assert.assertTrue(e.getMessage(), e.getMessage().contains("Java source compilation failed"));
-            Assert.assertTrue(e.getMessage(), e.getMessage().contains("insert \";\" to complete BlockStatements"));
-        }
-
-        try
-        {
-            execute("CREATE OR REPLACE FUNCTION " + KEYSPACE + ".jfinv() " +
-                    "RETURNS NULL ON NULL INPUT " +
-                    "RETURNS bigint " +
-                    "LANGUAGE JAVA\n" +
-                    "AS '\n" +
-                    "foobarbaz;" +
-                    "\n';");
-            Assert.fail();
-        }
-        catch (InvalidRequestException e)
-        {
-            Assert.assertTrue(e.getMessage(), e.getMessage().contains("Java source compilation failed"));
-            Assert.assertTrue(e.getMessage(), e.getMessage().contains("foobarbaz cannot be resolved to a type"));
-        }
-    }
-
-    @Test
-    public void testJavaFunctionInvalidReturn() throws Throwable
-    {
-        assertInvalidMessage("system keyspace is not user-modifiable",
-                             "CREATE OR REPLACE FUNCTION jfir(val double) " +
-                             "RETURNS NULL ON NULL INPUT " +
-                             "RETURNS double " +
-                             "LANGUAGE JAVA\n" +
-                             "AS 'return 1L;';");
-    }
-
-    @Test
-    public void testJavaFunctionArgumentTypeMismatch() throws Throwable
-    {
-        createTable("CREATE TABLE %s (key int primary key, val bigint)");
-
-        String fName = createFunction(KEYSPACE, "double",
-                                      "CREATE OR REPLACE FUNCTION %s(val double)" +
-                                      "RETURNS NULL ON NULL INPUT " +
-                                      "RETURNS double " +
-                                      "LANGUAGE JAVA " +
-                                      "AS 'return Double.valueOf(val);';");
-
-        execute("INSERT INTO %s (key, val) VALUES (?, ?)", 1, 1L);
-        execute("INSERT INTO %s (key, val) VALUES (?, ?)", 2, 2L);
-        execute("INSERT INTO %s (key, val) VALUES (?, ?)", 3, 3L);
-        assertInvalidMessage("val cannot be passed as argument 0 of function",
-                             "SELECT key, val, " + fName + "(val) FROM %s");
-    }
-
-    @Test
-    public void testJavaFunction() throws Throwable
-    {
-        createTable("CREATE TABLE %s (key int primary key, val double)");
-
-        String functionBody = '\n' +
-                              "  // parameter val is of type java.lang.Double\n" +
-                              "  /* return type is of type java.lang.Double */\n" +
-                              "  if (val == null) {\n" +
-                              "    return null;\n" +
-                              "  }\n" +
-                              "  return Math.sin(val);\n";
-
-        String fName = createFunction(KEYSPACE, "double",
-                                      "CREATE OR REPLACE FUNCTION %s(val double) " +
-                                      "CALLED ON NULL INPUT " +
-                                      "RETURNS double " +
-                                      "LANGUAGE JAVA " +
-                                      "AS '" + functionBody + "';");
-
-        FunctionName fNameName = parseFunctionName(fName);
-
-        assertRows(execute("SELECT language, body FROM system.schema_functions WHERE keyspace_name=? AND function_name=?",
-                           fNameName.keyspace, fNameName.name),
-                   row("java", functionBody));
-
-        execute("INSERT INTO %s (key, val) VALUES (?, ?)", 1, 1d);
-        execute("INSERT INTO %s (key, val) VALUES (?, ?)", 2, 2d);
-        execute("INSERT INTO %s (key, val) VALUES (?, ?)", 3, 3d);
-        assertRows(execute("SELECT key, val, " + fName + "(val) FROM %s"),
-                   row(1, 1d, Math.sin(1d)),
-                   row(2, 2d, Math.sin(2d)),
-                   row(3, 3d, Math.sin(3d))
-        );
-    }
-
-    @Test
-    public void testFunctionInTargetKeyspace() throws Throwable
-    {
-        createTable("CREATE TABLE %s (key int primary key, val double)");
-
-        execute("CREATE TABLE " + KEYSPACE_PER_TEST + ".second_tab (key int primary key, val double)");
-
-        String fName = createFunction(KEYSPACE_PER_TEST, "double",
-                                      "CREATE OR REPLACE FUNCTION %s(val double) " +
-                                      "RETURNS NULL ON NULL INPUT " +
-                                      "RETURNS double " +
-                                      "LANGUAGE JAVA " +
-                                      "AS 'return Double.valueOf(val);';");
-
-        execute("INSERT INTO %s (key, val) VALUES (?, ?)", 1, 1d);
-        execute("INSERT INTO %s (key, val) VALUES (?, ?)", 2, 2d);
-        execute("INSERT INTO %s (key, val) VALUES (?, ?)", 3, 3d);
-        assertInvalidMessage("Unknown function",
-                             "SELECT key, val, " + parseFunctionName(fName).name + "(val) FROM %s");
-
-        execute("INSERT INTO " + KEYSPACE_PER_TEST + ".second_tab (key, val) VALUES (?, ?)", 1, 1d);
-        execute("INSERT INTO " + KEYSPACE_PER_TEST + ".second_tab (key, val) VALUES (?, ?)", 2, 2d);
-        execute("INSERT INTO " + KEYSPACE_PER_TEST + ".second_tab (key, val) VALUES (?, ?)", 3, 3d);
-        assertRows(execute("SELECT key, val, " + fName + "(val) FROM " + KEYSPACE_PER_TEST + ".second_tab"),
-                   row(1, 1d, 1d),
-                   row(2, 2d, 2d),
-                   row(3, 3d, 3d)
-        );
-    }
-
-    @Test
-    public void testFunctionWithReservedName() throws Throwable
-    {
-        execute("CREATE TABLE " + KEYSPACE_PER_TEST + ".second_tab (key int primary key, val double)");
-
-        String fName = createFunction(KEYSPACE_PER_TEST, "",
-                                      "CREATE OR REPLACE FUNCTION %s() " +
-                                      "RETURNS NULL ON NULL INPUT " +
-                                      "RETURNS timestamp " +
-                                      "LANGUAGE JAVA " +
-                                      "AS 'return null;';");
-
-        execute("INSERT INTO " + KEYSPACE_PER_TEST + ".second_tab (key, val) VALUES (?, ?)", 1, 1d);
-        execute("INSERT INTO " + KEYSPACE_PER_TEST + ".second_tab (key, val) VALUES (?, ?)", 2, 2d);
-        execute("INSERT INTO " + KEYSPACE_PER_TEST + ".second_tab (key, val) VALUES (?, ?)", 3, 3d);
-
-        // ensure that system now() is executed
-        UntypedResultSet rows = execute("SELECT key, val, now() FROM " + KEYSPACE_PER_TEST + ".second_tab");
-        Assert.assertEquals(3, rows.size());
-        UntypedResultSet.Row row = rows.iterator().next();
-        Date ts = row.getTimestamp(row.getColumns().get(2).name.toString());
-        Assert.assertNotNull(ts);
-
-        // ensure that KEYSPACE_PER_TEST's now() is executed
-        rows = execute("SELECT key, val, " + fName + "() FROM " + KEYSPACE_PER_TEST + ".second_tab");
-        Assert.assertEquals(3, rows.size());
-        row = rows.iterator().next();
-        Assert.assertFalse(row.has(row.getColumns().get(2).name.toString()));
-    }
-
-    @Test
-    public void testFunctionInSystemKS() throws Throwable
-    {
-        execute("CREATE OR REPLACE FUNCTION " + KEYSPACE + ".totimestamp(val timeuuid) " +
-                "RETURNS NULL ON NULL INPUT " +
-                "RETURNS timestamp " +
-                "LANGUAGE JAVA\n" +
-
-                "AS 'return null;';");
-
-        assertInvalidMessage("system keyspace is not user-modifiable",
-                             "CREATE OR REPLACE FUNCTION system.jnft(val double) " +
-                             "RETURNS NULL ON NULL INPUT " +
-                             "RETURNS double " +
-                             "LANGUAGE JAVA\n" +
-                             "AS 'return null;';");
-        assertInvalidMessage("system keyspace is not user-modifiable",
-                             "CREATE OR REPLACE FUNCTION system.totimestamp(val timeuuid) " +
-                             "RETURNS NULL ON NULL INPUT " +
-                             "RETURNS timestamp " +
-                             "LANGUAGE JAVA\n" +
-
-                             "AS 'return null;';");
-        assertInvalidMessage("system keyspace is not user-modifiable",
-                             "DROP FUNCTION system.now");
-
-        // KS for executeInternal() is system
-        assertInvalidMessage("system keyspace is not user-modifiable",
-                             "CREATE OR REPLACE FUNCTION jnft(val double) " +
-                             "RETURNS NULL ON NULL INPUT " +
-                             "RETURNS double " +
-                             "LANGUAGE JAVA\n" +
-                             "AS 'return null;';");
-        assertInvalidMessage("system keyspace is not user-modifiable",
-                             "CREATE OR REPLACE FUNCTION totimestamp(val timeuuid) " +
-                             "RETURNS NULL ON NULL INPUT " +
-                             "RETURNS timestamp " +
-                             "LANGUAGE JAVA\n" +
-                             "AS 'return null;';");
-        assertInvalidMessage("system keyspace is not user-modifiable",
-                             "DROP FUNCTION now");
-    }
-
-    @Test
-    public void testFunctionNonExistingKeyspace() throws Throwable
-    {
-        assertInvalidMessage("to non existing keyspace",
-                             "CREATE OR REPLACE FUNCTION this_ks_does_not_exist.jnft(val double) " +
-                             "RETURNS NULL ON NULL INPUT " +
-                             "RETURNS double " +
-                             "LANGUAGE JAVA\n" +
-                             "AS 'return null;';");
-    }
-
-    @Test
-    public void testFunctionAfterOnDropKeyspace() throws Throwable
-    {
-        dropPerTestKeyspace();
-
-        assertInvalidMessage("to non existing keyspace",
-                             "CREATE OR REPLACE FUNCTION " + KEYSPACE_PER_TEST + ".jnft(val double) " +
-                             "RETURNS NULL ON NULL INPUT " +
-                             "RETURNS double " +
-                             "LANGUAGE JAVA\n" +
-                             "AS 'return null;';");
-    }
-
-    @Test
-    public void testJavaKeyspaceFunction() throws Throwable
-    {
-        createTable("CREATE TABLE %s (key int primary key, val double)");
-
-        String functionBody = '\n' +
-                              "  // parameter val is of type java.lang.Double\n" +
-                              "  /* return type is of type java.lang.Double */\n" +
-                              "  if (val == null) {\n" +
-                              "    return null;\n" +
-                              "  }\n" +
-                              "  return Math.sin( val );\n";
-
-        String fName = createFunction(KEYSPACE_PER_TEST, "double",
-                                     "CREATE OR REPLACE FUNCTION %s(val double) " +
-                                     "CALLED ON NULL INPUT " +
-                                     "RETURNS double " +
-                                     "LANGUAGE JAVA " +
-                                     "AS '" + functionBody + "';");
-
-        FunctionName fNameName = parseFunctionName(fName);
-
-        assertRows(execute("SELECT language, body FROM system.schema_functions WHERE keyspace_name=? AND function_name=?",
-                           fNameName.keyspace, fNameName.name),
-                   row("java", functionBody));
-
-        execute("INSERT INTO %s (key, val) VALUES (?, ?)", 1, 1d);
-        execute("INSERT INTO %s (key, val) VALUES (?, ?)", 2, 2d);
-        execute("INSERT INTO %s (key, val) VALUES (?, ?)", 3, 3d);
-        assertRows(execute("SELECT key, val, " + fName + "(val) FROM %s"),
-                   row(1, 1d, Math.sin(1d)),
-                   row(2, 2d, Math.sin(2d)),
-                   row(3, 3d, Math.sin(3d))
-        );
-    }
-
-    @Test
-    public void testJavaRuntimeException() throws Throwable
-    {
-        createTable("CREATE TABLE %s (key int primary key, val double)");
-
-        String functionBody = '\n' +
-                              "  throw new RuntimeException(\"oh no!\");\n";
-
-        String fName = createFunction(KEYSPACE_PER_TEST, "double",
-                                      "CREATE OR REPLACE FUNCTION %s(val double) " +
-                                      "RETURNS NULL ON NULL INPUT " +
-                                      "RETURNS double " +
-                                      "LANGUAGE JAVA\n" +
-                                      "AS '" + functionBody + "';");
-
-        FunctionName fNameName = parseFunctionName(fName);
-
-        assertRows(execute("SELECT language, body FROM system.schema_functions WHERE keyspace_name=? AND function_name=?",
-                           fNameName.keyspace, fNameName.name),
-                   row("java", functionBody));
-
-        execute("INSERT INTO %s (key, val) VALUES (?, ?)", 1, 1d);
-        execute("INSERT INTO %s (key, val) VALUES (?, ?)", 2, 2d);
-        execute("INSERT INTO %s (key, val) VALUES (?, ?)", 3, 3d);
-
-        // function throws a RuntimeException which is wrapped by FunctionExecutionException
-        assertInvalidThrowMessage("java.lang.RuntimeException: oh no", FunctionExecutionException.class,
-                                  "SELECT key, val, " + fName + "(val) FROM %s");
-    }
-
-    @Test
-    public void testJavaDollarQuotedFunction() throws Throwable
-    {
-        String functionBody = '\n' +
-                              "  // parameter val is of type java.lang.Double\n" +
-                              "  /* return type is of type java.lang.Double */\n" +
-                              "  if (input == null) {\n" +
-                              "    return null;\n" +
-                              "  }\n" +
-                              "  return \"'\"+Math.sin(input)+'\\\'';\n";
-
-        String fName = createFunction(KEYSPACE_PER_TEST, "double",
-                                      "CREATE FUNCTION %s( input double ) " +
-                                      "CALLED ON NULL INPUT " +
-                                      "RETURNS text " +
-                                      "LANGUAGE java\n" +
-                                      "AS $$" + functionBody + "$$;");
-
-        FunctionName fNameName = parseFunctionName(fName);
-
-        assertRows(execute("SELECT language, body FROM system.schema_functions WHERE keyspace_name=? AND function_name=?",
-                           fNameName.keyspace, fNameName.name),
-                   row("java", functionBody));
-    }
-
-    @Test
-    public void testJavaSimpleCollections() throws Throwable
-    {
-        createTable("CREATE TABLE %s (key int primary key, lst list<double>, st set<text>, mp map<int, boolean>)");
-
-        String fList = createFunction(KEYSPACE_PER_TEST, "list<double>",
-                                     "CREATE FUNCTION %s( lst list<double> ) " +
-                                     "RETURNS NULL ON NULL INPUT " +
-                                     "RETURNS list<double> " +
-                                     "LANGUAGE java\n" +
-                                     "AS $$return lst;$$;");
-        String fSet = createFunction(KEYSPACE_PER_TEST, "set<text>",
-                                     "CREATE FUNCTION %s( st set<text> ) " +
-                                     "RETURNS NULL ON NULL INPUT " +
-                                     "RETURNS set<text> " +
-                                     "LANGUAGE java\n" +
-                                     "AS $$return st;$$;");
-        String fMap = createFunction(KEYSPACE_PER_TEST, "map<int, boolean>",
-                                     "CREATE FUNCTION %s( mp map<int, boolean> ) " +
-                                     "RETURNS NULL ON NULL INPUT " +
-                                     "RETURNS map<int, boolean> " +
-                                     "LANGUAGE java\n" +
-                                     "AS $$return mp;$$;");
-
-        List<Double> list = Arrays.asList(1d, 2d, 3d);
-        Set<String> set = new TreeSet<>(Arrays.asList("one", "three", "two"));
-        Map<Integer, Boolean> map = new TreeMap<>();
-        map.put(1, true);
-        map.put(2, false);
-        map.put(3, true);
-
-        execute("INSERT INTO %s (key, lst, st, mp) VALUES (1, ?, ?, ?)", list, set, map);
-
-        assertRows(execute("SELECT " + fList + "(lst), " + fSet + "(st), " + fMap + "(mp) FROM %s WHERE key = 1"),
-                   row(list, set, map));
-
-        // same test - but via native protocol
-        for (int version = Server.VERSION_2; version <= maxProtocolVersion; version++)
-            assertRowsNet(version,
-                          executeNet(version, "SELECT " + fList + "(lst), " + fSet + "(st), " + fMap + "(mp) FROM %s WHERE key = 1"),
-                          row(list, set, map));
-    }
-
-    @Test
-    public void testWrongKeyspace() throws Throwable
-    {
-        String typeName = createType("CREATE TYPE %s (txt text, i int)");
-        String type = KEYSPACE + '.' + typeName;
-
-        assertInvalidMessage(String.format("Statement on keyspace %s cannot refer to a user type in keyspace %s; user types can only be used in the keyspace they are defined in",
-                                           KEYSPACE_PER_TEST, KEYSPACE),
-                             "CREATE FUNCTION " + KEYSPACE_PER_TEST + ".test_wrong_ks( val int ) " +
-                             "CALLED ON NULL INPUT " +
-                             "RETURNS " + type + " " +
-                             "LANGUAGE java\n" +
-                             "AS $$return val;$$;");
-
-        assertInvalidMessage(String.format("Statement on keyspace %s cannot refer to a user type in keyspace %s; user types can only be used in the keyspace they are defined in",
-                                           KEYSPACE_PER_TEST, KEYSPACE),
-                             "CREATE FUNCTION " + KEYSPACE_PER_TEST + ".test_wrong_ks( val " + type + " ) " +
-                             "CALLED ON NULL INPUT " +
-                             "RETURNS int " +
-                             "LANGUAGE java\n" +
-                             "AS $$return val;$$;");
-    }
-
-    @Test
-    public void testComplexNullValues() throws Throwable
-    {
-        String type = KEYSPACE + '.' + createType("CREATE TYPE %s (txt text, i int)");
-
-        createTable("CREATE TABLE %s (key int primary key, lst list<double>, st set<text>, mp map<int, boolean>," +
-                    "tup frozen<tuple<double, text, int, boolean>>, udt frozen<" + type + ">)");
-
-        String fList = createFunction(KEYSPACE, "list<double>",
-                                      "CREATE FUNCTION %s( coll list<double> ) " +
-                                      "CALLED ON NULL INPUT " +
-                                      "RETURNS list<double> " +
-                                      "LANGUAGE java\n" +
-                                      "AS $$return coll;$$;");
-        String fSet = createFunction(KEYSPACE, "set<text>",
-                                     "CREATE FUNCTION %s( coll set<text> ) " +
-                                     "CALLED ON NULL INPUT " +
-                                     "RETURNS set<text> " +
-                                     "LANGUAGE java\n" +
-                                     "AS $$return coll;$$;");
-        String fMap = createFunction(KEYSPACE, "map<int, boolean>",
-                                     "CREATE FUNCTION %s( coll map<int, boolean> ) " +
-                                     "CALLED ON NULL INPUT " +
-                                     "RETURNS map<int, boolean> " +
-                                     "LANGUAGE java\n" +
-                                     "AS $$return coll;$$;");
-        String fTup = createFunction(KEYSPACE, "tuple<double, text, int, boolean>",
-                                     "CREATE FUNCTION %s( val tuple<double, text, int, boolean> ) " +
-                                     "CALLED ON NULL INPUT " +
-                                     "RETURNS tuple<double, text, int, boolean> " +
-                                     "LANGUAGE java\n" +
-                                     "AS $$return val;$$;");
-        String fUdt = createFunction(KEYSPACE, type,
-                                     "CREATE FUNCTION %s( val " + type + " ) " +
-                                     "CALLED ON NULL INPUT " +
-                                     "RETURNS " + type + " " +
-                                     "LANGUAGE java\n" +
-                                     "AS $$return val;$$;");
-        List<Double> list = Arrays.asList(1d, 2d, 3d);
-        Set<String> set = new TreeSet<>(Arrays.asList("one", "three", "two"));
-        Map<Integer, Boolean> map = new TreeMap<>();
-        map.put(1, true);
-        map.put(2, false);
-        map.put(3, true);
-        Object t = tuple(1d, "one", 42, false);
-
-        execute("INSERT INTO %s (key, lst, st, mp, tup, udt) VALUES (1, ?, ?, ?, ?, {txt: 'one', i:1})", list, set, map, t);
-        execute("INSERT INTO %s (key, lst, st, mp, tup, udt) VALUES (2, ?, ?, ?, ?, null)", null, null, null, null);
-
-        execute("SELECT " +
-                fList + "(lst), " +
-                fSet + "(st), " +
-                fMap + "(mp), " +
-                fTup + "(tup), " +
-                fUdt + "(udt) FROM %s WHERE key = 1");
-        UntypedResultSet.Row row = execute("SELECT " +
-                                           fList + "(lst) as l, " +
-                                           fSet + "(st) as s, " +
-                                           fMap + "(mp) as m, " +
-                                           fTup + "(tup) as t, " +
-                                           fUdt + "(udt) as u " +
-                                           "FROM %s WHERE key = 1").one();
-        Assert.assertNotNull(row.getBytes("l"));
-        Assert.assertNotNull(row.getBytes("s"));
-        Assert.assertNotNull(row.getBytes("m"));
-        Assert.assertNotNull(row.getBytes("t"));
-        Assert.assertNotNull(row.getBytes("u"));
-        row = execute("SELECT " +
-                      fList + "(lst) as l, " +
-                      fSet + "(st) as s, " +
-                      fMap + "(mp) as m, " +
-                      fTup + "(tup) as t, " +
-                      fUdt + "(udt) as u " +
-                      "FROM %s WHERE key = 2").one();
-        Assert.assertNull(row.getBytes("l"));
-        Assert.assertNull(row.getBytes("s"));
-        Assert.assertNull(row.getBytes("m"));
-        Assert.assertNull(row.getBytes("t"));
-        Assert.assertNull(row.getBytes("u"));
-
-        for (int version = Server.VERSION_2; version <= maxProtocolVersion; version++)
-        {
-            Row r = executeNet(version, "SELECT " +
-                                        fList + "(lst) as l, " +
-                                        fSet + "(st) as s, " +
-                                        fMap + "(mp) as m, " +
-                                        fTup + "(tup) as t, " +
-                                        fUdt + "(udt) as u " +
-                                        "FROM %s WHERE key = 1").one();
-            Assert.assertNotNull(r.getBytesUnsafe("l"));
-            Assert.assertNotNull(r.getBytesUnsafe("s"));
-            Assert.assertNotNull(r.getBytesUnsafe("m"));
-            Assert.assertNotNull(r.getBytesUnsafe("t"));
-            Assert.assertNotNull(r.getBytesUnsafe("u"));
-            r = executeNet(version, "SELECT " +
-                                    fList + "(lst) as l, " +
-                                    fSet + "(st) as s, " +
-                                    fMap + "(mp) as m, " +
-                                    fTup + "(tup) as t, " +
-                                    fUdt + "(udt) as u " +
-                                    "FROM %s WHERE key = 2").one();
-            Assert.assertNull(r.getBytesUnsafe("l"));
-            Assert.assertNull(r.getBytesUnsafe("s"));
-            Assert.assertNull(r.getBytesUnsafe("m"));
-            Assert.assertNull(r.getBytesUnsafe("t"));
-            Assert.assertNull(r.getBytesUnsafe("u"));
-        }
-    }
-
-    @Test
-    public void testJavaTupleType() throws Throwable
-    {
-        createTable("CREATE TABLE %s (key int primary key, tup frozen<tuple<double, text, int, boolean>>)");
-
-        String fName = createFunction(KEYSPACE, "tuple<double, text, int, boolean>",
-                                     "CREATE FUNCTION %s( tup tuple<double, text, int, boolean> ) " +
-                                     "RETURNS NULL ON NULL INPUT " +
-                                     "RETURNS tuple<double, text, int, boolean> " +
-                                     "LANGUAGE java\n" +
-                                     "AS $$return tup;$$;");
-
-        Object t = tuple(1d, "foo", 2, true);
-
-        execute("INSERT INTO %s (key, tup) VALUES (1, ?)", t);
-
-        assertRows(execute("SELECT tup FROM %s WHERE key = 1"),
-                   row(t));
-
-        assertRows(execute("SELECT " + fName + "(tup) FROM %s WHERE key = 1"),
-                   row(t));
-    }
-
-    @Test
-    public void testJavaTupleTypeCollection() throws Throwable
-    {
-        String tupleTypeDef = "tuple<double, list<double>, set<text>, map<int, boolean>>";
-
-        createTable("CREATE TABLE %s (key int primary key, tup frozen<" + tupleTypeDef + ">)");
-
-        String fTup0 = createFunction(KEYSPACE_PER_TEST, tupleTypeDef,
-                "CREATE FUNCTION %s( tup " + tupleTypeDef + " ) " +
-                "CALLED ON NULL INPUT " +
-                "RETURNS " + tupleTypeDef + ' ' +
-                "LANGUAGE java\n" +
-                "AS $$return " +
-                "       tup;$$;");
-        String fTup1 = createFunction(KEYSPACE_PER_TEST, tupleTypeDef,
-                "CREATE FUNCTION %s( tup " + tupleTypeDef + " ) " +
-                "CALLED ON NULL INPUT " +
-                "RETURNS double " +
-                "LANGUAGE java\n" +
-                "AS $$return " +
-                "       Double.valueOf(tup.getDouble(0));$$;");
-        String fTup2 = createFunction(KEYSPACE_PER_TEST, tupleTypeDef,
-                                      "CREATE FUNCTION %s( tup " + tupleTypeDef + " ) " +
-                                      "RETURNS NULL ON NULL INPUT " +
-                                      "RETURNS list<double> " +
-                                      "LANGUAGE java\n" +
-                                      "AS $$return " +
-                                      "       tup.getList(1, Double.class);$$;");
-        String fTup3 = createFunction(KEYSPACE_PER_TEST, tupleTypeDef,
-                "CREATE FUNCTION %s( tup " + tupleTypeDef + " ) " +
-                "RETURNS NULL ON NULL INPUT " +
-                "RETURNS set<text> " +
-                "LANGUAGE java\n" +
-                "AS $$return " +
-                "       tup.getSet(2, String.class);$$;");
-        String fTup4 = createFunction(KEYSPACE_PER_TEST, tupleTypeDef,
-                "CREATE FUNCTION %s( tup " + tupleTypeDef + " ) " +
-                "RETURNS NULL ON NULL INPUT " +
-                "RETURNS map<int, boolean> " +
-                "LANGUAGE java\n" +
-                "AS $$return " +
-                "       tup.getMap(3, Integer.class, Boolean.class);$$;");
-
-        List<Double> list = Arrays.asList(1d, 2d, 3d);
-        Set<String> set = new TreeSet<>(Arrays.asList("one", "three", "two"));
-        Map<Integer, Boolean> map = new TreeMap<>();
-        map.put(1, true);
-        map.put(2, false);
-        map.put(3, true);
-
-        Object t = tuple(1d, list, set, map);
-
-        execute("INSERT INTO %s (key, tup) VALUES (1, ?)", t);
-
-        assertRows(execute("SELECT " + fTup0 + "(tup) FROM %s WHERE key = 1"),
-                   row(t));
-        assertRows(execute("SELECT " + fTup1 + "(tup) FROM %s WHERE key = 1"),
-                   row(1d));
-        assertRows(execute("SELECT " + fTup2 + "(tup) FROM %s WHERE key = 1"),
-                   row(list));
-        assertRows(execute("SELECT " + fTup3 + "(tup) FROM %s WHERE key = 1"),
-                   row(set));
-        assertRows(execute("SELECT " + fTup4 + "(tup) FROM %s WHERE key = 1"),
-                   row(map));
-
-        TupleType tType = TupleType.of(DataType.cdouble(),
-                                       DataType.list(DataType.cdouble()),
-                                       DataType.set(DataType.text()),
-                                       DataType.map(DataType.cint(), DataType.cboolean()));
-        TupleValue tup = tType.newValue(1d, list, set, map);
-        for (int version = Server.VERSION_2; version <= maxProtocolVersion; version++)
-        {
-            assertRowsNet(version,
-                          executeNet(version, "SELECT " + fTup0 + "(tup) FROM %s WHERE key = 1"),
-                          row(tup));
-            assertRowsNet(version,
-                          executeNet(version, "SELECT " + fTup1 + "(tup) FROM %s WHERE key = 1"),
-                          row(1d));
-            assertRowsNet(version,
-                          executeNet(version, "SELECT " + fTup2 + "(tup) FROM %s WHERE key = 1"),
-                          row(list));
-            assertRowsNet(version,
-                          executeNet(version, "SELECT " + fTup3 + "(tup) FROM %s WHERE key = 1"),
-                          row(set));
-            assertRowsNet(version,
-                          executeNet(version, "SELECT " + fTup4 + "(tup) FROM %s WHERE key = 1"),
-                          row(map));
-        }
-    }
-
-    @Test
-    public void testJavaUserTypeWithUse() throws Throwable
-    {
-        String type = createType("CREATE TYPE %s (txt text, i int)");
-        createTable("CREATE TABLE %s (key int primary key, udt frozen<" + KEYSPACE + '.' + type + ">)");
-        execute("INSERT INTO %s (key, udt) VALUES (1, {txt: 'one', i:1})");
-
-        for (int version = Server.VERSION_2; version <= maxProtocolVersion; version++)
-        {
-            executeNet(version, "USE " + KEYSPACE);
-
-            executeNet(version,
-                       "CREATE FUNCTION f_use1( udt " + type + " ) " +
-                       "RETURNS NULL ON NULL INPUT " +
-                       "RETURNS " + type + " " +
-                       "LANGUAGE java " +
-                       "AS $$return " +
-                       "     udt;$$;");
-            try
-            {
-                List<Row> rowsNet = executeNet(version, "SELECT f_use1(udt) FROM %s WHERE key = 1").all();
-                Assert.assertEquals(1, rowsNet.size());
-                UDTValue udtVal = rowsNet.get(0).getUDTValue(0);
-                Assert.assertEquals("one", udtVal.getString("txt"));
-                Assert.assertEquals(1, udtVal.getInt("i"));
-            }
-            finally
-            {
-                executeNet(version, "DROP FUNCTION f_use1");
-            }
-        }
-    }
-
-    @Test
-    public void testJavaUserType() throws Throwable
-    {
-        String type = KEYSPACE + '.' + createType("CREATE TYPE %s (txt text, i int)");
-
-        createTable("CREATE TABLE %s (key int primary key, udt frozen<" + type + ">)");
-
-        String fUdt0 = createFunction(KEYSPACE, type,
-                                      "CREATE FUNCTION %s( udt " + type + " ) " +
-                                      "RETURNS NULL ON NULL INPUT " +
-                                      "RETURNS " + type + " " +
-                                      "LANGUAGE java " +
-                                      "AS $$return " +
-                                      "     udt;$$;");
-        String fUdt1 = createFunction(KEYSPACE, type,
-                                      "CREATE FUNCTION %s( udt " + type + ") " +
-                                      "RETURNS NULL ON NULL INPUT " +
-                                      "RETURNS text " +
-                                      "LANGUAGE java " +
-                                      "AS $$return " +
-                                      "     udt.getString(\"txt\");$$;");
-        String fUdt2 = createFunction(KEYSPACE, type,
-                                      "CREATE FUNCTION %s( udt " + type + ") " +
-                                      "CALLED ON NULL INPUT " +
-                                      "RETURNS int " +
-                                      "LANGUAGE java " +
-                                      "AS $$return " +
-                                      "     Integer.valueOf(udt.getInt(\"i\"));$$;");
-
-        execute("INSERT INTO %s (key, udt) VALUES (1, {txt: 'one', i:1})");
-
-        UntypedResultSet rows = execute("SELECT " + fUdt0 + "(udt) FROM %s WHERE key = 1");
-        Assert.assertEquals(1, rows.size());
-        assertRows(execute("SELECT " + fUdt1 + "(udt) FROM %s WHERE key = 1"),
-                   row("one"));
-        assertRows(execute("SELECT " + fUdt2 + "(udt) FROM %s WHERE key = 1"),
-                   row(1));
-
-        for (int version = Server.VERSION_2; version <= maxProtocolVersion; version++)
-        {
-            List<Row> rowsNet = executeNet(version, "SELECT " + fUdt0 + "(udt) FROM %s WHERE key = 1").all();
-            Assert.assertEquals(1, rowsNet.size());
-            UDTValue udtVal = rowsNet.get(0).getUDTValue(0);
-            Assert.assertEquals("one", udtVal.getString("txt"));
-            Assert.assertEquals(1, udtVal.getInt("i"));
-            assertRowsNet(version,
-                          executeNet(version, "SELECT " + fUdt1 + "(udt) FROM %s WHERE key = 1"),
-                          row("one"));
-            assertRowsNet(version,
-                          executeNet(version, "SELECT " + fUdt2 + "(udt) FROM %s WHERE key = 1"),
-                          row(1));
-        }
-    }
-
-    @Test
-    public void testUserTypeDrop() throws Throwable
-    {
-        String type = KEYSPACE + '.' + createType("CREATE TYPE %s (txt text, i int)");
-
-        createTable("CREATE TABLE %s (key int primary key, udt frozen<" + type + ">)");
-
-        String fName = createFunction(KEYSPACE, type,
-                                      "CREATE FUNCTION %s( udt " + type + " ) " +
-                                      "CALLED ON NULL INPUT " +
-                                      "RETURNS int " +
-                                      "LANGUAGE java " +
-                                      "AS $$return " +
-                                      "     Integer.valueOf(udt.getInt(\"i\"));$$;");
-
-        FunctionName fNameName = parseFunctionName(fName);
-
-        Assert.assertEquals(1, Functions.find(fNameName).size());
-
-        ResultMessage.Prepared prepared = QueryProcessor.prepare(String.format("SELECT key, %s(udt) FROM %s.%s", fName, KEYSPACE, currentTable()),
-                                                                 ClientState.forInternalCalls(), false);
-        Assert.assertNotNull(QueryProcessor.instance.getPrepared(prepared.statementId));
-
-        // UT still referenced by table
-        assertInvalidMessage("Cannot drop user type", "DROP TYPE " + type);
-
-        execute("DROP TABLE %s");
-
-        // UT still referenced by UDF
-        assertInvalidMessage("as it is still used by function", "DROP TYPE " + type);
-
-        Assert.assertNull(QueryProcessor.instance.getPrepared(prepared.statementId));
-
-        // function stays
-        Assert.assertEquals(1, Functions.find(fNameName).size());
-    }
-
-    @Test
-    public void testJavaUserTypeRenameField() throws Throwable
-    {
-        String type = KEYSPACE + '.' + createType("CREATE TYPE %s (txt text, i int)");
-
-        createTable("CREATE TABLE %s (key int primary key, udt frozen<" + type + ">)");
-
-        String fName = createFunction(KEYSPACE, type,
-                                      "CREATE FUNCTION %s( udt " + type + " ) " +
-                                      "RETURNS NULL ON NULL INPUT " +
-                                      "RETURNS text " +
-                                      "LANGUAGE java\n" +
-                                      "AS $$return udt.getString(\"txt\");$$;");
-
-        execute("INSERT INTO %s (key, udt) VALUES (1, {txt: 'one', i:1})");
-
-        assertRows(execute("SELECT " + fName + "(udt) FROM %s WHERE key = 1"),
-                   row("one"));
-
-        execute("ALTER TYPE " + type + " RENAME txt TO str");
-
-        assertInvalidMessage("txt is not a field defined in this UDT",
-                             "SELECT " + fName + "(udt) FROM %s WHERE key = 1");
-
-        execute("ALTER TYPE " + type + " RENAME str TO txt");
-
-        assertRows(execute("SELECT " + fName + "(udt) FROM %s WHERE key = 1"),
-                   row("one"));
-    }
-
-    @Test
-    public void testJavaUserTypeAddFieldWithReplace() throws Throwable
-    {
-        String type = KEYSPACE + '.' + createType("CREATE TYPE %s (txt text, i int)");
-
-        createTable("CREATE TABLE %s (key int primary key, udt frozen<" + type + ">)");
-
-        String fName1replace = createFunction(KEYSPACE, type,
-                                              "CREATE FUNCTION %s( udt " + type + ") " +
-                                              "RETURNS NULL ON NULL INPUT " +
-                                              "RETURNS text " +
-                                              "LANGUAGE java\n" +
-                                              "AS $$return udt.getString(\"txt\");$$;");
-        String fName2replace = createFunction(KEYSPACE, type,
-                                              "CREATE FUNCTION %s( udt " + type + " ) " +
-                                              "CALLED ON NULL INPUT " +
-                                              "RETURNS int " +
-                                              "LANGUAGE java\n" +
-                                              "AS $$return Integer.valueOf(udt.getInt(\"i\"));$$;");
-        String fName3replace = createFunction(KEYSPACE, type,
-                                              "CREATE FUNCTION %s( udt " + type + " ) " +
-                                              "CALLED ON NULL INPUT " +
-                                              "RETURNS double " +
-                                              "LANGUAGE java\n" +
-                                              "AS $$return Double.valueOf(udt.getDouble(\"added\"));$$;");
-        String fName4replace = createFunction(KEYSPACE, type,
-                                              "CREATE FUNCTION %s( udt " + type + " ) " +
-                                              "RETURNS NULL ON NULL INPUT " +
-                                              "RETURNS " + type + " " +
-                                              "LANGUAGE java\n" +
-                                              "AS $$return udt;$$;");
-
-        String fName1noReplace = createFunction(KEYSPACE, type,
-                                              "CREATE FUNCTION %s( udt " + type + " ) " +
-                                              "RETURNS NULL ON NULL INPUT " +
-                                              "RETURNS text " +
-                                              "LANGUAGE java\n" +
-                                              "AS $$return udt.getString(\"txt\");$$;");
-        String fName2noReplace = createFunction(KEYSPACE, type,
-                                              "CREATE FUNCTION %s( udt " + type + " ) " +
-                                              "CALLED ON NULL INPUT " +
-                                              "RETURNS int " +
-                                              "LANGUAGE java\n" +
-                                              "AS $$return Integer.valueOf(udt.getInt(\"i\"));$$;");
-        String fName3noReplace = createFunction(KEYSPACE, type,
-                                                "CREATE FUNCTION %s( udt " + type + " ) " +
-                                                "CALLED ON NULL INPUT " +
-                                                "RETURNS double " +
-                                                "LANGUAGE java\n" +
-                                                "AS $$return Double.valueOf(udt.getDouble(\"added\"));$$;");
-        String fName4noReplace = createFunction(KEYSPACE, type,
-                                                "CREATE FUNCTION %s( udt " + type + " ) " +
-                                                "RETURNS NULL ON NULL INPUT " +
-                                                "RETURNS " + type + " " +
-                                                "LANGUAGE java\n" +
-                                                "AS $$return udt;$$;");
-
-        execute("INSERT INTO %s (key, udt) VALUES (1, {txt: 'one', i:1})");
-
-        assertRows(execute("SELECT " + fName1replace + "(udt) FROM %s WHERE key = 1"),
-                   row("one"));
-        assertRows(execute("SELECT " + fName2replace + "(udt) FROM %s WHERE key = 1"),
-                   row(1));
-
-        // add field
-
-        execute("ALTER TYPE " + type + " ADD added double");
-
-        execute("INSERT INTO %s (key, udt) VALUES (2, {txt: 'two', i:2, added: 2})");
-
-        // note: type references of functions remain at the state _before_ the type mutation
-        // means we need to recreate the functions
-
-        execute(String.format("CREATE OR REPLACE FUNCTION %s( udt %s ) " +
-                              "RETURNS NULL ON NULL INPUT " +
-                              "RETURNS text " +
-                              "LANGUAGE java\n" +
-                              "AS $$return " +
-                              "     udt.getString(\"txt\");$$;",
-                              fName1replace, type));
-        Assert.assertEquals(1, Functions.find(parseFunctionName(fName1replace)).size());
-        execute(String.format("CREATE OR REPLACE FUNCTION %s( udt %s ) " +
-                              "CALLED ON NULL INPUT " +
-                              "RETURNS int " +
-                              "LANGUAGE java\n" +
-                              "AS $$return " +
-                              "     Integer.valueOf(udt.getInt(\"i\"));$$;",
-                              fName2replace, type));
-        Assert.assertEquals(1, Functions.find(parseFunctionName(fName2replace)).size());
-        execute(String.format("CREATE OR REPLACE FUNCTION %s( udt %s ) " +
-                              "CALLED ON NULL INPUT " +
-                              "RETURNS double " +
-                              "LANGUAGE java\n" +
-                              "AS $$return " +
-                              "     Double.valueOf(udt.getDouble(\"added\"));$$;",
-                              fName3replace, type));
-        Assert.assertEquals(1, Functions.find(parseFunctionName(fName3replace)).size());
-        execute(String.format("CREATE OR REPLACE FUNCTION %s( udt %s ) " +
-                              "RETURNS NULL ON NULL INPUT " +
-                              "RETURNS %s " +
-                              "LANGUAGE java\n" +
-                              "AS $$return " +
-                              "     udt;$$;",
-                              fName4replace, type, type));
-        Assert.assertEquals(1, Functions.find(parseFunctionName(fName4replace)).size());
-
-        assertRows(execute("SELECT " + fName1replace + "(udt) FROM %s WHERE key = 2"),
-                   row("two"));
-        assertRows(execute("SELECT " + fName2replace + "(udt) FROM %s WHERE key = 2"),
-                   row(2));
-        assertRows(execute("SELECT " + fName3replace + "(udt) FROM %s WHERE key = 2"),
-                   row(2d));
-        assertRows(execute("SELECT " + fName3replace + "(udt) FROM %s WHERE key = 1"),
-                   row(0d));
-
-        // un-replaced functions will work since the user type has changed
-        // and the UDF has exchanged the user type reference
-
-        assertRows(execute("SELECT " + fName1noReplace + "(udt) FROM %s WHERE key = 2"),
-                   row("two"));
-        assertRows(execute("SELECT " + fName2noReplace + "(udt) FROM %s WHERE key = 2"),
-                   row(2));
-        assertRows(execute("SELECT " + fName3noReplace + "(udt) FROM %s WHERE key = 2"),
-                   row(2d));
-        assertRows(execute("SELECT " + fName3noReplace + "(udt) FROM %s WHERE key = 1"),
-                   row(0d));
-
-        execute("DROP FUNCTION " + fName1replace);
-        execute("DROP FUNCTION " + fName2replace);
-        execute("DROP FUNCTION " + fName3replace);
-        execute("DROP FUNCTION " + fName4replace);
-        execute("DROP FUNCTION " + fName1noReplace);
-        execute("DROP FUNCTION " + fName2noReplace);
-        execute("DROP FUNCTION " + fName3noReplace);
-        execute("DROP FUNCTION " + fName4noReplace);
-    }
-
-    @Test
-    public void testJavaUTCollections() throws Throwable
-    {
-        String type = KEYSPACE + '.' + createType("CREATE TYPE %s (txt text, i int)");
-
-        createTable(String.format("CREATE TABLE %%s " +
-                                  "(key int primary key, lst list<frozen<%s>>, st set<frozen<%s>>, mp map<int, frozen<%s>>)",
-                                  type, type, type));
-
-        String fName1 = createFunction(KEYSPACE, "list<frozen<" + type + ">>",
-                              "CREATE FUNCTION %s( lst list<frozen<" + type + ">> ) " +
-                              "RETURNS NULL ON NULL INPUT " +
-                              "RETURNS text " +
-                              "LANGUAGE java\n" +
-                              "AS $$" +
-                              "     com.datastax.driver.core.UDTValue udtVal = (com.datastax.driver.core.UDTValue)lst.get(1);" +
-                              "     return udtVal.getString(\"txt\");$$;");
-        String fName2 = createFunction(KEYSPACE, "set<frozen<" + type + ">>",
-                                       "CREATE FUNCTION %s( st set<frozen<" + type + ">> ) " +
-                                       "RETURNS NULL ON NULL INPUT " +
-                                       "RETURNS text " +
-                                       "LANGUAGE java\n" +
-                                       "AS $$" +
-                                       "     com.datastax.driver.core.UDTValue udtVal = (com.datastax.driver.core.UDTValue)st.iterator().next();" +
-                                       "     return udtVal.getString(\"txt\");$$;");
-        String fName3 = createFunction(KEYSPACE, "map<int, frozen<" + type + ">>",
-                              "CREATE FUNCTION %s( mp map<int, frozen<" + type + ">> ) " +
-                              "RETURNS NULL ON NULL INPUT " +
-                              "RETURNS text " +
-                              "LANGUAGE java\n" +
-                              "AS $$" +
-                              "     com.datastax.driver.core.UDTValue udtVal = (com.datastax.driver.core.UDTValue)mp.get(Integer.valueOf(3));" +
-                              "     return udtVal.getString(\"txt\");$$;");
-
-        execute("INSERT INTO %s (key, lst, st, mp) values (1, " +
-                "[ {txt: 'one', i:1}, {txt: 'three', i:1}, {txt: 'one', i:1} ] , " +
-                "{ {txt: 'one', i:1}, {txt: 'three', i:3}, {txt: 'two', i:2} }, " +
-                "{ 1: {txt: 'one', i:1}, 2: {txt: 'one', i:3}, 3: {txt: 'two', i:2} })");
-
-        assertRows(execute("SELECT " + fName1 + "(lst), " + fName2 + "(st), " + fName3 + "(mp) FROM %s WHERE key = 1"),
-                   row("three", "one", "two"));
-
-        for (int version = Server.VERSION_2; version <= maxProtocolVersion; version++)
-            assertRowsNet(version,
-                          executeNet(version, "SELECT " + fName1 + "(lst), " + fName2 + "(st), " + fName3 + "(mp) FROM %s WHERE key = 1"),
-                          row("three", "one", "two"));
-    }
-
-    @Test
-    public void testJavascriptSimpleCollections() throws Throwable
-    {
-        createTable("CREATE TABLE %s (key int primary key, lst list<double>, st set<text>, mp map<int, boolean>)");
-
-        String fName1 = createFunction(KEYSPACE_PER_TEST, "list<double>",
-                "CREATE FUNCTION %s( lst list<double> ) " +
-                "RETURNS NULL ON NULL INPUT " +
-                "RETURNS list<double> " +
-                "LANGUAGE javascript\n" +
-                "AS 'lst;';");
-        String fName2 = createFunction(KEYSPACE_PER_TEST, "set<text>",
-                "CREATE FUNCTION %s( st set<text> ) " +
-                "RETURNS NULL ON NULL INPUT " +
-                "RETURNS set<text> " +
-                "LANGUAGE javascript\n" +
-                "AS 'st;';");
-        String fName3 = createFunction(KEYSPACE_PER_TEST, "map<int, boolean>",
-                "CREATE FUNCTION %s( mp map<int, boolean> ) " +
-                "RETURNS NULL ON NULL INPUT " +
-                "RETURNS map<int, boolean> " +
-                "LANGUAGE javascript\n" +
-                "AS 'mp;';");
-
-        List<Double> list = Arrays.asList(1d, 2d, 3d);
-        Set<String> set = new TreeSet<>(Arrays.asList("one", "three", "two"));
-        Map<Integer, Boolean> map = new TreeMap<>();
-        map.put(1, true);
-        map.put(2, false);
-        map.put(3, true);
-
-        execute("INSERT INTO %s (key, lst, st, mp) VALUES (1, ?, ?, ?)", list, set, map);
-
-        assertRows(execute("SELECT lst, st, mp FROM %s WHERE key = 1"),
-                   row(list, set, map));
-
-        assertRows(execute("SELECT " + fName1 + "(lst), " + fName2 + "(st), " + fName3 + "(mp) FROM %s WHERE key = 1"),
-                   row(list, set, map));
-
-        for (int version = Server.VERSION_2; version <= maxProtocolVersion; version++)
-            assertRowsNet(version,
-                          executeNet(version, "SELECT " + fName1 + "(lst), " + fName2 + "(st), " + fName3 + "(mp) FROM %s WHERE key = 1"),
-                          row(list, set, map));
-    }
-
-    @Test
-    public void testJavascriptTupleType() throws Throwable
-    {
-        createTable("CREATE TABLE %s (key int primary key, tup frozen<tuple<double, text, int, boolean>>)");
-
-        String fName = createFunction(KEYSPACE_PER_TEST, "tuple<double, text, int, boolean>",
-                "CREATE FUNCTION %s( tup tuple<double, text, int, boolean> ) " +
-                "RETURNS NULL ON NULL INPUT " +
-                "RETURNS tuple<double, text, int, boolean> " +
-                "LANGUAGE javascript\n" +
-                "AS $$tup;$$;");
-
-        Object t = tuple(1d, "foo", 2, true);
-
-        execute("INSERT INTO %s (key, tup) VALUES (1, ?)", t);
-
-        assertRows(execute("SELECT tup FROM %s WHERE key = 1"),
-                   row(t));
-
-        assertRows(execute("SELECT " + fName + "(tup) FROM %s WHERE key = 1"),
-                   row(t));
-    }
-
-    @Test
-    public void testJavascriptTupleTypeCollection() throws Throwable
-    {
-        String tupleTypeDef = "tuple<double, list<double>, set<text>, map<int, boolean>>";
-        createTable("CREATE TABLE %s (key int primary key, tup frozen<" + tupleTypeDef + ">)");
-
-        String fTup1 = createFunction(KEYSPACE_PER_TEST, tupleTypeDef,
-                "CREATE FUNCTION %s( tup " + tupleTypeDef + " ) " +
-                "RETURNS NULL ON NULL INPUT " +
-                "RETURNS tuple<double, list<double>, set<text>, map<int, boolean>> " +
-                "LANGUAGE javascript\n" +
-                "AS $$" +
-                "       tup;$$;");
-        String fTup2 = createFunction(KEYSPACE_PER_TEST, tupleTypeDef,
-                "CREATE FUNCTION %s( tup " + tupleTypeDef + " ) " +
-                "RETURNS NULL ON NULL INPUT " +
-                "RETURNS double " +
-                "LANGUAGE javascript\n" +
-                "AS $$" +
-                "       tup.getDouble(0);$$;");
-        String fTup3 = createFunction(KEYSPACE_PER_TEST, tupleTypeDef,
-                "CREATE FUNCTION %s( tup " + tupleTypeDef + " ) " +
-                "RETURNS NULL ON NULL INPUT " +
-                "RETURNS list<double> " +
-                "LANGUAGE javascript\n" +
-                "AS $$" +
-                "       tup.getList(1, java.lang.Class.forName(\"java.lang.Double\"));$$;");
-        String fTup4 = createFunction(KEYSPACE_PER_TEST, tupleTypeDef,
-                "CREATE FUNCTION %s( tup " + tupleTypeDef + " ) " +
-                "RETURNS NULL ON NULL INPUT " +
-                "RETURNS set<text> " +
-                "LANGUAGE javascript\n" +
-                "AS $$" +
-                "       tup.getSet(2, java.lang.Class.forName(\"java.lang.String\"));$$;");
-        String fTup5 = createFunction(KEYSPACE_PER_TEST, tupleTypeDef,
-                "CREATE FUNCTION %s( tup " + tupleTypeDef + " ) " +
-                "RETURNS NULL ON NULL INPUT " +
-                "RETURNS map<int, boolean> " +
-                "LANGUAGE javascript\n" +
-                "AS $$" +
-                "       tup.getMap(3, java.lang.Class.forName(\"java.lang.Integer\"), java.lang.Class.forName(\"java.lang.Boolean\"));$$;");
-
-        List<Double> list = Arrays.asList(1d, 2d, 3d);
-        Set<String> set = new TreeSet<>(Arrays.asList("one", "three", "two"));
-        Map<Integer, Boolean> map = new TreeMap<>();
-        map.put(1, true);
-        map.put(2, false);
-        map.put(3, true);
-
-        Object t = tuple(1d, list, set, map);
-
-        execute("INSERT INTO %s (key, tup) VALUES (1, ?)", t);
-
-        assertRows(execute("SELECT " + fTup1 + "(tup) FROM %s WHERE key = 1"),
-                   row(t));
-        assertRows(execute("SELECT " + fTup2 + "(tup) FROM %s WHERE key = 1"),
-                   row(1d));
-        assertRows(execute("SELECT " + fTup3 + "(tup) FROM %s WHERE key = 1"),
-                   row(list));
-        assertRows(execute("SELECT " + fTup4 + "(tup) FROM %s WHERE key = 1"),
-                   row(set));
-        assertRows(execute("SELECT " + fTup5 + "(tup) FROM %s WHERE key = 1"),
-                   row(map));
-
-        // same test - but via native protocol
-        TupleType tType = TupleType.of(DataType.cdouble(),
-                                       DataType.list(DataType.cdouble()),
-                                       DataType.set(DataType.text()),
-                                       DataType.map(DataType.cint(), DataType.cboolean()));
-        TupleValue tup = tType.newValue(1d, list, set, map);
-        for (int version = Server.VERSION_2; version <= maxProtocolVersion; version++)
-        {
-            assertRowsNet(version,
-                          executeNet(version, "SELECT " + fTup1 + "(tup) FROM %s WHERE key = 1"),
-                          row(tup));
-            assertRowsNet(version,
-                          executeNet(version, "SELECT " + fTup2 + "(tup) FROM %s WHERE key = 1"),
-                          row(1d));
-            assertRowsNet(version,
-                          executeNet(version, "SELECT " + fTup3 + "(tup) FROM %s WHERE key = 1"),
-                          row(list));
-            assertRowsNet(version,
-                          executeNet(version, "SELECT " + fTup4 + "(tup) FROM %s WHERE key = 1"),
-                          row(set));
-            assertRowsNet(version,
-                          executeNet(version, "SELECT " + fTup5 + "(tup) FROM %s WHERE key = 1"),
-                          row(map));
-        }
-    }
-
-    @Test
-    public void testJavascriptUserType() throws Throwable
-    {
-        String type = createType("CREATE TYPE %s (txt text, i int)");
-
-        createTable("CREATE TABLE %s (key int primary key, udt frozen<" + type + ">)");
-
-        String fUdt1 = createFunction(KEYSPACE, type,
-                              "CREATE FUNCTION %s( udt " + type + " ) " +
-                              "RETURNS NULL ON NULL INPUT " +
-                              "RETURNS " + type + " " +
-                              "LANGUAGE javascript\n" +
-                              "AS $$" +
-                              "     udt;$$;");
-        String fUdt2 = createFunction(KEYSPACE, type,
-                              "CREATE FUNCTION %s( udt " + type + " ) " +
-                              "RETURNS NULL ON NULL INPUT " +
-                              "RETURNS text " +
-                              "LANGUAGE javascript\n" +
-                              "AS $$" +
-                              "     udt.getString(\"txt\");$$;");
-        String fUdt3 = createFunction(KEYSPACE, type,
-                                      "CREATE FUNCTION %s( udt " + type + " ) " +
-                                      "RETURNS NULL ON NULL INPUT " +
-                                      "RETURNS int " +
-                                      "LANGUAGE javascript\n" +
-                                      "AS $$" +
-                                      "     udt.getInt(\"i\");$$;");
-
-        execute("INSERT INTO %s (key, udt) VALUES (1, {txt: 'one', i:1})");
-
-        UntypedResultSet rows = execute("SELECT " + fUdt1 + "(udt) FROM %s WHERE key = 1");
-        Assert.assertEquals(1, rows.size());
-        assertRows(execute("SELECT " + fUdt2 + "(udt) FROM %s WHERE key = 1"),
-                   row("one"));
-        assertRows(execute("SELECT " + fUdt3 + "(udt) FROM %s WHERE key = 1"),
-                   row(1));
-    }
-
-    @Test
-    public void testJavascriptUTCollections() throws Throwable
-    {
-        String type = createType("CREATE TYPE %s (txt text, i int)");
-
-        createTable(String.format("CREATE TABLE %%s " +
-                                  "(key int primary key, lst list<frozen<%s>>, st set<frozen<%s>>, mp map<int, frozen<%s>>)",
-                                  type, type, type));
-
-        String fName = createFunction(KEYSPACE, "list<frozen<" + type + ">>",
-                                      "CREATE FUNCTION %s( lst list<frozen<" + type + ">> ) " +
-                                      "RETURNS NULL ON NULL INPUT " +
-                                      "RETURNS text " +
-                                      "LANGUAGE javascript\n" +
-                                      "AS $$" +
-                                      "        lst.get(1).getString(\"txt\");$$;");
-        createFunctionOverload(fName, "set<frozen<" + type + ">>",
-                               "CREATE FUNCTION %s( st set<frozen<" + type + ">> ) " +
-                               "RETURNS NULL ON NULL INPUT " +
-                               "RETURNS text " +
-                               "LANGUAGE javascript\n" +
-                               "AS $$" +
-                               "        st.iterator().next().getString(\"txt\");$$;");
-        createFunctionOverload(fName, "map<int, frozen<" + type + ">>",
-                               "CREATE FUNCTION %s( mp map<int, frozen<" + type + ">> ) " +
-                               "RETURNS NULL ON NULL INPUT " +
-                               "RETURNS text " +
-                 

<TRUNCATED>

[06/32] cassandra git commit: Migrate CQL tests from dtest to unit tests

Posted by jm...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/f797bfa4/test/unit/org/apache/cassandra/cql3/ThriftCompatibilityTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/ThriftCompatibilityTest.java b/test/unit/org/apache/cassandra/cql3/ThriftCompatibilityTest.java
index 662800b..6f0e439 100644
--- a/test/unit/org/apache/cassandra/cql3/ThriftCompatibilityTest.java
+++ b/test/unit/org/apache/cassandra/cql3/ThriftCompatibilityTest.java
@@ -23,6 +23,7 @@ import org.junit.Test;
 import org.apache.cassandra.SchemaLoader;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
 
 public class ThriftCompatibilityTest extends SchemaLoader
 {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f797bfa4/test/unit/org/apache/cassandra/cql3/TimestampTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/TimestampTest.java b/test/unit/org/apache/cassandra/cql3/TimestampTest.java
deleted file mode 100644
index 6673904..0000000
--- a/test/unit/org/apache/cassandra/cql3/TimestampTest.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/*
- * 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;
-
-import org.junit.Test;
-
-public class TimestampTest extends CQLTester
-{
-    @Test
-    public void testNegativeTimestamps() throws Throwable
-    {
-        createTable("CREATE TABLE %s (k int PRIMARY KEY, v int)");
-
-        execute("INSERT INTO %s (k, v) VALUES (?, ?) USING TIMESTAMP ?", 1, 1, -42L);
-        assertRows(execute("SELECT writetime(v) FROM %s WHERE k = ?", 1),
-            row(-42L)
-        );
-
-        assertInvalid("INSERT INTO %s (k, v) VALUES (?, ?) USING TIMESTAMP ?", 2, 2, Long.MIN_VALUE);
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f797bfa4/test/unit/org/apache/cassandra/cql3/TupleTypeTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/TupleTypeTest.java b/test/unit/org/apache/cassandra/cql3/TupleTypeTest.java
deleted file mode 100644
index ce935e3..0000000
--- a/test/unit/org/apache/cassandra/cql3/TupleTypeTest.java
+++ /dev/null
@@ -1,101 +0,0 @@
-/*
- * 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;
-
-import org.junit.Test;
-
-public class TupleTypeTest extends CQLTester
-{
-    @Test
-    public void testTuplePutAndGet() throws Throwable
-    {
-        String[] valueTypes = {"frozen<tuple<int, text, double>>", "tuple<int, text, double>"};
-        for (String valueType : valueTypes)
-        {
-            createTable("CREATE TABLE %s (k int PRIMARY KEY, t " + valueType + ")");
-
-            execute("INSERT INTO %s (k, t) VALUES (?, ?)", 0, tuple(3, "foo", 3.4));
-            execute("INSERT INTO %s (k, t) VALUES (?, ?)", 1, tuple(8, "bar", 0.2));
-            assertAllRows(
-                row(0, tuple(3, "foo", 3.4)),
-                row(1, tuple(8, "bar", 0.2))
-            );
-
-            // nulls
-            execute("INSERT INTO %s (k, t) VALUES (?, ?)", 2, tuple(5, null, 3.4));
-            assertRows(execute("SELECT * FROM %s WHERE k=?", 2),
-                row(2, tuple(5, null, 3.4))
-            );
-
-            // incomplete tuple
-            execute("INSERT INTO %s (k, t) VALUES (?, ?)", 3, tuple(5, "bar"));
-            assertRows(execute("SELECT * FROM %s WHERE k=?", 3),
-                row(3, tuple(5, "bar"))
-            );
-        }
-    }
-
-    @Test
-    public void testNestedTuple() throws Throwable
-    {
-        createTable("CREATE TABLE %s (k int PRIMARY KEY, t frozen<tuple<int, tuple<text, double>>>)");
-
-        execute("INSERT INTO %s (k, t) VALUES (?, ?)", 0, tuple(3, tuple("foo", 3.4)));
-        execute("INSERT INTO %s (k, t) VALUES (?, ?)", 1, tuple(8, tuple("bar", 0.2)));
-        assertAllRows(
-            row(0, tuple(3, tuple("foo", 3.4))),
-            row(1, tuple(8, tuple("bar", 0.2)))
-        );
-    }
-
-    @Test
-    public void testTupleInPartitionKey() throws Throwable
-    {
-        createTable("CREATE TABLE %s (t frozen<tuple<int, text>> PRIMARY KEY)");
-
-        execute("INSERT INTO %s (t) VALUES (?)", tuple(3, "foo"));
-        assertAllRows(row(tuple(3, "foo")));
-    }
-
-    @Test
-    public void testTupleInClusteringKey() throws Throwable
-    {
-        createTable("CREATE TABLE %s (k int, t frozen<tuple<int, text>>, PRIMARY KEY (k, t))");
-
-        execute("INSERT INTO %s (k, t) VALUES (?, ?)", 0, tuple(5, "bar"));
-        execute("INSERT INTO %s (k, t) VALUES (?, ?)", 0, tuple(3, "foo"));
-        execute("INSERT INTO %s (k, t) VALUES (?, ?)", 0, tuple(6, "bar"));
-        execute("INSERT INTO %s (k, t) VALUES (?, ?)", 0, tuple(5, "foo"));
-
-        assertAllRows(
-            row(0, tuple(3, "foo")),
-            row(0, tuple(5, "bar")),
-            row(0, tuple(5, "foo")),
-            row(0, tuple(6, "bar"))
-        );
-    }
-
-    @Test
-    public void testInvalidQueries() throws Throwable
-    {
-        createTable("CREATE TABLE %s (k int PRIMARY KEY, t frozen<tuple<int, text, double>>)");
-
-        assertInvalidSyntax("INSERT INTO %s (k, t) VALUES (0, ())");
-        assertInvalid("INSERT INTO %s (k, t) VALUES (0, (2, 'foo', 3.1, 'bar'))");
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f797bfa4/test/unit/org/apache/cassandra/cql3/TypeTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/TypeTest.java b/test/unit/org/apache/cassandra/cql3/TypeTest.java
deleted file mode 100644
index ec82d41..0000000
--- a/test/unit/org/apache/cassandra/cql3/TypeTest.java
+++ /dev/null
@@ -1,86 +0,0 @@
-/*
- * 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;
-
-import org.apache.cassandra.exceptions.ConfigurationException;
-import org.junit.Test;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-public class TypeTest extends CQLTester
-{
-    @Test
-    public void testNowToUUIDCompatibility() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a int, b uuid, PRIMARY KEY (a, b))");
-        execute("INSERT INTO %s (a, b) VALUES (0, now())");
-        UntypedResultSet results = execute("SELECT * FROM %s WHERE a=0 AND b < now()");
-        assertEquals(1, results.size());
-    }
-
-    @Test
-    public void testDateCompatibility() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a int, b timestamp, c bigint, d varint, PRIMARY KEY (a, b, c, d))");
-        execute("INSERT INTO %s (a, b, c, d) VALUES (0, unixTimestampOf(now()), dateOf(now()), dateOf(now()))");
-        UntypedResultSet results = execute("SELECT * FROM %s WHERE a=0 AND b < unixTimestampOf(now())");
-        assertEquals(1, results.size());
-    }
-
-    @Test
-    public void testReversedTypeCompatibility() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a int, b timeuuid, PRIMARY KEY (a, b)) WITH CLUSTERING ORDER BY (b DESC)");
-        execute("INSERT INTO %s (a, b) VALUES (0, now())");
-        UntypedResultSet results = execute("SELECT * FROM %s WHERE a=0 AND b < now()");
-        assertEquals(1, results.size());
-    }
-
-    @Test
-    // tests CASSANDRA-7797
-    public void testAlterReversedColumn() throws Throwable
-    {
-        createTable("CREATE TABLE IF NOT EXISTS %s (a int, b 'org.apache.cassandra.db.marshal.DateType', PRIMARY KEY (a, b)) WITH CLUSTERING ORDER BY (b DESC)");
-        alterTable("ALTER TABLE %s ALTER b TYPE 'org.apache.cassandra.db.marshal.ReversedType(org.apache.cassandra.db.marshal.TimestampType)'");
-    }
-
-    @Test
-    public void testIncompatibleReversedTypes() throws Throwable
-    {
-        createTable("CREATE TABLE IF NOT EXISTS %s (a int, b 'org.apache.cassandra.db.marshal.DateType', PRIMARY KEY (a, b)) WITH CLUSTERING ORDER BY (b DESC)");
-        try
-        {
-            alterTable("ALTER TABLE %s ALTER b TYPE 'org.apache.cassandra.db.marshal.ReversedType(org.apache.cassandra.db.marshal.TimeUUIDType)'");
-            fail("Expected error for ALTER statement");
-        }
-        catch (RuntimeException e) { }
-    }
-
-    @Test
-    public void testReversedAndNonReversed() throws Throwable
-    {
-        createTable("CREATE TABLE IF NOT EXISTS %s (a int, b 'org.apache.cassandra.db.marshal.DateType', PRIMARY KEY (a, b))");
-        try
-        {
-            alterTable("ALTER TABLE %s ALTER b TYPE 'org.apache.cassandra.db.marshal.ReversedType(org.apache.cassandra.db.marshal.DateType)'");
-            fail("Expected error for ALTER statement");
-        }
-        catch (RuntimeException e) { }
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f797bfa4/test/unit/org/apache/cassandra/cql3/UseStatementTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/UseStatementTest.java b/test/unit/org/apache/cassandra/cql3/UseStatementTest.java
deleted file mode 100644
index 66b4b42..0000000
--- a/test/unit/org/apache/cassandra/cql3/UseStatementTest.java
+++ /dev/null
@@ -1,29 +0,0 @@
-/*
- * 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;
-
-import org.junit.Test;
-
-public class UseStatementTest extends CQLTester
-{
-    @Test
-    public void testUseStatementWithBindVariable() throws Throwable
-    {
-        assertInvalidSyntaxMessage("Bind variables cannot be used for keyspace names", "USE ?");
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f797bfa4/test/unit/org/apache/cassandra/cql3/UserTypesTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/UserTypesTest.java b/test/unit/org/apache/cassandra/cql3/UserTypesTest.java
deleted file mode 100644
index c820ece..0000000
--- a/test/unit/org/apache/cassandra/cql3/UserTypesTest.java
+++ /dev/null
@@ -1,319 +0,0 @@
-/*
- * 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;
-
-import org.junit.Test;
-
-public class UserTypesTest extends CQLTester
-{
-    @Test
-    public void testInvalidField() throws Throwable
-    {
-        String myType = createType("CREATE TYPE %s (f int)");
-        createTable("CREATE TABLE %s (k int PRIMARY KEY, v frozen<" + myType + ">)");
-
-        // 's' is not a field of myType
-        assertInvalid("INSERT INTO %s (k, v) VALUES (?, {s : ?})", 0, 1);
-    }
-
-    @Test
-    public void testCassandra8105() throws Throwable
-    {
-        String ut1 = createType("CREATE TYPE %s (a int, b int)");
-        String ut2 = createType("CREATE TYPE %s (j frozen<" + KEYSPACE + "." + ut1 + ">, k int)");
-        createTable("CREATE TABLE %s (x int PRIMARY KEY, y set<frozen<" + KEYSPACE + "." + ut2 + ">>)");
-        execute("INSERT INTO %s (x, y) VALUES (1, { { k: 1 } })");
-
-        String ut3 = createType("CREATE TYPE %s (a int, b int)");
-        String ut4 = createType("CREATE TYPE %s (j frozen<" + KEYSPACE + "." + ut3 + ">, k int)");
-        createTable("CREATE TABLE %s (x int PRIMARY KEY, y list<frozen<" + KEYSPACE + "." + ut4 + ">>)");
-        execute("INSERT INTO %s (x, y) VALUES (1, [ { k: 1 } ])");
-
-        String ut5 = createType("CREATE TYPE %s (a int, b int)");
-        String ut6 = createType("CREATE TYPE %s (i int, j frozen<" + KEYSPACE + "." + ut5 + ">)");
-        createTable("CREATE TABLE %s (x int PRIMARY KEY, y set<frozen<" + KEYSPACE + "." + ut6 + ">>)");
-        execute("INSERT INTO %s (x, y) VALUES (1, { { i: 1 } })");
-    }
-
-    @Test
-    public void testFor7684() throws Throwable
-    {
-        String myType = createType("CREATE TYPE %s (x double)");
-        createTable("CREATE TABLE %s (k int, v frozen<" + myType + ">, b boolean static, PRIMARY KEY (k, v))");
-
-        execute("INSERT INTO %s(k, v) VALUES (?, {x:?})", 1, -104.99251);
-        execute("UPDATE %s SET b = ? WHERE k = ?", true, 1);
-
-        assertRows(execute("SELECT v.x FROM %s WHERE k = ? AND v = {x:?}", 1, -104.99251),
-            row(-104.99251)
-        );
-
-        flush();
-
-        assertRows(execute("SELECT v.x FROM %s WHERE k = ? AND v = {x:?}", 1, -104.99251),
-            row(-104.99251)
-        );
-    }
-
-    @Test
-    public void testCreateInvalidTablesWithUDT() throws Throwable
-    {
-        String myType = createType("CREATE TYPE %s (f int)");
-
-        // Using a UDT without frozen shouldn't work
-        assertInvalidMessage("Non-frozen User-Defined types are not supported, please use frozen<>",
-                             "CREATE TABLE " + KEYSPACE + ".wrong (k int PRIMARY KEY, v " + KEYSPACE + '.' + myType + ")");
-
-        assertInvalidMessage("Statement on keyspace " + KEYSPACE + " cannot refer to a user type in keyspace otherkeyspace;" +
-                             " user types can only be used in the keyspace they are defined in",
-                             "CREATE TABLE " + KEYSPACE + ".wrong (k int PRIMARY KEY, v frozen<otherKeyspace.myType>)");
-
-        assertInvalidMessage("Unknown type " + KEYSPACE + ".unknowntype",
-                             "CREATE TABLE " + KEYSPACE + ".wrong (k int PRIMARY KEY, v frozen<" + KEYSPACE + '.' + "unknownType>)");
-    }
-
-    @Test
-    public void testAlterUDT() throws Throwable
-    {
-        String myType = KEYSPACE + '.' + createType("CREATE TYPE %s (a int)");
-        createTable("CREATE TABLE %s (a int PRIMARY KEY, b frozen<" + myType + ">)");
-        execute("INSERT INTO %s (a, b) VALUES (1, {a: 1})");
-
-        assertRows(execute("SELECT b.a FROM %s"), row(1));
-
-        flush();
-
-        execute("ALTER TYPE " + myType + " ADD b int");
-        execute("INSERT INTO %s (a, b) VALUES (2, {a: 2, b :2})");
-
-        assertRows(execute("SELECT b.a, b.b FROM %s"),
-                   row(1, null),
-                   row(2, 2));
-
-        flush();
-
-        assertRows(execute("SELECT b.a, b.b FROM %s"),
-                   row(1, null),
-                   row(2, 2));
-    }
-
-    @Test
-    public void testAlteringUserTypeNestedWithinMap() throws Throwable
-    {
-        // test frozen and non-frozen collections
-        String[] columnTypePrefixes = {"frozen<map<text, ", "map<text, frozen<"};
-        for (String columnTypePrefix : columnTypePrefixes)
-        {
-            String ut1 = createType("CREATE TYPE %s (a int)");
-            String columnType = columnTypePrefix + KEYSPACE + "." + ut1 + ">>";
-
-            createTable("CREATE TABLE %s (x int PRIMARY KEY, y " + columnType + ")");
-
-            execute("INSERT INTO %s (x, y) VALUES(1, {'firstValue':{a:1}})");
-            assertRows(execute("SELECT * FROM %s"), row(1, map("firstValue", userType(1))));
-            flush();
-
-            execute("ALTER TYPE " + KEYSPACE + "." + ut1 + " ADD b int");
-            execute("INSERT INTO %s (x, y) VALUES(2, {'secondValue':{a:2, b:2}})");
-            execute("INSERT INTO %s (x, y) VALUES(3, {'thirdValue':{a:3}})");
-            execute("INSERT INTO %s (x, y) VALUES(4, {'fourthValue':{b:4}})");
-
-            assertRows(execute("SELECT * FROM %s"),
-                    row(1, map("firstValue", userType(1))),
-                    row(2, map("secondValue", userType(2, 2))),
-                    row(3, map("thirdValue", userType(3, null))),
-                    row(4, map("fourthValue", userType(null, 4))));
-
-            flush();
-
-            assertRows(execute("SELECT * FROM %s"),
-                    row(1, map("firstValue", userType(1))),
-                    row(2, map("secondValue", userType(2, 2))),
-                    row(3, map("thirdValue", userType(3, null))),
-                    row(4, map("fourthValue", userType(null, 4))));
-        }
-    }
-
-    @Test
-    public void testAlteringUserTypeNestedWithinSet() throws Throwable
-    {
-        // test frozen and non-frozen collections
-        String[] columnTypePrefixes = {"frozen<set<", "set<frozen<"};
-        for (String columnTypePrefix : columnTypePrefixes)
-        {
-            String ut1 = createType("CREATE TYPE %s (a int)");
-            String columnType = columnTypePrefix + KEYSPACE + "." + ut1 + ">>";
-
-            createTable("CREATE TABLE %s (x int PRIMARY KEY, y " + columnType + ")");
-
-            execute("INSERT INTO %s (x, y) VALUES(1, {1} )");
-            assertRows(execute("SELECT * FROM %s"), row(1, set(userType(1))));
-            flush();
-
-            execute("ALTER TYPE " + KEYSPACE + "." + ut1 + " ADD b int");
-            execute("INSERT INTO %s (x, y) VALUES(2, {{a:2, b:2}})");
-            execute("INSERT INTO %s (x, y) VALUES(3, {{a:3}})");
-            execute("INSERT INTO %s (x, y) VALUES(4, {{b:4}})");
-
-            assertRows(execute("SELECT * FROM %s"),
-                    row(1, set(userType(1))),
-                    row(2, set(userType(2, 2))),
-                    row(3, set(userType(3, null))),
-                    row(4, set(userType(null, 4))));
-
-            flush();
-
-            assertRows(execute("SELECT * FROM %s"),
-                    row(1, set(userType(1))),
-                    row(2, set(userType(2, 2))),
-                    row(3, set(userType(3, null))),
-                    row(4, set(userType(null, 4))));
-        }
-    }
-
-    @Test
-    public void testAlteringUserTypeNestedWithinList() throws Throwable
-    {
-        // test frozen and non-frozen collections
-        String[] columnTypePrefixes = {"frozen<list<", "list<frozen<"};
-        for (String columnTypePrefix : columnTypePrefixes)
-        {
-            String ut1 = createType("CREATE TYPE %s (a int)");
-            String columnType = columnTypePrefix + KEYSPACE + "." + ut1 + ">>";
-
-            createTable("CREATE TABLE %s (x int PRIMARY KEY, y " + columnType + ")");
-
-            execute("INSERT INTO %s (x, y) VALUES(1, [1] )");
-            assertRows(execute("SELECT * FROM %s"), row(1, list(userType(1))));
-            flush();
-
-            execute("ALTER TYPE " + KEYSPACE + "." + ut1 + " ADD b int");
-            execute("INSERT INTO %s (x, y) VALUES(2, [{a:2, b:2}])");
-            execute("INSERT INTO %s (x, y) VALUES(3, [{a:3}])");
-            execute("INSERT INTO %s (x, y) VALUES(4, [{b:4}])");
-
-            assertRows(execute("SELECT * FROM %s"),
-                    row(1, list(userType(1))),
-                    row(2, list(userType(2, 2))),
-                    row(3, list(userType(3, null))),
-                    row(4, list(userType(null, 4))));
-
-            flush();
-
-            assertRows(execute("SELECT * FROM %s"),
-                    row(1, list(userType(1))),
-                    row(2, list(userType(2, 2))),
-                    row(3, list(userType(3, null))),
-                    row(4, list(userType(null, 4))));
-        }
-    }
-
-    @Test
-    public void testAlteringUserTypeNestedWithinTuple() throws Throwable
-    {
-        String type = createType("CREATE TYPE %s (a int, b int)");
-
-        createTable("CREATE TABLE %s (a int PRIMARY KEY, b frozen<tuple<int, " + KEYSPACE + "." + type + ">>)");
-
-        execute("INSERT INTO %s (a, b) VALUES(1, (1, {a:1, b:1}))");
-        assertRows(execute("SELECT * FROM %s"), row(1, tuple(1, userType(1, 1))));
-        flush();
-
-        execute("ALTER TYPE " + KEYSPACE + "." + type + " ADD c int");
-        execute("INSERT INTO %s (a, b) VALUES(2, (2, {a: 2, b: 2, c: 2}))");
-        execute("INSERT INTO %s (a, b) VALUES(3, (3, {a: 3, b: 3}))");
-        execute("INSERT INTO %s (a, b) VALUES(4, (4, {b:4}))");
-
-        assertRows(execute("SELECT * FROM %s"),
-                   row(1, tuple(1, userType(1, 1))),
-                   row(2, tuple(2, userType(2, 2, 2))),
-                   row(3, tuple(3, userType(3, 3, null))),
-                   row(4, tuple(4, userType(null, 4, null))));
-
-        flush();
-
-        assertRows(execute("SELECT * FROM %s"),
-                   row(1, tuple(1, userType(1, 1))),
-                   row(2, tuple(2, userType(2, 2, 2))),
-                   row(3, tuple(3, userType(3, 3, null))),
-                   row(4, tuple(4, userType(null, 4, null))));
-    }
-
-    @Test
-    public void testAlteringUserTypeNestedWithinNestedTuple() throws Throwable
-    {
-        String type = createType("CREATE TYPE %s (a int, b int)");
-
-        createTable("CREATE TABLE %s (a int PRIMARY KEY, b frozen<tuple<int, tuple<int, " + KEYSPACE + "." + type + ">>>)");
-
-        execute("INSERT INTO %s (a, b) VALUES(1, (1, (1, {a:1, b:1})))");
-        assertRows(execute("SELECT * FROM %s"), row(1, tuple(1, tuple(1, userType(1, 1)))));
-        flush();
-
-        execute("ALTER TYPE " + KEYSPACE + "." + type + " ADD c int");
-        execute("INSERT INTO %s (a, b) VALUES(2, (2, (1, {a: 2, b: 2, c: 2})))");
-        execute("INSERT INTO %s (a, b) VALUES(3, (3, (1, {a: 3, b: 3})))");
-        execute("INSERT INTO %s (a, b) VALUES(4, (4, (1, {b:4})))");
-
-        assertRows(execute("SELECT * FROM %s"),
-                   row(1, tuple(1, tuple(1, userType(1, 1)))),
-                   row(2, tuple(2, tuple(1, userType(2, 2, 2)))),
-                   row(3, tuple(3, tuple(1, userType(3, 3, null)))),
-                   row(4, tuple(4, tuple(1, userType(null, 4, null)))));
-
-        flush();
-
-        assertRows(execute("SELECT * FROM %s"),
-                   row(1, tuple(1, tuple(1, userType(1, 1)))),
-                   row(2, tuple(2, tuple(1, userType(2, 2, 2)))),
-                   row(3, tuple(3, tuple(1, userType(3, 3, null)))),
-                   row(4, tuple(4, tuple(1, userType(null, 4, null)))));
-    }
-
-    @Test
-    public void testAlteringUserTypeNestedWithinUserType() throws Throwable
-    {
-        String type = createType("CREATE TYPE %s (a int, b int)");
-        String otherType = createType("CREATE TYPE %s (x frozen<" + KEYSPACE + "." + type + ">)");
-
-        createTable("CREATE TABLE %s (a int PRIMARY KEY, b frozen<" + KEYSPACE + "." + otherType + ">)");
-
-        execute("INSERT INTO %s (a, b) VALUES(1, {x: {a:1, b:1}})");
-        assertRows(execute("SELECT b.x.a, b.x.b FROM %s"), row(1, 1));
-        flush();
-
-        execute("ALTER TYPE " + KEYSPACE + "." + type + " ADD c int");
-        execute("INSERT INTO %s (a, b) VALUES(2, {x: {a: 2, b: 2, c: 2}})");
-        execute("INSERT INTO %s (a, b) VALUES(3, {x: {a: 3, b: 3}})");
-        execute("INSERT INTO %s (a, b) VALUES(4, {x: {b:4}})");
-
-        assertRows(execute("SELECT b.x.a, b.x.b, b.x.c FROM %s"),
-                   row(1, 1, null),
-                   row(2, 2, 2),
-                   row(3, 3, null),
-                   row(null, 4, null));
-
-        flush();
-
-        assertRows(execute("SELECT b.x.a, b.x.b, b.x.c FROM %s"),
-                   row(1, 1, null),
-                   row(2, 2, 2),
-                   row(3, 3, null),
-                   row(null, 4, null));
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f797bfa4/test/unit/org/apache/cassandra/cql3/statements/SelectionColumnMappingTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/statements/SelectionColumnMappingTest.java b/test/unit/org/apache/cassandra/cql3/statements/SelectionColumnMappingTest.java
index 09b2bdd..ccd3570 100644
--- a/test/unit/org/apache/cassandra/cql3/statements/SelectionColumnMappingTest.java
+++ b/test/unit/org/apache/cassandra/cql3/statements/SelectionColumnMappingTest.java
@@ -3,12 +3,15 @@ package org.apache.cassandra.cql3.statements;
 import java.util.ArrayList;
 import java.util.List;
 
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.cql3.*;
 import org.apache.cassandra.db.marshal.*;
+import org.apache.cassandra.dht.ByteOrderedPartitioner;
 import org.apache.cassandra.exceptions.RequestValidationException;
 import org.apache.cassandra.service.ClientState;
 
@@ -20,6 +23,12 @@ public class SelectionColumnMappingTest extends CQLTester
     String tableName;
     String typeName;
 
+    @BeforeClass
+    public static void setUpClass()
+    {
+        DatabaseDescriptor.setPartitioner(new ByteOrderedPartitioner());
+    }
+
     @Test
     public void testSelectionColumnMapping() throws Throwable
     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f797bfa4/test/unit/org/apache/cassandra/cql3/validation/entities/CollectionsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/CollectionsTest.java b/test/unit/org/apache/cassandra/cql3/validation/entities/CollectionsTest.java
new file mode 100644
index 0000000..0241d4f
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/CollectionsTest.java
@@ -0,0 +1,488 @@
+/*
+ * 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.validation.entities;
+
+import java.util.UUID;
+
+import org.junit.Test;
+
+import org.apache.cassandra.cql3.CQLTester;
+
+public class CollectionsTest extends CQLTester
+{
+    @Test
+    public void testMapBulkRemoval() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, m map<text, text>)");
+
+        execute("INSERT INTO %s(k, m) VALUES (?, ?)", 0, map("k1", "v1", "k2", "v2", "k3", "v3"));
+
+        assertRows(execute("SELECT * FROM %s"),
+            row(0, map("k1", "v1", "k2", "v2", "k3", "v3"))
+        );
+
+        execute("UPDATE %s SET m = m - ? WHERE k = ?", set("k2"), 0);
+
+        assertRows(execute("SELECT * FROM %s"),
+            row(0, map("k1", "v1", "k3", "v3"))
+        );
+
+        execute("UPDATE %s SET m = m + ?, m = m - ? WHERE k = ?", map("k4", "v4"), set("k3"), 0);
+
+        assertRows(execute("SELECT * FROM %s"),
+            row(0, map("k1", "v1", "k4", "v4"))
+        );
+    }
+
+    @Test
+    public void testInvalidCollectionsMix() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, l list<text>, s set<text>, m map<text, text>)");
+
+        // Note: we force the non-prepared form for some of those tests because a list and a set
+        // have the same serialized format in practice and CQLTester don't validate that the type
+        // of what's passed as a value in the prepared case, so the queries would work (which is ok,
+        // CQLTester is just a "dumb" client).
+
+        assertInvalid("UPDATE %s SET l = l + { 'a', 'b' } WHERE k = 0");
+        assertInvalid("UPDATE %s SET l = l - { 'a', 'b' } WHERE k = 0");
+        assertInvalid("UPDATE %s SET l = l + ? WHERE k = 0", map("a", "b", "c", "d"));
+        assertInvalid("UPDATE %s SET l = l - ? WHERE k = 0", map("a", "b", "c", "d"));
+
+        assertInvalid("UPDATE %s SET s = s + [ 'a', 'b' ] WHERE k = 0");
+        assertInvalid("UPDATE %s SET s = s - [ 'a', 'b' ] WHERE k = 0");
+        assertInvalid("UPDATE %s SET s = s + ? WHERE k = 0", map("a", "b", "c", "d"));
+        assertInvalid("UPDATE %s SET s = s - ? WHERE k = 0", map("a", "b", "c", "d"));
+
+        assertInvalid("UPDATE %s SET m = m + ? WHERE k = 0", list("a", "b"));
+        assertInvalid("UPDATE %s SET m = m - [ 'a', 'b' ] WHERE k = 0");
+        assertInvalid("UPDATE %s SET m = m + ? WHERE k = 0", set("a", "b"));
+        assertInvalid("UPDATE %s SET m = m - ? WHERE k = 0", map("a", "b", "c", "d"));
+    }
+
+    @Test
+    public void testSets() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, s set<text>)");
+
+        execute("INSERT INTO %s(k, s) VALUES (0, ?)", set("v1", "v2", "v3", "v4"));
+
+        assertRows(execute("SELECT s FROM %s WHERE k = 0"),
+            row(set("v1", "v2", "v3", "v4"))
+        );
+
+        execute("DELETE s[?] FROM %s WHERE k = 0", "v1");
+
+        assertRows(execute("SELECT s FROM %s WHERE k = 0"),
+                   row(set("v2", "v3", "v4"))
+        );
+
+        // Full overwrite
+        execute("UPDATE %s SET s = ? WHERE k = 0", set("v6", "v5"));
+
+        assertRows(execute("SELECT s FROM %s WHERE k = 0"),
+                   row(set("v5", "v6"))
+        );
+
+        execute("UPDATE %s SET s = s + ? WHERE k = 0", set("v7"));
+
+        assertRows(execute("SELECT s FROM %s WHERE k = 0"),
+                   row(set("v5", "v6", "v7"))
+        );
+
+        execute("UPDATE %s SET s = s - ? WHERE k = 0", set("v6", "v5"));
+
+        assertRows(execute("SELECT s FROM %s WHERE k = 0"),
+                   row(set("v7"))
+        );
+
+        execute("DELETE s[?] FROM %s WHERE k = 0", set("v7"));
+
+        // Deleting an element that does not exist will succeed
+        execute("DELETE s[?] FROM %s WHERE k = 0", set("v7"));
+
+        execute("DELETE s FROM %s WHERE k = 0");
+
+        assertRows(execute("SELECT s FROM %s WHERE k = 0"),
+                   row((Object) null)
+        );
+    }
+
+    @Test
+    public void testMaps() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, m map<text, int>)");
+
+        execute("INSERT INTO %s(k, m) VALUES (0, ?)", map("v1", 1, "v2", 2));
+
+        assertRows(execute("SELECT m FROM %s WHERE k = 0"),
+            row(map("v1", 1, "v2", 2))
+        );
+
+        execute("UPDATE %s SET m[?] = ?, m[?] = ? WHERE k = 0", "v3", 3, "v4", 4);
+
+        assertRows(execute("SELECT m FROM %s WHERE k = 0"),
+            row(map("v1", 1, "v2", 2, "v3", 3, "v4", 4))
+        );
+
+        execute("DELETE m[?] FROM %s WHERE k = 0", "v1");
+
+        assertRows(execute("SELECT m FROM %s WHERE k = 0"),
+            row(map("v2", 2, "v3", 3, "v4", 4))
+        );
+
+        // Full overwrite
+        execute("UPDATE %s SET m = ? WHERE k = 0", map("v6", 6, "v5", 5));
+
+        assertRows(execute("SELECT m FROM %s WHERE k = 0"),
+                   row(map("v5", 5, "v6", 6))
+        );
+
+        execute("UPDATE %s SET m = m + ? WHERE k = 0", map("v7", 7));
+
+        assertRows(execute("SELECT m FROM %s WHERE k = 0"),
+                   row(map("v5", 5, "v6", 6, "v7", 7))
+        );
+
+        execute("DELETE m[?] FROM %s WHERE k = 0", "v7");
+
+        assertRows(execute("SELECT m FROM %s WHERE k = 0"),
+                   row(map("v5", 5, "v6", 6))
+        );
+
+        execute("DELETE m[?] FROM %s WHERE k = 0", "v6");
+
+        assertRows(execute("SELECT m FROM %s WHERE k = 0"),
+                   row(map("v5", 5))
+        );
+
+        execute("DELETE m[?] FROM %s WHERE k = 0", "v5");
+
+        assertRows(execute("SELECT m FROM %s WHERE k = 0"),
+                   row((Object) null)
+        );
+
+        // Deleting a non-existing key should succeed
+        execute("DELETE m[?] FROM %s WHERE k = 0", "v5");
+
+        assertRows(execute("SELECT m FROM %s WHERE k = 0"),
+                   row((Object) null)
+        );
+
+        // The empty map is parsed as an empty set (because we don't have enough info at parsing
+        // time when we see a {}) and special cased later. This test checks this work properly
+        execute("UPDATE %s SET m = {} WHERE k = 0");
+
+        assertRows(execute("SELECT m FROM %s WHERE k = 0"),
+            row((Object)null)
+        );
+    }
+
+    @Test
+    public void testLists() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, l list<text>)");
+
+        execute("INSERT INTO %s(k, l) VALUES (0, ?)", list("v1", "v2", "v3"));
+
+        assertRows(execute("SELECT l FROM %s WHERE k = 0"), row(list("v1", "v2", "v3")));
+
+        execute("DELETE l[?] FROM %s WHERE k = 0", 1);
+
+        assertRows(execute("SELECT l FROM %s WHERE k = 0"), row(list("v1", "v3")));
+
+        execute("UPDATE %s SET l[?] = ? WHERE k = 0", 1, "v4");
+
+        assertRows(execute("SELECT l FROM %s WHERE k = 0"), row(list("v1", "v4")));
+
+        // Full overwrite
+        execute("UPDATE %s SET l = ? WHERE k = 0", list("v6", "v5"));
+
+        assertRows(execute("SELECT l FROM %s WHERE k = 0"), row(list("v6", "v5")));
+
+        execute("UPDATE %s SET l = l + ? WHERE k = 0", list("v7", "v8"));
+
+        assertRows(execute("SELECT l FROM %s WHERE k = 0"), row(list("v6", "v5", "v7", "v8")));
+
+        execute("UPDATE %s SET l = ? + l WHERE k = 0", list("v9"));
+
+        assertRows(execute("SELECT l FROM %s WHERE k = 0"), row(list("v9", "v6", "v5", "v7", "v8")));
+
+        execute("UPDATE %s SET l = l - ? WHERE k = 0", list("v5", "v8"));
+
+        assertRows(execute("SELECT l FROM %s WHERE k = 0"), row(list("v9", "v6", "v7")));
+
+        execute("DELETE l FROM %s WHERE k = 0");
+
+        assertRows(execute("SELECT l FROM %s WHERE k = 0"), row((Object) null));
+
+        assertInvalidMessage("Attempted to delete an element from a list which is null",
+                             "DELETE l[0] FROM %s WHERE k=0 ");
+
+        assertInvalidMessage("Attempted to set an element on a list which is null",
+                             "UPDATE %s SET l[0] = ? WHERE k=0", list("v10"));
+
+        execute("UPDATE %s SET l = l - ? WHERE k=0 ", list("v11"));
+
+        assertRows(execute("SELECT l FROM %s WHERE k = 0"), row((Object) null));
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.set_test()
+     */
+    @Test
+    public void testSet() throws Throwable
+    {
+        createTable("CREATE TABLE %s ( fn text, ln text, tags set<text>, PRIMARY KEY (fn, ln) )");
+
+        execute("UPDATE %s SET tags = tags + { 'foo' } WHERE fn='Tom' AND ln='Bombadil'");
+        execute("UPDATE %s SET tags = tags + { 'bar' } WHERE fn='Tom' AND ln='Bombadil'");
+        execute("UPDATE %s SET tags = tags + { 'foo' } WHERE fn='Tom' AND ln='Bombadil'");
+        execute("UPDATE %s SET tags = tags + { 'foobar' } WHERE fn='Tom' AND ln='Bombadil'");
+        execute("UPDATE %s SET tags = tags - { 'bar' } WHERE fn='Tom' AND ln='Bombadil'");
+
+        assertRows(execute("SELECT tags FROM %s"),
+                   row(set("foo", "foobar")));
+
+        execute("UPDATE %s SET tags = { 'a', 'c', 'b' } WHERE fn='Bilbo' AND ln='Baggins'");
+        assertRows(execute("SELECT tags FROM %s WHERE fn='Bilbo' AND ln='Baggins'"),
+                   row(set("a", "b", "c")));
+
+        execute("UPDATE %s SET tags = { 'm', 'n' } WHERE fn='Bilbo' AND ln='Baggins'");
+        assertRows(execute("SELECT tags FROM %s WHERE fn='Bilbo' AND ln='Baggins'"),
+                   row(set("m", "n")));
+
+        execute("DELETE tags['m'] FROM %s WHERE fn='Bilbo' AND ln='Baggins'");
+        assertRows(execute("SELECT tags FROM %s WHERE fn='Bilbo' AND ln='Baggins'"),
+                   row(set("n")));
+
+        execute("DELETE tags FROM %s WHERE fn='Bilbo' AND ln='Baggins'");
+        assertEmpty(execute("SELECT tags FROM %s WHERE fn='Bilbo' AND ln='Baggins'"));
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.map_test()
+     */
+    @Test
+    public void testMap() throws Throwable
+    {
+        createTable("CREATE TABLE %s (fn text, ln text, m map<text, int>, PRIMARY KEY (fn, ln))");
+
+        execute("UPDATE %s SET m['foo'] = 3 WHERE fn='Tom' AND ln='Bombadil'");
+        execute("UPDATE %s SET m['bar'] = 4 WHERE fn='Tom' AND ln='Bombadil'");
+        execute("UPDATE %s SET m['woot'] = 5 WHERE fn='Tom' AND ln='Bombadil'");
+        execute("UPDATE %s SET m['bar'] = 6 WHERE fn='Tom' AND ln='Bombadil'");
+        execute("DELETE m['foo'] FROM %s WHERE fn='Tom' AND ln='Bombadil'");
+
+        assertRows(execute("SELECT m FROM %s"),
+                   row(map("bar", 6, "woot", 5)));
+
+        execute("UPDATE %s SET m = { 'a' : 4 , 'c' : 3, 'b' : 2 } WHERE fn='Bilbo' AND ln='Baggins'");
+        assertRows(execute("SELECT m FROM %s WHERE fn='Bilbo' AND ln='Baggins'"),
+                   row(map("a", 4, "b", 2, "c", 3)));
+
+        execute("UPDATE %s SET m =  { 'm' : 4 , 'n' : 1, 'o' : 2 } WHERE fn='Bilbo' AND ln='Baggins'");
+        assertRows(execute("SELECT m FROM %s WHERE fn='Bilbo' AND ln='Baggins'"),
+                   row(map("m", 4, "n", 1, "o", 2)));
+
+        execute("UPDATE %s SET m = {} WHERE fn='Bilbo' AND ln='Baggins'");
+        assertEmpty(execute("SELECT m FROM %s WHERE fn='Bilbo' AND ln='Baggins'"));
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.list_test()
+     */
+    @Test
+    public void testList() throws Throwable
+    {
+        createTable("CREATE TABLE %s (fn text, ln text, tags list<text>, PRIMARY KEY (fn, ln))");
+
+        execute("UPDATE %s SET tags = tags + [ 'foo' ] WHERE fn='Tom' AND ln='Bombadil'");
+        execute("UPDATE %s SET tags = tags + [ 'bar' ] WHERE fn='Tom' AND ln='Bombadil'");
+        execute("UPDATE %s SET tags = tags + [ 'foo' ] WHERE fn='Tom' AND ln='Bombadil'");
+        execute("UPDATE %s SET tags = tags + [ 'foobar' ] WHERE fn='Tom' AND ln='Bombadil'");
+
+        assertRows(execute("SELECT tags FROM %s"),
+                   row(list("foo", "bar", "foo", "foobar")));
+
+        execute("UPDATE %s SET tags = [ 'a', 'c', 'b', 'c' ] WHERE fn='Bilbo' AND ln='Baggins'");
+        assertRows(execute("SELECT tags FROM %s WHERE fn='Bilbo' AND ln='Baggins'"),
+                   row(list("a", "c", "b", "c")));
+
+        execute("UPDATE %s SET tags = [ 'm', 'n' ] + tags WHERE fn='Bilbo' AND ln='Baggins'");
+        assertRows(execute("SELECT tags FROM %s WHERE fn='Bilbo' AND ln='Baggins'"),
+                   row(list("m", "n", "a", "c", "b", "c")));
+
+        execute("UPDATE %s SET tags[2] = 'foo', tags[4] = 'bar' WHERE fn='Bilbo' AND ln='Baggins'");
+        assertRows(execute("SELECT tags FROM %s WHERE fn='Bilbo' AND ln='Baggins'"),
+                   row(list("m", "n", "foo", "c", "bar", "c")));
+
+        execute("DELETE tags[2] FROM %s WHERE fn='Bilbo' AND ln='Baggins'");
+        assertRows(execute("SELECT tags FROM %s WHERE fn='Bilbo' AND ln='Baggins'"),
+                   row(list("m", "n", "c", "bar", "c")));
+
+        execute("UPDATE %s SET tags = tags - [ 'bar' ] WHERE fn='Bilbo' AND ln='Baggins'");
+        assertRows(execute("SELECT tags FROM %s WHERE fn='Bilbo' AND ln='Baggins'"),
+                   row(list("m", "n", "c", "c")));
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.multi_collection_test()
+     */
+    @Test
+    public void testMultiCollections() throws Throwable
+    {
+        UUID id = UUID.fromString("b017f48f-ae67-11e1-9096-005056c00008");
+
+        createTable("CREATE TABLE %s (k uuid PRIMARY KEY, L list<int>, M map<text, int>, S set<int> )");
+
+        execute("UPDATE %s SET L = [1, 3, 5] WHERE k = ?", id);
+        execute("UPDATE %s SET L = L + [7, 11, 13] WHERE k = ?;", id);
+        execute("UPDATE %s SET S = {1, 3, 5} WHERE k = ?", id);
+        execute("UPDATE %s SET S = S + {7, 11, 13} WHERE k = ?", id);
+        execute("UPDATE %s SET M = {'foo': 1, 'bar' : 3} WHERE k = ?", id);
+        execute("UPDATE %s SET M = M + {'foobar' : 4} WHERE k = ?", id);
+
+        assertRows(execute("SELECT L, M, S FROM %s WHERE k = ?", id),
+                   row(list(1, 3, 5, 7, 11, 13),
+                       map("bar", 3, "foo", 1, "foobar", 4),
+                       set(1, 3, 5, 7, 11, 13)));
+    }
+
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.collection_and_regular_test()
+     */
+    @Test
+    public void testCollectionAndRegularColumns() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, l list<int>, c int)");
+
+        execute("INSERT INTO %s (k, l, c) VALUES(3, [0, 1, 2], 4)");
+        execute("UPDATE %s SET l[0] = 1, c = 42 WHERE k = 3");
+        assertRows(execute("SELECT l, c FROM %s WHERE k = 3"),
+                   row(list(1, 1, 2), 42));
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.multi_list_set_test()
+     */
+    @Test
+    public void testMultipleLists() throws Throwable
+    {
+        createTable(" CREATE TABLE %s (k int PRIMARY KEY, l1 list<int>, l2 list<int>)");
+
+        execute("INSERT INTO %s (k, l1, l2) VALUES (0, [1, 2, 3], [4, 5, 6])");
+        execute("UPDATE %s SET l2[1] = 42, l1[1] = 24  WHERE k = 0");
+
+        assertRows(execute("SELECT l1, l2 FROM %s WHERE k = 0"),
+                   row(list(1, 24, 3), list(4, 42, 6)));
+    }
+
+    /**
+     * Test you can add columns in a table with collections (#4982 bug),
+     * migrated from cql_tests.py:TestCQL.alter_with_collections_test()
+     */
+    @Test
+    public void testAlterCollections() throws Throwable
+    {
+        createTable("CREATE TABLE %s (key int PRIMARY KEY, aset set<text>)");
+        execute("ALTER TABLE %s ADD c text");
+        execute("ALTER TABLE %s ADD alist list<text>");
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.collection_compact_test()
+     */
+    @Test
+    public void testCompactCollections() throws Throwable
+    {
+        String tableName = KEYSPACE + "." + createTableName();
+        assertInvalid(String.format("CREATE TABLE %s (user ascii PRIMARY KEY, mails list < text >) WITH COMPACT STORAGE;", tableName));
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.collection_function_test()
+     */
+    @Test
+    public void testFunctionsOnCollections() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, l set<int>)");
+
+        assertInvalid("SELECT ttl(l) FROM %s WHERE k = 0");
+        assertInvalid("SELECT writetime(l) FROM %s WHERE k = 0");
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.bug_5376()
+     */
+    @Test
+    public void testInClauseWithCollections() throws Throwable
+    {
+        createTable("CREATE TABLE %s (key text, c bigint, v text, x set < text >, PRIMARY KEY(key, c) )");
+
+        assertInvalid("select * from %s where key = 'foo' and c in (1,3,4)");
+    }
+
+    /**
+     * Test for bug #5795,
+     * migrated from cql_tests.py:TestCQL.nonpure_function_collection_test()
+     */
+    @Test
+    public void testNonPureFunctionCollection() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, v list<timeuuid>)");
+
+        // we just want to make sure this doesn't throw
+        execute("INSERT INTO %s (k, v) VALUES (0, [now()])");
+    }
+
+    /**
+     * Test for 5805 bug,
+     * migrated from cql_tests.py:TestCQL.collection_flush_test()
+     */
+    @Test
+    public void testCollectionFlush() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, s set<int>)");
+
+        execute("INSERT INTO %s (k, s) VALUES (1, {1})");
+        flush();
+
+        execute("INSERT INTO %s (k, s) VALUES (1, {2})");
+        flush();
+
+        assertRows(execute("SELECT * FROM %s"),
+                   row(1, set(2)));
+    }
+
+    /**
+     * Test for 6276,
+     * migrated from cql_tests.py:TestCQL.drop_and_readd_collection_test()
+     */
+    @Test
+    public void testDropAndReaddCollection() throws Throwable
+    {
+        createTable("create table %s (k int primary key, v set<text>, x int)");
+        execute("insert into %s (k, v) VALUES (0, {'fffffffff'})");
+        flush();
+        execute("alter table %s drop v");
+        assertInvalid("alter table %s add v set<int>");
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f797bfa4/test/unit/org/apache/cassandra/cql3/validation/entities/CountersTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/CountersTest.java b/test/unit/org/apache/cassandra/cql3/validation/entities/CountersTest.java
new file mode 100644
index 0000000..c66a04d
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/CountersTest.java
@@ -0,0 +1,85 @@
+/*
+ * 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.validation.entities;
+
+import org.junit.Test;
+
+import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+
+public class CountersTest extends CQLTester
+{
+    /**
+     * Check for a table with counters,
+     * migrated from cql_tests.py:TestCQL.counters_test()
+     */
+    @Test
+    public void testCounters() throws Throwable
+    {
+        createTable("CREATE TABLE %s (userid int, url text, total counter, PRIMARY KEY (userid, url)) WITH COMPACT STORAGE");
+
+        execute("UPDATE %s SET total = total + 1 WHERE userid = 1 AND url = 'http://foo.com'");
+        assertRows(execute("SELECT total FROM %s WHERE userid = 1 AND url = 'http://foo.com'"),
+                   row(1L));
+
+        execute("UPDATE %s SET total = total - 4 WHERE userid = 1 AND url = 'http://foo.com'");
+        assertRows(execute("SELECT total FROM %s WHERE userid = 1 AND url = 'http://foo.com'"),
+                   row(-3L));
+
+        execute("UPDATE %s SET total = total+1 WHERE userid = 1 AND url = 'http://foo.com'");
+        assertRows(execute("SELECT total FROM %s WHERE userid = 1 AND url = 'http://foo.com'"),
+                   row(-2L));
+
+        execute("UPDATE %s SET total = total -2 WHERE userid = 1 AND url = 'http://foo.com'");
+        assertRows(execute("SELECT total FROM %s WHERE userid = 1 AND url = 'http://foo.com'"),
+                   row(-4L));
+    }
+
+    /**
+     * Test for the validation bug of #4706,
+     * migrated from cql_tests.py:TestCQL.validate_counter_regular_test()
+     */
+    @Test
+    public void testRegularCounters() throws Throwable
+    {
+        assertInvalidThrowMessage("Cannot add a non counter column",
+                                  ConfigurationException.class,
+                                  String.format("CREATE TABLE %s.%s (id bigint PRIMARY KEY, count counter, things set<text>)", KEYSPACE, createTableName()));
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.collection_counter_test()
+     */
+    @Test
+    public void testCountersOnCollections() throws Throwable
+    {
+        String tableName = KEYSPACE + "." + createTableName();
+        assertInvalidThrow(InvalidRequestException.class,
+                           String.format("CREATE TABLE %s (k int PRIMARY KEY, l list<counter>)", tableName));
+
+        tableName = KEYSPACE + "." + createTableName();
+        assertInvalidThrow(InvalidRequestException.class,
+                           String.format("CREATE TABLE %s (k int PRIMARY KEY, s set<counter>)", tableName));
+
+        tableName = KEYSPACE + "." + createTableName();
+        assertInvalidThrow(InvalidRequestException.class,
+                           String.format("CREATE TABLE %s (k int PRIMARY KEY, m map<text, counter>)", tableName));
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f797bfa4/test/unit/org/apache/cassandra/cql3/validation/entities/DateTypeTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/DateTypeTest.java b/test/unit/org/apache/cassandra/cql3/validation/entities/DateTypeTest.java
new file mode 100644
index 0000000..7fa5e67
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/DateTypeTest.java
@@ -0,0 +1,39 @@
+/*
+ * 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.validation.entities;
+
+import org.junit.Test;
+
+import org.apache.cassandra.cql3.CQLTester;
+
+public class DateTypeTest extends CQLTester
+{
+    /**
+     * Check dates are correctly recognized and validated,
+     * migrated from cql_tests.py:TestCQL.date_test()
+     */
+    @Test
+    public void testDate() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, t timestamp)");
+
+        execute("INSERT INTO %s (k, t) VALUES (0, '2011-02-03')");
+        assertInvalid("INSERT INTO %s (k, t) VALUES (0, '2011-42-42')");
+    }
+}


[14/32] cassandra git commit: 2.2 commit for CASSANDRA-9160

Posted by jm...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/01115f72/test/unit/org/apache/cassandra/cql3/validation/operations/BatchTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/BatchTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/BatchTest.java
new file mode 100644
index 0000000..1447845
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/BatchTest.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.validation.operations;
+
+import org.junit.Test;
+
+import org.apache.cassandra.cql3.CQLTester;
+
+public class BatchTest extends CQLTester
+{
+    /**
+     * Test batch statements
+     * migrated from cql_tests.py:TestCQL.batch_test()
+     */
+    @Test
+    public void testBatch() throws Throwable
+    {
+        createTable("CREATE TABLE %s (userid text PRIMARY KEY, name text, password text)");
+
+        String query = "BEGIN BATCH\n"
+                       + "INSERT INTO %1$s (userid, password, name) VALUES ('user2', 'ch@ngem3b', 'second user');\n"
+                       + "UPDATE %1$s SET password = 'ps22dhds' WHERE userid = 'user3';\n"
+                       + "INSERT INTO %1$s (userid, password) VALUES ('user4', 'ch@ngem3c');\n"
+                       + "DELETE name FROM %1$s WHERE userid = 'user1';\n"
+                       + "APPLY BATCH;";
+
+        execute(query);
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.batch_and_list_test()
+     */
+    @Test
+    public void testBatchAndList() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, l list<int>)");
+
+        execute("BEGIN BATCH " +
+                "UPDATE %1$s SET l = l +[ 1 ] WHERE k = 0; " +
+                "UPDATE %1$s SET l = l + [ 2 ] WHERE k = 0; " +
+                "UPDATE %1$s SET l = l + [ 3 ] WHERE k = 0; " +
+                "APPLY BATCH");
+
+        assertRows(execute("SELECT l FROM %s WHERE k = 0"),
+                   row(list(1, 2, 3)));
+
+        execute("BEGIN BATCH " +
+                "UPDATE %1$s SET l =[ 1 ] + l WHERE k = 1; " +
+                "UPDATE %1$s SET l = [ 2 ] + l WHERE k = 1; " +
+                "UPDATE %1$s SET l = [ 3 ] + l WHERE k = 1; " +
+                "APPLY BATCH ");
+
+        assertRows(execute("SELECT l FROM %s WHERE k = 1"),
+                   row(list(3, 2, 1)));
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.bug_6115_test()
+     */
+    @Test
+    public void testBatchDeleteInsert() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int, v int, PRIMARY KEY (k, v))");
+
+        execute("INSERT INTO %s (k, v) VALUES (0, 1)");
+        execute("BEGIN BATCH DELETE FROM %1$s WHERE k=0 AND v=1; INSERT INTO %1$s (k, v) VALUES (0, 2); APPLY BATCH");
+
+        assertRows(execute("SELECT * FROM %s"),
+                   row(0, 2));
+    }
+
+    @Test
+    public void testBatchWithUnset() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, s text, i int)");
+
+        // test batch and update
+        String qualifiedTable = keyspace() + "." + currentTable();
+        execute("BEGIN BATCH " +
+                "INSERT INTO %s (k, s, i) VALUES (100, 'batchtext', 7); " +
+                "INSERT INTO " + qualifiedTable + " (k, s, i) VALUES (111, 'batchtext', 7); " +
+                "UPDATE " + qualifiedTable + " SET s=?, i=? WHERE k = 100; " +
+                "UPDATE " + qualifiedTable + " SET s=?, i=? WHERE k=111; " +
+                "APPLY BATCH;", null, unset(), unset(), null);
+        assertRows(execute("SELECT k, s, i FROM %s where k in (100,111)"),
+                   row(100, null, 7),
+                   row(111, "batchtext", null)
+        );
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/01115f72/test/unit/org/apache/cassandra/cql3/validation/operations/CreateTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/CreateTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/CreateTest.java
new file mode 100644
index 0000000..f3d98ff
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/CreateTest.java
@@ -0,0 +1,498 @@
+/*
+ * 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.validation.operations;
+
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.UUID;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.config.TriggerDefinition;
+import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.db.ColumnFamily;
+import org.apache.cassandra.db.Mutation;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.exceptions.SyntaxException;
+import org.apache.cassandra.triggers.ITrigger;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+import static junit.framework.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+public class CreateTest extends CQLTester
+{
+    @Test
+    public void testCQL3PartitionKeyOnlyTable()
+    {
+        createTable("CREATE TABLE %s (id text PRIMARY KEY);");
+        assertFalse(currentTableMetadata().isThriftCompatible());
+    }
+
+    @Test
+    public void testCreateTableWithSmallintColumns() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a text, b smallint, c smallint, primary key (a, b));");
+        execute("INSERT INTO %s (a, b, c) VALUES ('1', 1, 2)");
+        execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", "2", Short.MAX_VALUE, Short.MIN_VALUE);
+
+        assertRows(execute("SELECT * FROM %s"),
+                   row("2", Short.MAX_VALUE, Short.MIN_VALUE),
+                   row("1", (short) 1, (short) 2));
+
+        assertInvalidMessage("Expected 2 bytes for a smallint (4)",
+                             "INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", "3", 1, 2);
+        assertInvalidMessage("Expected 2 bytes for a smallint (0)",
+                             "INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", "3", (short) 1, ByteBufferUtil.EMPTY_BYTE_BUFFER);
+    }
+
+    @Test
+    public void testCreateTinyintColumns() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a text, b tinyint, c tinyint, primary key (a, b));");
+        execute("INSERT INTO %s (a, b, c) VALUES ('1', 1, 2)");
+        execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", "2", Byte.MAX_VALUE, Byte.MIN_VALUE);
+
+        assertRows(execute("SELECT * FROM %s"),
+                   row("2", Byte.MAX_VALUE, Byte.MIN_VALUE),
+                   row("1", (byte) 1, (byte) 2));
+
+        assertInvalidMessage("Expected 1 byte for a tinyint (4)",
+                             "INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", "3", 1, 2);
+
+        assertInvalidMessage("Expected 1 byte for a tinyint (0)",
+                             "INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", "3", (byte) 1, ByteBufferUtil.EMPTY_BYTE_BUFFER);
+    }
+
+    /**
+     * Creation and basic operations on a static table,
+     * migrated from cql_tests.py:TestCQL.static_cf_test()
+     */
+    @Test
+    public void testStaticTable() throws Throwable
+    {
+        createTable("CREATE TABLE %s (userid uuid PRIMARY KEY, firstname text, lastname text, age int)");
+
+        UUID id1 = UUID.fromString("550e8400-e29b-41d4-a716-446655440000");
+        UUID id2 = UUID.fromString("f47ac10b-58cc-4372-a567-0e02b2c3d479");
+
+        execute("INSERT INTO %s (userid, firstname, lastname, age) VALUES (?, ?, ?, ?)", id1, "Frodo", "Baggins", 32);
+        execute("UPDATE %s SET firstname = ?, lastname = ?, age = ? WHERE userid = ?", "Samwise", "Gamgee", 33, id2);
+
+        assertRows(execute("SELECT firstname, lastname FROM %s WHERE userid = ?", id1),
+                   row("Frodo", "Baggins"));
+
+        assertRows(execute("SELECT * FROM %s WHERE userid = ?", id1),
+                   row(id1, 32, "Frodo", "Baggins"));
+
+        assertRows(execute("SELECT * FROM %s"),
+                   row(id2, 33, "Samwise", "Gamgee"),
+                   row(id1, 32, "Frodo", "Baggins")
+        );
+
+        String batch = "BEGIN BATCH "
+                       + "INSERT INTO %1$s (userid, age) VALUES (?, ?) "
+                       + "UPDATE %1$s SET age = ? WHERE userid = ? "
+                       + "DELETE firstname, lastname FROM %1$s WHERE userid = ? "
+                       + "DELETE firstname, lastname FROM %1$s WHERE userid = ? "
+                       + "APPLY BATCH";
+
+        execute(batch, id1, 36, 37, id2, id1, id2);
+
+        assertRows(execute("SELECT * FROM %s"),
+                   row(id2, 37, null, null),
+                   row(id1, 36, null, null));
+    }
+
+    /**
+     * Creation and basic operations on a static table with compact storage,
+     * migrated from cql_tests.py:TestCQL.noncomposite_static_cf_test()
+     */
+    @Test
+    public void testDenseStaticTable() throws Throwable
+    {
+        createTable("CREATE TABLE %s (userid uuid PRIMARY KEY, firstname text, lastname text, age int) WITH COMPACT STORAGE");
+
+        UUID id1 = UUID.fromString("550e8400-e29b-41d4-a716-446655440000");
+        UUID id2 = UUID.fromString("f47ac10b-58cc-4372-a567-0e02b2c3d479");
+
+        execute("INSERT INTO %s (userid, firstname, lastname, age) VALUES (?, ?, ?, ?)", id1, "Frodo", "Baggins", 32);
+        execute("UPDATE %s SET firstname = ?, lastname = ?, age = ? WHERE userid = ?", "Samwise", "Gamgee", 33, id2);
+
+        assertRows(execute("SELECT firstname, lastname FROM %s WHERE userid = ?", id1),
+                   row("Frodo", "Baggins"));
+
+        assertRows(execute("SELECT * FROM %s WHERE userid = ?", id1),
+                   row(id1, 32, "Frodo", "Baggins"));
+
+        assertRows(execute("SELECT * FROM %s"),
+                   row(id2, 33, "Samwise", "Gamgee"),
+                   row(id1, 32, "Frodo", "Baggins")
+        );
+
+        String batch = "BEGIN BATCH "
+                       + "INSERT INTO %1$s (userid, age) VALUES (?, ?) "
+                       + "UPDATE %1$s SET age = ? WHERE userid = ? "
+                       + "DELETE firstname, lastname FROM %1$s WHERE userid = ? "
+                       + "DELETE firstname, lastname FROM %1$s WHERE userid = ? "
+                       + "APPLY BATCH";
+
+        execute(batch, id1, 36, 37, id2, id1, id2);
+
+        assertRows(execute("SELECT * FROM %s"),
+                   row(id2, 37, null, null),
+                   row(id1, 36, null, null));
+    }
+
+    /**
+     * Creation and basic operations on a non-composite table with compact storage,
+     * migrated from cql_tests.py:TestCQL.dynamic_cf_test()
+     */
+    @Test
+    public void testDenseNonCompositeTable() throws Throwable
+    {
+        createTable("CREATE TABLE %s (userid uuid, url text, time bigint, PRIMARY KEY (userid, url)) WITH COMPACT STORAGE");
+
+        UUID id1 = UUID.fromString("550e8400-e29b-41d4-a716-446655440000");
+        UUID id2 = UUID.fromString("f47ac10b-58cc-4372-a567-0e02b2c3d479");
+        UUID id3 = UUID.fromString("810e8500-e29b-41d4-a716-446655440000");
+
+        execute("INSERT INTO %s (userid, url, time) VALUES (?, ?, ?)", id1, "http://foo.bar", 42L);
+        execute("INSERT INTO %s (userid, url, time) VALUES (?, ?, ?)", id1, "http://foo-2.bar", 24L);
+        execute("INSERT INTO %s (userid, url, time) VALUES (?, ?, ?)", id1, "http://bar.bar", 128L);
+        execute("UPDATE %s SET time = 24 WHERE userid = ? and url = 'http://bar.foo'", id2);
+        execute("UPDATE %s SET time = 12 WHERE userid IN (?, ?) and url = 'http://foo-3'", id2, id1);
+
+        assertRows(execute("SELECT url, time FROM %s WHERE userid = ?", id1),
+                   row("http://bar.bar", 128L),
+                   row("http://foo-2.bar", 24L),
+                   row("http://foo-3", 12L),
+                   row("http://foo.bar", 42L));
+
+        assertRows(execute("SELECT * FROM %s WHERE userid = ?", id2),
+                   row(id2, "http://bar.foo", 24L),
+                   row(id2, "http://foo-3", 12L));
+
+        assertRows(execute("SELECT time FROM %s"),
+                   row(24L), // id2
+                   row(12L),
+                   row(128L), // id1
+                   row(24L),
+                   row(12L),
+                   row(42L)
+        );
+
+        // Check we don't allow empty values for url since this is the full underlying cell name (#6152)
+        assertInvalid("INSERT INTO %s (userid, url, time) VALUES (?, '', 42)", id3);
+    }
+
+    /**
+     * Creation and basic operations on a composite table with compact storage,
+     * migrated from cql_tests.py:TestCQL.dense_cf_test()
+     */
+    @Test
+    public void testDenseCompositeTable() throws Throwable
+    {
+        createTable("CREATE TABLE %s (userid uuid, ip text, port int, time bigint, PRIMARY KEY (userid, ip, port)) WITH COMPACT STORAGE");
+
+        UUID id1 = UUID.fromString("550e8400-e29b-41d4-a716-446655440000");
+        UUID id2 = UUID.fromString("f47ac10b-58cc-4372-a567-0e02b2c3d479");
+
+        execute("INSERT INTO %s (userid, ip, port, time) VALUES (?, '192.168.0.1', 80, 42)", id1);
+        execute("INSERT INTO %s (userid, ip, port, time) VALUES (?, '192.168.0.2', 80, 24)", id1);
+        execute("INSERT INTO %s (userid, ip, port, time) VALUES (?, '192.168.0.2', 90, 42)", id1);
+        execute("UPDATE %s SET time = 24 WHERE userid = ? AND ip = '192.168.0.2' AND port = 80", id2);
+
+        // we don't have to include all of the clustering columns (see CASSANDRA-7990)
+        execute("INSERT INTO %s (userid, ip, time) VALUES (?, '192.168.0.3', 42)", id2);
+        execute("UPDATE %s SET time = 42 WHERE userid = ? AND ip = '192.168.0.4'", id2);
+
+        assertRows(execute("SELECT ip, port, time FROM %s WHERE userid = ?", id1),
+                   row("192.168.0.1", 80, 42L),
+                   row("192.168.0.2", 80, 24L),
+                   row("192.168.0.2", 90, 42L));
+
+        assertRows(execute("SELECT ip, port, time FROM %s WHERE userid = ? and ip >= '192.168.0.2'", id1),
+                   row("192.168.0.2", 80, 24L),
+                   row("192.168.0.2", 90, 42L));
+
+        assertRows(execute("SELECT ip, port, time FROM %s WHERE userid = ? and ip = '192.168.0.2'", id1),
+                   row("192.168.0.2", 80, 24L),
+                   row("192.168.0.2", 90, 42L));
+
+        assertEmpty(execute("SELECT ip, port, time FROM %s WHERE userid = ? and ip > '192.168.0.2'", id1));
+
+        assertRows(execute("SELECT ip, port, time FROM %s WHERE userid = ? AND ip = '192.168.0.3'", id2),
+                   row("192.168.0.3", null, 42L));
+
+        assertRows(execute("SELECT ip, port, time FROM %s WHERE userid = ? AND ip = '192.168.0.4'", id2),
+                   row("192.168.0.4", null, 42L));
+
+        execute("DELETE time FROM %s WHERE userid = ? AND ip = '192.168.0.2' AND port = 80", id1);
+
+        assertRowCount(execute("SELECT * FROM %s WHERE userid = ?", id1), 2);
+
+        execute("DELETE FROM %s WHERE userid = ?", id1);
+        assertEmpty(execute("SELECT * FROM %s WHERE userid = ?", id1));
+
+        execute("DELETE FROM %s WHERE userid = ? AND ip = '192.168.0.3'", id2);
+        assertEmpty(execute("SELECT * FROM %s WHERE userid = ? AND ip = '192.168.0.3'", id2));
+    }
+
+    /**
+     * Creation and basic operations on a composite table,
+     * migrated from cql_tests.py:TestCQL.sparse_cf_test()
+     */
+    @Test
+    public void testSparseCompositeTable() throws Throwable
+    {
+        createTable("CREATE TABLE %s (userid uuid, posted_month int, posted_day int, body text, posted_by text, PRIMARY KEY (userid, posted_month, posted_day))");
+
+        UUID id1 = UUID.fromString("550e8400-e29b-41d4-a716-446655440000");
+        UUID id2 = UUID.fromString("f47ac10b-58cc-4372-a567-0e02b2c3d479");
+
+        execute("INSERT INTO %s (userid, posted_month, posted_day, body, posted_by) VALUES (?, 1, 12, 'Something else', 'Frodo Baggins')", id1);
+        execute("INSERT INTO %s (userid, posted_month, posted_day, body, posted_by) VALUES (?, 1, 24, 'Something something', 'Frodo Baggins')", id1);
+        execute("UPDATE %s SET body = 'Yo Froddo', posted_by = 'Samwise Gamgee' WHERE userid = ? AND posted_month = 1 AND posted_day = 3", id2);
+        execute("UPDATE %s SET body = 'Yet one more message' WHERE userid = ? AND posted_month = 1 and posted_day = 30", id1);
+
+        assertRows(execute("SELECT body, posted_by FROM %s WHERE userid = ? AND posted_month = 1 AND posted_day = 24", id1),
+                   row("Something something", "Frodo Baggins"));
+
+        assertRows(execute("SELECT posted_day, body, posted_by FROM %s WHERE userid = ? AND posted_month = 1 AND posted_day > 12", id1),
+                   row(24, "Something something", "Frodo Baggins"),
+                   row(30, "Yet one more message", null));
+
+        assertRows(execute("SELECT posted_day, body, posted_by FROM %s WHERE userid = ? AND posted_month = 1", id1),
+                   row(12, "Something else", "Frodo Baggins"),
+                   row(24, "Something something", "Frodo Baggins"),
+                   row(30, "Yet one more message", null));
+    }
+
+    /**
+     * Check invalid create table statements,
+     * migrated from cql_tests.py:TestCQL.create_invalid_test()
+     */
+    @Test
+    public void testInvalidCreateTableStatements() throws Throwable
+    {
+        assertInvalidThrow(SyntaxException.class, "CREATE TABLE test ()");
+
+        assertInvalid("CREATE TABLE test (c1 text, c2 text, c3 text)");
+        assertInvalid("CREATE TABLE test (key1 text PRIMARY KEY, key2 text PRIMARY KEY)");
+
+        assertInvalid("CREATE TABLE test (key text PRIMARY KEY, key int)");
+        assertInvalid("CREATE TABLE test (key text PRIMARY KEY, c int, c text)");
+
+        assertInvalid("CREATE TABLE test (key text, key2 text, c int, d text, PRIMARY KEY (key, key2)) WITH COMPACT STORAGE");
+    }
+
+    /**
+     * Check obsolete properties from CQL2 are rejected
+     * migrated from cql_tests.py:TestCQL.invalid_old_property_test()
+     */
+    @Test
+    public void testObsoleteTableProperties() throws Throwable
+    {
+        assertInvalidThrow(SyntaxException.class, "CREATE TABLE test (foo text PRIMARY KEY, c int) WITH default_validation=timestamp");
+
+        createTable("CREATE TABLE %s (foo text PRIMARY KEY, c int)");
+        assertInvalidThrow(SyntaxException.class, "ALTER TABLE %s WITH default_validation=int");
+    }
+
+    /**
+     * Test create and drop keyspace
+     * migrated from cql_tests.py:TestCQL.keyspace_test()
+     */
+    @Test
+    public void testKeyspace() throws Throwable
+    {
+        assertInvalidThrow(SyntaxException.class, "CREATE KEYSPACE %s testXYZ ");
+
+        execute("CREATE KEYSPACE testXYZ WITH replication = { 'class' : 'SimpleStrategy', 'replication_factor' : 1 }");
+
+        assertInvalid(
+                     "CREATE KEYSPACE My_much_much_too_long_identifier_that_should_not_work WITH replication = { 'class' : 'SimpleStrategy', 'replication_factor' : 1 }");
+
+        execute("DROP KEYSPACE testXYZ");
+        assertInvalidThrow(ConfigurationException.class, "DROP KEYSPACE non_existing");
+
+        execute("CREATE KEYSPACE testXYZ WITH replication = { 'class' : 'SimpleStrategy', 'replication_factor' : 1 }");
+
+        // clean-up
+        execute("DROP KEYSPACE testXYZ");
+    }
+
+    /**
+     * Test create and drop table
+     * migrated from cql_tests.py:TestCQL.table_test()
+     */
+    @Test
+    public void testTable() throws Throwable
+    {
+        String table1 = createTable(" CREATE TABLE %s (k int PRIMARY KEY, c int)");
+        createTable(" CREATE TABLE %s (k int, name int, value int, PRIMARY KEY(k, name)) WITH COMPACT STORAGE ");
+        createTable(" CREATE TABLE %s (k int, c int, PRIMARY KEY (k),)");
+
+        String table4 = createTableName();
+
+        // repeated column
+        assertInvalidMessage("Multiple definition of identifier k", String.format("CREATE TABLE %s (k int PRIMARY KEY, c int, k text)", table4));
+
+        // compact storage limitations
+        assertInvalidThrow(SyntaxException.class,
+                           String.format("CREATE TABLE %s (k int, name, int, c1 int, c2 int, PRIMARY KEY(k, name)) WITH COMPACT STORAGE", table4));
+
+        execute(String.format("DROP TABLE %s.%s", keyspace(), table1));
+
+        createTable(String.format("CREATE TABLE %s.%s ( k int PRIMARY KEY, c1 int, c2 int, ) ", keyspace(), table1));
+    }
+
+    /**
+     * Test truncate statement,
+     * migrated from cql_tests.py:TestCQL.table_test().
+     */
+    @Test
+    public void testTruncate() throws Throwable
+    {
+        createTable(" CREATE TABLE %s (k int, name int, value int, PRIMARY KEY(k, name)) WITH COMPACT STORAGE ");
+        execute("TRUNCATE %s");
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.multiordering_validation_test()
+     */
+    @Test
+    public void testMultiOrderingValidation() throws Throwable
+    {
+        String tableName = KEYSPACE + "." + createTableName();
+        assertInvalid(String.format("CREATE TABLE test (k int, c1 int, c2 int, PRIMARY KEY (k, c1, c2)) WITH CLUSTERING ORDER BY (c2 DESC)", tableName));
+
+        tableName = KEYSPACE + "." + createTableName();
+        assertInvalid(String.format("CREATE TABLE test (k int, c1 int, c2 int, PRIMARY KEY (k, c1, c2)) WITH CLUSTERING ORDER BY (c2 ASC, c1 DESC)", tableName));
+
+        tableName = KEYSPACE + "." + createTableName();
+        assertInvalid(String.format("CREATE TABLE test (k int, c1 int, c2 int, PRIMARY KEY (k, c1, c2)) WITH CLUSTERING ORDER BY (c1 DESC, c2 DESC, c3 DESC)", tableName));
+
+        createTable("CREATE TABLE %s (k int, c1 int, c2 int, PRIMARY KEY (k, c1, c2)) WITH CLUSTERING ORDER BY (c1 DESC, c2 DESC)");
+        createTable("CREATE TABLE %s (k int, c1 int, c2 int, PRIMARY KEY (k, c1, c2)) WITH CLUSTERING ORDER BY (c1 ASC, c2 DESC)");
+    }
+
+    @Test
+    public void testCreateTrigger() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int, b int, c int, PRIMARY KEY (a))");
+        execute("CREATE TRIGGER trigger_1 ON %s USING '" + TestTrigger.class.getName() + "'");
+        assertTriggerExists("trigger_1", TestTrigger.class);
+        execute("CREATE TRIGGER trigger_2 ON %s USING '" + TestTrigger.class.getName() + "'");
+        assertTriggerExists("trigger_2", TestTrigger.class);
+        assertInvalid("CREATE TRIGGER trigger_1 ON %s USING '" + TestTrigger.class.getName() + "'");
+        execute("CREATE TRIGGER \"Trigger 3\" ON %s USING '" + TestTrigger.class.getName() + "'");
+        assertTriggerExists("Trigger 3", TestTrigger.class);
+    }
+
+    @Test
+    public void testCreateTriggerIfNotExists() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int, b int, c int, PRIMARY KEY (a, b))");
+
+        execute("CREATE TRIGGER IF NOT EXISTS trigger_1 ON %s USING '" + TestTrigger.class.getName() + "'");
+        assertTriggerExists("trigger_1", TestTrigger.class);
+
+        execute("CREATE TRIGGER IF NOT EXISTS trigger_1 ON %s USING '" + TestTrigger.class.getName() + "'");
+        assertTriggerExists("trigger_1", TestTrigger.class);
+    }
+
+    @Test
+    public void testDropTrigger() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int, b int, c int, PRIMARY KEY (a))");
+
+        execute("CREATE TRIGGER trigger_1 ON %s USING '" + TestTrigger.class.getName() + "'");
+        assertTriggerExists("trigger_1", TestTrigger.class);
+
+        execute("DROP TRIGGER trigger_1 ON %s");
+        assertTriggerDoesNotExists("trigger_1", TestTrigger.class);
+
+        execute("CREATE TRIGGER trigger_1 ON %s USING '" + TestTrigger.class.getName() + "'");
+        assertTriggerExists("trigger_1", TestTrigger.class);
+
+        assertInvalid("DROP TRIGGER trigger_2 ON %s");
+
+        execute("CREATE TRIGGER \"Trigger 3\" ON %s USING '" + TestTrigger.class.getName() + "'");
+        assertTriggerExists("Trigger 3", TestTrigger.class);
+
+        execute("DROP TRIGGER \"Trigger 3\" ON %s");
+        assertTriggerDoesNotExists("Trigger 3", TestTrigger.class);
+    }
+
+    @Test
+    public void testDropTriggerIfExists() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int, b int, c int, PRIMARY KEY (a))");
+
+        execute("DROP TRIGGER IF EXISTS trigger_1 ON %s");
+        assertTriggerDoesNotExists("trigger_1", TestTrigger.class);
+
+        execute("CREATE TRIGGER trigger_1 ON %s USING '" + TestTrigger.class.getName() + "'");
+        assertTriggerExists("trigger_1", TestTrigger.class);
+
+        execute("DROP TRIGGER IF EXISTS trigger_1 ON %s");
+        assertTriggerDoesNotExists("trigger_1", TestTrigger.class);
+    }
+
+    @Test
+    // tests CASSANDRA-9565
+    public void testDoubleWith() throws Throwable
+    {
+        String[] stmts = new String[] { "CREATE KEYSPACE WITH WITH DURABLE_WRITES = true",
+                                        "CREATE KEYSPACE ks WITH WITH DURABLE_WRITES = true" };
+
+        for (String stmt : stmts) {
+            assertInvalidSyntaxMessage("no viable alternative at input 'WITH'", stmt);
+        }
+    }
+
+    private void assertTriggerExists(String name, Class<?> clazz)
+    {
+        CFMetaData cfm = Schema.instance.getCFMetaData(keyspace(), currentTable()).copy();
+        assertTrue("the trigger does not exist", cfm.containsTriggerDefinition(TriggerDefinition.create(name,
+                                                                                                        clazz.getName())));
+    }
+
+    private void assertTriggerDoesNotExists(String name, Class<?> clazz)
+    {
+        CFMetaData cfm = Schema.instance.getCFMetaData(keyspace(), currentTable()).copy();
+        Assert.assertFalse("the trigger exists", cfm.containsTriggerDefinition(TriggerDefinition.create(name,
+                                                                                                        clazz.getName())));
+    }
+
+    public static class TestTrigger implements ITrigger
+    {
+        public TestTrigger() { }
+        public Collection<Mutation> augment(ByteBuffer key, ColumnFamily update)
+        {
+            return Collections.emptyList();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/01115f72/test/unit/org/apache/cassandra/cql3/validation/operations/DeleteTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/DeleteTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/DeleteTest.java
new file mode 100644
index 0000000..476ec83
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/DeleteTest.java
@@ -0,0 +1,329 @@
+/*
+ * 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.validation.operations;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+
+import org.junit.Test;
+
+import org.apache.cassandra.cql3.CQLTester;
+
+import static org.junit.Assert.assertEquals;
+
+public class DeleteTest extends CQLTester
+{
+    /** Test for cassandra 8558 */
+    @Test
+    public void testRangeDeletion() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int, b int, c int, d int, PRIMARY KEY (a, b, c))");
+
+        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 1, 1, 1, 1);
+        flush();
+        execute("DELETE FROM %s WHERE a=? AND b=?", 1, 1);
+        flush();
+        assertEmpty(execute("SELECT * FROM %s WHERE a=? AND b=? AND c=?", 1, 1, 1));
+    }
+
+    /**
+     * Test simple deletion and in particular check for #4193 bug
+     * migrated from cql_tests.py:TestCQL.deletion_test()
+     */
+    @Test
+    public void testDeletion() throws Throwable
+    {
+        createTable("CREATE TABLE %s (username varchar, id int, name varchar, stuff varchar, PRIMARY KEY(username, id))");
+
+        execute("INSERT INTO %s (username, id, name, stuff) VALUES (?, ?, ?, ?)", "abc", 2, "rst", "some value");
+        execute("INSERT INTO %s (username, id, name, stuff) VALUES (?, ?, ?, ?)", "abc", 4, "xyz", "some other value");
+
+        assertRows(execute("SELECT * FROM %s"),
+                   row("abc", 2, "rst", "some value"),
+                   row("abc", 4, "xyz", "some other value"));
+
+        execute("DELETE FROM %s WHERE username='abc' AND id=2");
+
+        assertRows(execute("SELECT * FROM %s"),
+                   row("abc", 4, "xyz", "some other value"));
+
+        createTable("CREATE TABLE %s (username varchar, id int, name varchar, stuff varchar, PRIMARY KEY(username, id, name)) WITH COMPACT STORAGE");
+
+        execute("INSERT INTO %s (username, id, name, stuff) VALUES (?, ?, ?, ?)", "abc", 2, "rst", "some value");
+        execute("INSERT INTO %s (username, id, name, stuff) VALUES (?, ?, ?, ?)", "abc", 4, "xyz", "some other value");
+
+        assertRows(execute("SELECT * FROM %s"),
+                   row("abc", 2, "rst", "some value"),
+                   row("abc", 4, "xyz", "some other value"));
+
+        execute("DELETE FROM %s WHERE username='abc' AND id=2");
+
+        assertRows(execute("SELECT * FROM %s"),
+                   row("abc", 4, "xyz", "some other value"));
+    }
+
+    /**
+     * Test deletion by 'composite prefix' (range tombstones)
+     * migrated from cql_tests.py:TestCQL.range_tombstones_test()
+     */
+    @Test
+    public void testDeleteByCompositePrefix() throws Throwable
+    { // This test used 3 nodes just to make sure RowMutation are correctly serialized
+
+        createTable("CREATE TABLE %s ( k int, c1 int, c2 int, v1 int, v2 int, PRIMARY KEY (k, c1, c2))");
+
+        int numRows = 5;
+        int col1 = 2;
+        int col2 = 2;
+        int cpr = col1 * col2;
+
+        for (int i = 0; i < numRows; i++)
+            for (int j = 0; j < col1; j++)
+                for (int k = 0; k < col2; k++)
+                {
+                    int n = (i * cpr) + (j * col2) + k;
+                    execute("INSERT INTO %s (k, c1, c2, v1, v2) VALUES (?, ?, ?, ?, ?)", i, j, k, n, n);
+                }
+
+        for (int i = 0; i < numRows; i++)
+        {
+            Object[][] rows = getRows(execute("SELECT v1, v2 FROM %s where k = ?", i));
+            for (int x = i * cpr; x < (i + 1) * cpr; x++)
+            {
+                assertEquals(x, rows[x - i * cpr][0]);
+                assertEquals(x, rows[x - i * cpr][1]);
+            }
+        }
+
+        for (int i = 0; i < numRows; i++)
+            execute("DELETE FROM %s WHERE k = ? AND c1 = 0", i);
+
+        for (int i = 0; i < numRows; i++)
+        {
+            Object[][] rows = getRows(execute("SELECT v1, v2 FROM %s WHERE k = ?", i));
+            for (int x = i * cpr + col1; x < (i + 1) * cpr; x++)
+            {
+                assertEquals(x, rows[x - i * cpr - col1][0]);
+                assertEquals(x, rows[x - i * cpr - col1][1]);
+            }
+        }
+
+        for (int i = 0; i < numRows; i++)
+        {
+            Object[][] rows = getRows(execute("SELECT v1, v2 FROM %s WHERE k = ?", i));
+            for (int x = i * cpr + col1; x < (i + 1) * cpr; x++)
+            {
+                assertEquals(x, rows[x - i * cpr - col1][0]);
+                assertEquals(x, rows[x - i * cpr - col1][1]);
+            }
+        }
+    }
+
+    /**
+     * Test deletion by 'composite prefix' (range tombstones) with compaction
+     * migrated from cql_tests.py:TestCQL.range_tombstones_compaction_test()
+     */
+    @Test
+    public void testDeleteByCompositePrefixWithCompaction() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int, c1 int, c2 int, v1 text, PRIMARY KEY (k, c1, c2))");
+
+        for (int c1 = 0; c1 < 4; c1++)
+            for (int c2 = 0; c2 < 2; c2++)
+                execute("INSERT INTO %s (k, c1, c2, v1) VALUES (0, ?, ?, ?)", c1, c2, String.format("%d%d", c1, c2));
+
+        flush();
+
+        execute("DELETE FROM %s WHERE k = 0 AND c1 = 1");
+
+        flush();
+        compact();
+
+        Object[][] rows = getRows(execute("SELECT v1 FROM %s WHERE k = 0"));
+
+        int idx = 0;
+        for (int c1 = 0; c1 < 4; c1++)
+            for (int c2 = 0; c2 < 2; c2++)
+                if (c1 != 1)
+                    assertEquals(String.format("%d%d", c1, c2), rows[idx++][0]);
+    }
+
+    /**
+     * Test deletion of rows
+     * migrated from cql_tests.py:TestCQL.delete_row_test()
+     */
+    @Test
+    public void testRowDeletion() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int, c1 int, c2 int, v1 int, v2 int, PRIMARY KEY (k, c1, c2))");
+
+        execute("INSERT INTO %s (k, c1, c2, v1, v2) VALUES (?, ?, ?, ?, ?)", 0, 0, 0, 0, 0);
+        execute("INSERT INTO %s (k, c1, c2, v1, v2) VALUES (?, ?, ?, ?, ?)", 0, 0, 1, 1, 1);
+        execute("INSERT INTO %s (k, c1, c2, v1, v2) VALUES (?, ?, ?, ?, ?)", 0, 0, 2, 2, 2);
+        execute("INSERT INTO %s (k, c1, c2, v1, v2) VALUES (?, ?, ?, ?, ?)", 0, 1, 0, 3, 3);
+
+        execute("DELETE FROM %s WHERE k = 0 AND c1 = 0 AND c2 = 0");
+
+        assertRowCount(execute("SELECT * FROM %s"), 3);
+    }
+
+    /**
+     * Check the semantic of CQL row existence (part of #4361),
+     * migrated from cql_tests.py:TestCQL.row_existence_test()
+     */
+    @Test
+    public void testRowExistence() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int, c int, v1 int, v2 int, PRIMARY KEY (k, c))");
+
+        execute("INSERT INTO %s (k, c, v1, v2) VALUES (1, 1, 1, 1)");
+        assertRows(execute("SELECT * FROM %s"),
+                   row(1, 1, 1, 1));
+
+        assertInvalid("DELETE c FROM %s WHERE k = 1 AND c = 1");
+
+        execute("DELETE v2 FROM %s WHERE k = 1 AND c = 1");
+        assertRows(execute("SELECT * FROM %s"),
+                   row(1, 1, 1, null));
+
+        execute("DELETE v1 FROM %s WHERE k = 1 AND c = 1");
+        assertRows(execute("SELECT * FROM %s"),
+                   row(1, 1, null, null));
+
+        execute("DELETE FROM %s WHERE k = 1 AND c = 1");
+        assertEmpty(execute("SELECT * FROM %s"));
+
+        execute("INSERT INTO %s (k, c) VALUES (2, 2)");
+        assertRows(execute("SELECT * FROM %s"),
+                   row(2, 2, null, null));
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.remove_range_slice_test()
+     */
+    @Test
+    public void testRemoveRangeSlice() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, v int)");
+
+        for (int i = 0; i < 3; i++)
+            execute("INSERT INTO %s (k, v) VALUES (?, ?)", i, i);
+
+        execute("DELETE FROM %s WHERE k = 1");
+        assertRows(execute("SELECT * FROM %s"),
+                   row(0, 0),
+                   row(2, 2));
+    }
+
+    /**
+     * Test deletions
+     * migrated from cql_tests.py:TestCQL.no_range_ghost_test()
+     */
+    @Test
+    public void testNoRangeGhost() throws Throwable
+    {
+        createTable("CREATE TABLE %s ( k int PRIMARY KEY, v int ) ");
+
+        for (int k = 0; k < 5; k++)
+            execute("INSERT INTO %s (k, v) VALUES (?, 0)", k);
+
+        Object[][] rows = getRows(execute("SELECT k FROM %s"));
+
+        int[] ordered = sortIntRows(rows);
+        for (int k = 0; k < 5; k++)
+            assertEquals(k, ordered[k]);
+
+        execute("DELETE FROM %s WHERE k=2");
+
+        rows = getRows(execute("SELECT k FROM %s"));
+        ordered = sortIntRows(rows);
+
+        int idx = 0;
+        for (int k = 0; k < 5; k++)
+            if (k != 2)
+                assertEquals(k, ordered[idx++]);
+
+        // Example from #3505
+        createTable("CREATE TABLE %s ( KEY varchar PRIMARY KEY, password varchar, gender varchar, birth_year bigint)");
+        execute("INSERT INTO %s (KEY, password) VALUES ('user1', 'ch@ngem3a')");
+        execute("UPDATE %s SET gender = 'm', birth_year = 1980 WHERE KEY = 'user1'");
+
+        assertRows(execute("SELECT * FROM %s WHERE KEY='user1'"),
+                   row("user1", 1980L, "m", "ch@ngem3a"));
+
+        execute("TRUNCATE %s");
+        assertEmpty(execute("SELECT * FROM %s"));
+
+        assertEmpty(execute("SELECT * FROM %s WHERE KEY='user1'"));
+    }
+
+    private int[] sortIntRows(Object[][] rows)
+    {
+        int[] ret = new int[rows.length];
+        for (int i = 0; i < ret.length; i++)
+            ret[i] = rows[i][0] == null ? Integer.MIN_VALUE : (Integer) rows[i][0];
+        Arrays.sort(ret);
+        return ret;
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.range_with_deletes_test()
+     */
+    @Test
+    public void testRandomDeletions() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, v int,)");
+
+        int nb_keys = 30;
+        int nb_deletes = 5;
+
+        List<Integer> deletions = new ArrayList<>(nb_keys);
+        for (int i = 0; i < nb_keys; i++)
+        {
+            execute("INSERT INTO %s (k, v) VALUES (?, ?)", i, i);
+            deletions.add(i);
+        }
+
+        Collections.shuffle(deletions);
+
+        for (int i = 0; i < nb_deletes; i++)
+            execute("DELETE FROM %s WHERE k = ?", deletions.get(i));
+
+        assertRowCount(execute("SELECT * FROM %s LIMIT ?", (nb_keys / 2)), nb_keys / 2);
+    }
+
+    /**
+     * Test for CASSANDRA-8558, deleted row still can be selected out
+     * migrated from cql_tests.py:TestCQL.bug_8558_test()
+     */
+    @Test
+    public void testDeletedRowCannotBeSelected() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int, b int, c text,primary key(a,b))");
+        execute("INSERT INTO %s (a,b,c) VALUES(1,1,'1')");
+        flush();
+
+        execute("DELETE FROM %s  where a=1 and b=1");
+        flush();
+
+        assertEmpty(execute("select * from %s  where a=1 and b=1"));
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/01115f72/test/unit/org/apache/cassandra/cql3/validation/operations/InsertTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/InsertTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/InsertTest.java
new file mode 100644
index 0000000..6e9d212
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/InsertTest.java
@@ -0,0 +1,59 @@
+/*
+ * 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.validation.operations;
+
+import org.junit.Test;
+
+import org.apache.cassandra.cql3.CQLTester;
+
+public class InsertTest extends CQLTester
+{
+    @Test
+    public void testInsertWithUnset() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, s text, i int)");
+
+        // insert using nulls
+        execute("INSERT INTO %s (k, s, i) VALUES (10, ?, ?)", "text", 10);
+        execute("INSERT INTO %s (k, s, i) VALUES (10, ?, ?)", null, null);
+        assertRows(execute("SELECT s, i FROM %s WHERE k = 10"),
+                   row(null, null) // sending null deletes the data
+        );
+        // insert using UNSET
+        execute("INSERT INTO %s (k, s, i) VALUES (11, ?, ?)", "text", 10);
+        execute("INSERT INTO %s (k, s, i) VALUES (11, ?, ?)", unset(), unset());
+        assertRows(execute("SELECT s, i FROM %s WHERE k=11"),
+                   row("text", 10) // unset columns does not delete the existing data
+        );
+
+        assertInvalidMessage("Invalid unset value for column k", "UPDATE %s SET i = 0 WHERE k = ?", unset());
+        assertInvalidMessage("Invalid unset value for column k", "DELETE FROM %s WHERE k = ?", unset());
+        assertInvalidMessage("Invalid unset value for argument in call to function blobasint", "SELECT * FROM %s WHERE k = blobAsInt(?)", unset());
+    }
+
+    @Test
+    public void testInsertTtlWithUnset() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, i int)");
+        execute("INSERT INTO %s (k, i) VALUES (1, 1) USING TTL ?", unset()); // treat as 'unlimited'
+        assertRows(execute("SELECT ttl(i) FROM %s"),
+                   row(new Object[]{ null })
+        );
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/01115f72/test/unit/org/apache/cassandra/cql3/validation/operations/InsertUpdateIfCondition.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/InsertUpdateIfCondition.java b/test/unit/org/apache/cassandra/cql3/validation/operations/InsertUpdateIfCondition.java
new file mode 100644
index 0000000..662fbf4
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/InsertUpdateIfCondition.java
@@ -0,0 +1,861 @@
+/*
+ * 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.validation.operations;
+
+import org.junit.Test;
+
+import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.exceptions.SyntaxException;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+public class InsertUpdateIfCondition extends CQLTester
+{
+    /**
+     * Migrated from cql_tests.py:TestCQL.cas_simple_test()
+     */
+    @Test
+    public void testSimpleCas() throws Throwable
+    {
+        createTable("CREATE TABLE %s (tkn int, consumed boolean, PRIMARY KEY (tkn))");
+
+        for (int i = 0; i < 10; i++)
+        {
+            execute("INSERT INTO %s (tkn, consumed) VALUES (?, FALSE)", i);
+
+            assertRows(execute("UPDATE %s SET consumed = TRUE WHERE tkn = ? IF consumed = FALSE", i), row(true));
+            assertRows(execute("UPDATE %s SET consumed = TRUE WHERE tkn = ? IF consumed = FALSE", i), row(false, true));
+        }
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.conditional_update_test()
+     */
+    @Test
+    public void testConditionalUpdate() throws Throwable
+    {
+        createTable(" CREATE TABLE %s (k int PRIMARY KEY, v1 int, v2 text, v3 int)");
+
+        // Shouldn't apply
+        assertRows(execute("UPDATE %s SET v1 = 3, v2 = 'bar' WHERE k = 0 IF v1 = 4"), row(false));
+        assertRows(execute("UPDATE %s SET v1 = 3, v2 = 'bar' WHERE k = 0 IF EXISTS"), row(false));
+
+        // Should apply
+        assertRows(execute("INSERT INTO %s (k, v1, v2) VALUES (0, 2, 'foo') IF NOT EXISTS"), row(true));
+
+        // Shouldn't apply
+        assertRows(execute("INSERT INTO %s (k, v1, v2) VALUES (0, 5, 'bar') IF NOT EXISTS"), row(false, 0, 2, "foo", null));
+        assertRows(execute("SELECT * FROM %s"), row(0, 2, "foo", null));
+
+        // Shouldn't apply
+        assertRows(execute("UPDATE %s SET v1 = 3, v2 = 'bar' WHERE k = 0 IF v1 = 4"), row(false, 2));
+        assertRows(execute("SELECT * FROM %s"), row(0, 2, "foo", null));
+
+        // Should apply (note: we want v2 before v1 in the statement order to exercise #5786)
+        assertRows(execute("UPDATE %s SET v2 = 'bar', v1 = 3 WHERE k = 0 IF v1 = 2"), row(true));
+        assertRows(execute("UPDATE %s SET v2 = 'bar', v1 = 3 WHERE k = 0 IF EXISTS"), row(true));
+        assertRows(execute("SELECT * FROM %s"), row(0, 3, "bar", null));
+
+        // Shouldn't apply, only one condition is ok
+        assertRows(execute("UPDATE %s SET v1 = 5, v2 = 'foobar' WHERE k = 0 IF v1 = 3 AND v2 = 'foo'"), row(false, 3, "bar"));
+        assertRows(execute("SELECT * FROM %s"), row(0, 3, "bar", null));
+
+        // Should apply
+        assertRows(execute("UPDATE %s SET v1 = 5, v2 = 'foobar' WHERE k = 0 IF v1 = 3 AND v2 = 'bar'"), row(true));
+        assertRows(execute("SELECT * FROM %s"), row(0, 5, "foobar", null));
+
+        // Shouldn't apply
+        assertRows(execute("DELETE v2 FROM %s WHERE k = 0 IF v1 = 3"), row(false, 5));
+        assertRows(execute("SELECT * FROM %s"), row(0, 5, "foobar", null));
+
+        // Shouldn't apply
+        assertRows(execute("DELETE v2 FROM %s WHERE k = 0 IF v1 = null"), row(false, 5));
+        assertRows(execute("SELECT * FROM %s"), row(0, 5, "foobar", null));
+
+        // Should apply
+        assertRows(execute("DELETE v2 FROM %s WHERE k = 0 IF v1 = 5"), row(true));
+        assertRows(execute("SELECT * FROM %s"), row(0, 5, null, null));
+
+        // Shouln't apply
+        assertRows(execute("DELETE v1 FROM %s WHERE k = 0 IF v3 = 4"), row(false, null));
+
+        // Should apply
+        assertRows(execute("DELETE v1 FROM %s WHERE k = 0 IF v3 = null"), row(true));
+        assertRows(execute("SELECT * FROM %s"), row(0, null, null, null));
+
+        // Should apply
+        assertRows(execute("DELETE FROM %s WHERE k = 0 IF v1 = null"), row(true));
+        assertEmpty(execute("SELECT * FROM %s"));
+
+        // Shouldn't apply
+        assertRows(execute("UPDATE %s SET v1 = 3, v2 = 'bar' WHERE k = 0 IF EXISTS"), row(false));
+
+        // Should apply
+        assertRows(execute("DELETE FROM %s WHERE k = 0 IF v1 IN (null)"), row(true));
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.non_eq_conditional_update_test()
+     */
+    @Test
+    public void testNonEqConditionalUpdate() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, v1 int, v2 text, v3 int)");
+
+        // non-EQ conditions
+        execute("INSERT INTO %s (k, v1, v2) VALUES (0, 2, 'foo')");
+
+        assertRows(execute("UPDATE %s SET v2 = 'bar' WHERE k = 0 IF v1 < 3"), row(true));
+        assertRows(execute("UPDATE %s SET v2 = 'bar' WHERE k = 0 IF v1 <= 3"), row(true));
+        assertRows(execute("UPDATE %s SET v2 = 'bar' WHERE k = 0 IF v1 > 1"), row(true));
+        assertRows(execute("UPDATE %s SET v2 = 'bar' WHERE k = 0 IF v1 >= 1"), row(true));
+        assertRows(execute("UPDATE %s SET v2 = 'bar' WHERE k = 0 IF v1 != 1"), row(true));
+        assertRows(execute("UPDATE %s SET v2 = 'bar' WHERE k = 0 IF v1 != 2"), row(false, 2));
+        assertRows(execute("UPDATE %s SET v2 = 'bar' WHERE k = 0 IF v1 IN (0, 1, 2)"), row(true));
+        assertRows(execute("UPDATE %s SET v2 = 'bar' WHERE k = 0 IF v1 IN (142, 276)"), row(false, 2));
+        assertRows(execute("UPDATE %s SET v2 = 'bar' WHERE k = 0 IF v1 IN ()"), row(false, 2));
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.conditional_delete_test()
+     */
+    @Test
+    public void testConditionalDelete() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, v1 int,)");
+
+        assertRows(execute("DELETE FROM %s WHERE k=1 IF EXISTS"), row(false));
+
+        execute("INSERT INTO %s (k, v1) VALUES (1, 2)");
+        assertRows(execute("DELETE FROM %s WHERE k=1 IF EXISTS"), row(true));
+        assertEmpty(execute("SELECT * FROM %s WHERE k=1"));
+        assertRows(execute("DELETE FROM %s WHERE k=1 IF EXISTS"), row(false));
+
+        execute("UPDATE %s USING TTL 1 SET v1=2 WHERE k=1");
+        Thread.sleep(1001);
+        assertRows(execute("DELETE FROM %s WHERE k=1 IF EXISTS"), row(false));
+        assertEmpty(execute("SELECT * FROM %s WHERE k=1"));
+
+        execute("INSERT INTO %s (k, v1) VALUES (2, 2) USING TTL 1");
+        Thread.sleep(1001);
+        assertRows(execute("DELETE FROM %s WHERE k=2 IF EXISTS"), row(false));
+        assertEmpty(execute("SELECT * FROM %s WHERE k=2"));
+
+        execute("INSERT INTO %s (k, v1) VALUES (3, 2)");
+        assertRows(execute("DELETE v1 FROM %s WHERE k=3 IF EXISTS"), row(true));
+        assertRows(execute("SELECT * FROM %s WHERE k=3"), row(3, null));
+        assertRows(execute("DELETE v1 FROM %s WHERE k=3 IF EXISTS"), row(true));
+        assertRows(execute("DELETE FROM %s WHERE k=3 IF EXISTS"), row(true));
+
+        // static columns
+        createTable("CREATE TABLE %s (k text, s text static, i int, v text, PRIMARY KEY (k, i) )");
+
+        execute("INSERT INTO %s (k, s, i, v) VALUES ('k', 's', 0, 'v')");
+        assertRows(execute("DELETE v FROM %s WHERE k='k' AND i=0 IF EXISTS"), row(true));
+        assertRows(execute("DELETE FROM %s WHERE k='k' AND i=0 IF EXISTS"), row(true));
+        assertRows(execute("DELETE v FROM %s WHERE k='k' AND i=0 IF EXISTS"), row(false));
+        assertRows(execute("DELETE FROM %s WHERE k='k' AND i=0 IF EXISTS"), row(false));
+
+        // CASSANDRA-6430
+        assertInvalid("DELETE FROM %s WHERE k = 'k' IF EXISTS");
+        assertInvalid("DELETE FROM %s WHERE k = 'k' IF v = 'foo'");
+        assertInvalid("DELETE FROM %s WHERE i = 0 IF EXISTS");
+        assertInvalid("DELETE FROM %s WHERE k = 0 AND i > 0 IF EXISTS");
+        assertInvalid("DELETE FROM %s WHERE k = 0 AND i > 0 IF v = 'foo'");
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.static_columns_cas_test()
+     */
+    @Test
+    public void testStaticColumnsCas() throws Throwable
+    {
+        createTable("CREATE TABLE %s (id int, k text, version int static, v text, PRIMARY KEY (id, k))");
+
+        // Test that INSERT IF NOT EXISTS concerns only the static column if no clustering nor regular columns
+        // is provided, but concerns the CQL3 row targetted by the clustering columns otherwise
+        execute("INSERT INTO %s (id, k, v) VALUES (1, 'foo', 'foo')");
+        assertRows(execute("INSERT INTO %s (id, k, version) VALUES (1, 'foo', 1) IF NOT EXISTS"), row(false, 1, "foo", null, "foo"));
+        assertRows(execute("INSERT INTO %s (id, version) VALUES (1, 1) IF NOT EXISTS"), row(true));
+        assertRows(execute("SELECT * FROM %s"), row(1, "foo", 1, "foo"));
+        execute("DELETE FROM %s WHERE id = 1");
+
+        execute("INSERT INTO %s(id, version) VALUES (0, 0)");
+
+        assertRows(execute("UPDATE %s SET v='foo', version=1 WHERE id=0 AND k='k1' IF version = 0"), row(true));
+        assertRows(execute("SELECT * FROM %s"), row(0, "k1", 1, "foo"));
+
+        assertRows(execute("UPDATE %s SET v='bar', version=1 WHERE id=0 AND k='k2' IF version = 0"), row(false, 1));
+        assertRows(execute("SELECT * FROM %s"), row(0, "k1", 1, "foo"));
+
+        assertRows(execute("UPDATE %s SET v='bar', version=2 WHERE id=0 AND k='k2' IF version = 1"), row(true));
+        assertRows(execute("SELECT * FROM %s"), row(0, "k1", 2, "foo"), row(0, "k2", 2, "bar"));
+
+        // Testing batches
+        assertRows(execute("BEGIN BATCH " +
+                           "UPDATE %1$s SET v='foobar' WHERE id=0 AND k='k1'; " +
+                           "UPDATE %1$s SET v='barfoo' WHERE id=0 AND k='k2'; " +
+                           "UPDATE %1$s SET version=3 WHERE id=0 IF version=1; " +
+                           "APPLY BATCH "),
+                   row(false, 0, null, 2));
+
+        assertRows(execute("BEGIN BATCH " +
+                           "UPDATE %1$s SET v = 'foobar' WHERE id = 0 AND k = 'k1'; " +
+                           "UPDATE %1$s SET v = 'barfoo' WHERE id = 0 AND k = 'k2'; " +
+                           "UPDATE %1$s SET version = 3 WHERE id = 0 IF version = 2; " +
+                           "APPLY BATCH "),
+                   row(true));
+
+        assertRows(execute("SELECT * FROM %s"),
+                   row(0, "k1", 3, "foobar"),
+                   row(0, "k2", 3, "barfoo"));
+
+        assertRows(execute("BEGIN BATCH " +
+                           "UPDATE %1$s SET version = 4 WHERE id = 0 IF version = 3; " +
+                           "UPDATE %1$s SET v='row1' WHERE id=0 AND k='k1' IF v='foo'; " +
+                           "UPDATE %1$s SET v='row2' WHERE id=0 AND k='k2' IF v='bar'; " +
+                           "APPLY BATCH "),
+                   row(false, 0, "k1", 3, "foobar"),
+                   row(false, 0, "k2", 3, "barfoo"));
+
+        assertRows(execute("BEGIN BATCH " +
+                           "UPDATE %1$s SET version = 4 WHERE id = 0 IF version = 3; " +
+                           "UPDATE %1$s SET v='row1' WHERE id = 0 AND k='k1' IF v='foobar'; " +
+                           "UPDATE %1$s SET v='row2' WHERE id = 0 AND k='k2' IF v='barfoo'; " +
+                           "APPLY BATCH "),
+                   row(true));
+
+        assertRows(execute("SELECT * FROM %s"),
+                   row(0, "k1", 4, "row1"),
+                   row(0, "k2", 4, "row2"));
+
+        assertInvalid("BEGIN BATCH " +
+                      "UPDATE %1$s SET version=5 WHERE id=0 IF version=4; " +
+                      "UPDATE %1$s SET v='row1' WHERE id=0 AND k='k1'; " +
+                      "UPDATE %1$s SET v='row2' WHERE id=1 AND k='k2'; " +
+                      "APPLY BATCH ");
+
+        assertRows(execute("BEGIN BATCH " +
+                           "INSERT INTO %1$s (id, k, v) VALUES (1, 'k1', 'val1') IF NOT EXISTS; " +
+                           "INSERT INTO %1$s (id, k, v) VALUES (1, 'k2', 'val2') IF NOT EXISTS; " +
+                           "APPLY BATCH "),
+                   row(true));
+
+        assertRows(execute("SELECT * FROM %s WHERE id=1"),
+                   row(1, "k1", null, "val1"),
+                   row(1, "k2", null, "val2"));
+
+        assertRows(execute("INSERT INTO %s (id, k, v) VALUES (1, 'k2', 'val2') IF NOT EXISTS"), row(false, 1, "k2", null, "val2"));
+
+        assertRows(execute("BEGIN BATCH " +
+                           "INSERT INTO %1$s (id, k, v) VALUES (1, 'k2', 'val2') IF NOT EXISTS; " +
+                           "INSERT INTO %1$s (id, k, v) VALUES (1, 'k3', 'val3') IF NOT EXISTS; " +
+                           "APPLY BATCH"),
+                   row(false, 1, "k2", null, "val2"));
+
+        assertRows(execute("BEGIN BATCH " +
+                           "UPDATE %1$s SET v = 'newVal' WHERE id = 1 AND k = 'k2' IF v = 'val0'; " +
+                           "INSERT INTO %1$s (id, k, v) VALUES (1, 'k3', 'val3') IF NOT EXISTS; " +
+                           "APPLY BATCH"),
+                   row(false, 1, "k2", null, "val2"));
+
+        assertRows(execute("SELECT * FROM %s WHERE id=1"),
+                   row(1, "k1", null, "val1"),
+                   row(1, "k2", null, "val2"));
+
+        assertRows(execute("BEGIN BATCH " +
+                           "UPDATE %1$s SET v = 'newVal' WHERE id = 1 AND k = 'k2' IF v = 'val2'; " +
+                           "INSERT INTO %1$s (id, k, v, version) VALUES(1, 'k3', 'val3', 1) IF NOT EXISTS; " +
+                           "APPLY BATCH"),
+                   row(true));
+
+        assertRows(execute("SELECT * FROM %s WHERE id=1"),
+                   row(1, "k1", 1, "val1"),
+                   row(1, "k2", 1, "newVal"),
+                   row(1, "k3", 1, "val3"));
+
+        assertRows(execute("BEGIN BATCH " +
+                           "UPDATE %1$s SET v = 'newVal1' WHERE id = 1 AND k = 'k2' IF v = 'val2'; " +
+                           "UPDATE %1$s SET v = 'newVal2' WHERE id = 1 AND k = 'k2' IF v = 'val3'; " +
+                           "APPLY BATCH"),
+                   row(false, 1, "k2", "newVal"));
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.bug_6069_test()
+     */
+    @Test
+    public void testInsertSetIfNotExists() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, s set<int>)");
+
+        assertRows(execute("INSERT INTO %s (k, s) VALUES (0, {1, 2, 3}) IF NOT EXISTS"),
+                   row(true));
+        assertRows(execute("SELECT * FROM %s "), row(0, set(1, 2, 3)));
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.cas_and_ttl_test()
+     */
+    @Test
+    public void testCasAndTTL() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, v int, lock boolean)");
+
+        execute("INSERT INTO %s (k, v, lock) VALUES (0, 0, false)");
+        execute("UPDATE %s USING TTL 1 SET lock=true WHERE k=0");
+
+        Thread.sleep(1001);
+        assertRows(execute("UPDATE %s SET v = 1 WHERE k = 0 IF lock = null"),
+                   row(true));
+    }
+
+    /**
+     * Test for CAS with compact storage table, and #6813 in particular,
+     * migrated from cql_tests.py:TestCQL.cas_and_compact_test()
+     */
+    @Test
+    public void testCompactStorage() throws Throwable
+    {
+        createTable("CREATE TABLE %s (partition text, key text, owner text, PRIMARY KEY (partition, key) ) WITH COMPACT STORAGE");
+
+        execute("INSERT INTO %s (partition, key, owner) VALUES ('a', 'b', null)");
+        assertRows(execute("UPDATE %s SET owner='z' WHERE partition='a' AND key='b' IF owner=null"), row(true));
+
+        assertRows(execute("UPDATE %s SET owner='b' WHERE partition='a' AND key='b' IF owner='a'"), row(false, "z"));
+        assertRows(execute("UPDATE %s SET owner='b' WHERE partition='a' AND key='b' IF owner='z'"), row(true));
+
+        assertRows(execute("INSERT INTO %s (partition, key, owner) VALUES ('a', 'c', 'x') IF NOT EXISTS"), row(true));
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.whole_list_conditional_test()
+     */
+    @Test
+    public void testWholeList() throws Throwable
+    {
+        for (boolean frozen : new boolean[] {false, true})
+        {
+            createTable(String.format("CREATE TABLE %%s (k int PRIMARY KEY, l %s)",
+                                      frozen
+                                      ? "frozen<list<text>>"
+                                      : "list<text>"));
+
+            execute("INSERT INTO %s(k, l) VALUES (0, ['foo', 'bar', 'foobar'])");
+
+            check_applies_list("l = ['foo', 'bar', 'foobar']");
+            check_applies_list("l != ['baz']");
+            check_applies_list("l > ['a']");
+            check_applies_list("l >= ['a']");
+            check_applies_list("l < ['z']");
+            check_applies_list("l <= ['z']");
+            check_applies_list("l IN (null, ['foo', 'bar', 'foobar'], ['a'])");
+
+            // multiple conditions
+            check_applies_list("l > ['aaa', 'bbb'] AND l > ['aaa']");
+            check_applies_list("l != null AND l IN (['foo', 'bar', 'foobar'])");
+
+            // should not apply
+            check_does_not_apply_list("l = ['baz']");
+            check_does_not_apply_list("l != ['foo', 'bar', 'foobar']");
+            check_does_not_apply_list("l > ['z']");
+            check_does_not_apply_list("l >= ['z']");
+            check_does_not_apply_list("l < ['a']");
+            check_does_not_apply_list("l <= ['a']");
+            check_does_not_apply_list("l IN (['a'], null)");
+            check_does_not_apply_list("l IN ()");
+
+            // multiple conditions
+            check_does_not_apply_list("l IN () AND l IN (['foo', 'bar', 'foobar'])");
+            check_does_not_apply_list("l > ['zzz'] AND l < ['zzz']");
+
+            check_invalid_list("l = [null]", InvalidRequestException.class);
+            check_invalid_list("l < null", InvalidRequestException.class);
+            check_invalid_list("l <= null", InvalidRequestException.class);
+            check_invalid_list("l > null", InvalidRequestException.class);
+            check_invalid_list("l >= null", InvalidRequestException.class);
+            check_invalid_list("l IN null", SyntaxException.class);
+            check_invalid_list("l IN 367", SyntaxException.class);
+            check_invalid_list("l CONTAINS KEY 123", SyntaxException.class);
+
+            // not supported yet
+            check_invalid_list("m CONTAINS 'bar'", SyntaxException.class);
+        }
+    }
+
+    void check_applies_list(String condition) throws Throwable
+    {
+        assertRows(execute("UPDATE %s SET l = ['foo', 'bar', 'foobar'] WHERE k=0 IF " + condition), row(true));
+        assertRows(execute("SELECT * FROM %s"), row(0, list("foo", "bar", "foobar")));
+    }
+
+    void check_does_not_apply_list(String condition) throws Throwable
+    {
+        assertRows(execute("UPDATE %s SET l = ['foo', 'bar', 'foobar'] WHERE k=0 IF " + condition),
+                   row(false, list("foo", "bar", "foobar")));
+        assertRows(execute("SELECT * FROM %s"), row(0, list("foo", "bar", "foobar")));
+    }
+
+    void check_invalid_list(String condition, Class<? extends Throwable> expected) throws Throwable
+    {
+        assertInvalidThrow(expected, "UPDATE %s SET l = ['foo', 'bar', 'foobar'] WHERE k=0 IF " + condition);
+        assertRows(execute("SELECT * FROM %s"), row(0, list("foo", "bar", "foobar")));
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.list_item_conditional_test()
+     */
+    @Test
+    public void testListItem() throws Throwable
+    {
+        for (boolean frozen : new boolean[]{ false, true })
+        {
+            createTable(String.format("CREATE TABLE %%s (k int PRIMARY KEY, l %s)",
+                                      frozen
+                                      ? "frozen<list<text>>"
+                                      : "list<text>"));
+
+            execute("INSERT INTO %s(k, l) VALUES (0, ['foo', 'bar', 'foobar'])");
+
+            assertInvalid("DELETE FROM %s WHERE k=0 IF l[null] = 'foobar'");
+            assertInvalid("DELETE FROM %s WHERE k=0 IF l[-2] = 'foobar'");
+
+            assertRows(execute("DELETE FROM %s WHERE k=0 IF l[1] = null"), row(false, list("foo", "bar", "foobar")));
+            assertRows(execute("DELETE FROM %s WHERE k=0 IF l[1] = 'foobar'"), row(false, list("foo", "bar", "foobar")));
+            assertRows(execute("SELECT * FROM %s"), row(0, list("foo", "bar", "foobar")));
+
+            assertRows(execute("DELETE FROM %s WHERE k=0 IF l[1] = 'bar'"), row(true));
+            assertEmpty(execute("SELECT * FROM %s"));
+        }
+    }
+
+    /**
+     * Test expanded functionality from CASSANDRA-6839, 
+     * migrated from cql_tests.py:TestCQL.expanded_list_item_conditional_test()
+     */
+    @Test
+    public void testExpandedListItem() throws Throwable
+    {
+        for (boolean frozen : new boolean[] {false, true})
+        {
+            createTable(String.format("CREATE TABLE %%s (k int PRIMARY KEY, l %s)",
+                                      frozen
+                                      ? "frozen<list<text>>"
+                                      : "list<text>"));
+
+            execute("INSERT INTO %s (k, l) VALUES (0, ['foo', 'bar', 'foobar'])");
+
+            check_applies_list("l[1] < 'zzz'");
+            check_applies_list("l[1] <= 'bar'");
+            check_applies_list("l[1] > 'aaa'");
+            check_applies_list("l[1] >= 'bar'");
+            check_applies_list("l[1] != 'xxx'");
+            check_applies_list("l[1] != null");
+            check_applies_list("l[1] IN (null, 'xxx', 'bar')");
+            check_applies_list("l[1] > 'aaa' AND l[1] < 'zzz'");
+
+            // check beyond end of list
+            check_applies_list("l[3] = null");
+            check_applies_list("l[3] IN (null, 'xxx', 'bar')");
+
+            check_does_not_apply_list("l[1] < 'aaa'");
+            check_does_not_apply_list("l[1] <= 'aaa'");
+            check_does_not_apply_list("l[1] > 'zzz'");
+            check_does_not_apply_list("l[1] >= 'zzz'");
+            check_does_not_apply_list("l[1] != 'bar'");
+            check_does_not_apply_list("l[1] IN (null, 'xxx')");
+            check_does_not_apply_list("l[1] IN ()");
+            check_does_not_apply_list("l[1] != null AND l[1] IN ()");
+
+            // check beyond end of list
+            check_does_not_apply_list("l[3] != null");
+            check_does_not_apply_list("l[3] = 'xxx'");
+
+            check_invalid_list("l[1] < null", InvalidRequestException.class);
+            check_invalid_list("l[1] <= null", InvalidRequestException.class);
+            check_invalid_list("l[1] > null", InvalidRequestException.class);
+            check_invalid_list("l[1] >= null", InvalidRequestException.class);
+            check_invalid_list("l[1] IN null", SyntaxException.class);
+            check_invalid_list("l[1] IN 367", SyntaxException.class);
+            check_invalid_list("l[1] IN (1, 2, 3)", InvalidRequestException.class);
+            check_invalid_list("l[1] CONTAINS 367", SyntaxException.class);
+            check_invalid_list("l[1] CONTAINS KEY 367", SyntaxException.class);
+            check_invalid_list("l[null] = null", InvalidRequestException.class);
+        }
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.whole_set_conditional_test()
+     */
+    @Test
+    public void testWholeSet() throws Throwable
+    {
+        for (boolean frozen : new boolean[] {false, true})
+        {
+            createTable(String.format("CREATE TABLE %%s (k int PRIMARY KEY, s %s)",
+                                      frozen
+                                      ? "frozen<set<text>>"
+                                      : "set<text>"));
+
+            execute("INSERT INTO %s (k, s) VALUES (0, {'bar', 'foo'})");
+
+            check_applies_set("s = {'bar', 'foo'}");
+            check_applies_set("s = {'foo', 'bar'}");
+            check_applies_set("s != {'baz'}");
+            check_applies_set("s > {'a'}");
+            check_applies_set("s >= {'a'}");
+            check_applies_set("s < {'z'}");
+            check_applies_set("s <= {'z'}");
+            check_applies_set("s IN (null, {'bar', 'foo'}, {'a'})");
+
+            // multiple conditions
+            check_applies_set("s > {'a'} AND s < {'z'}");
+            check_applies_set("s IN (null, {'bar', 'foo'}, {'a'}) AND s IN ({'a'}, {'bar', 'foo'}, null)");
+
+            // should not apply
+            check_does_not_apply_set("s = {'baz'}");
+            check_does_not_apply_set("s != {'bar', 'foo'}");
+            check_does_not_apply_set("s > {'z'}");
+            check_does_not_apply_set("s >= {'z'}");
+            check_does_not_apply_set("s < {'a'}");
+            check_does_not_apply_set("s <= {'a'}");
+            check_does_not_apply_set("s IN ({'a'}, null)");
+            check_does_not_apply_set("s IN ()");
+            check_does_not_apply_set("s != null AND s IN ()");
+
+            check_invalid_set("s = {null}", InvalidRequestException.class);
+            check_invalid_set("s < null", InvalidRequestException.class);
+            check_invalid_set("s <= null", InvalidRequestException.class);
+            check_invalid_set("s > null", InvalidRequestException.class);
+            check_invalid_set("s >= null", InvalidRequestException.class);
+            check_invalid_set("s IN null", SyntaxException.class);
+            check_invalid_set("s IN 367", SyntaxException.class);
+            check_invalid_set("s CONTAINS KEY 123", SyntaxException.class);
+
+            // element access is not allow for sets
+            check_invalid_set("s['foo'] = 'foobar'", InvalidRequestException.class);
+
+            // not supported yet
+            check_invalid_set("m CONTAINS 'bar'", SyntaxException.class);
+        }
+    }
+
+    void check_applies_set(String condition) throws Throwable
+    {
+        assertRows(execute("UPDATE %s SET s = {'bar', 'foo'} WHERE k=0 IF " + condition), row(true));
+        assertRows(execute("SELECT * FROM %s"), row(0, set("bar", "foo")));
+    }
+
+    void check_does_not_apply_set(String condition) throws Throwable
+    {
+        assertRows(execute("UPDATE %s SET s = {'bar', 'foo'} WHERE k=0 IF " + condition), row(false, set("bar", "foo")));
+        assertRows(execute("SELECT * FROM %s"), row(0, set("bar", "foo")));
+    }
+
+    void check_invalid_set(String condition, Class<? extends Throwable> expected) throws Throwable
+    {
+        assertInvalidThrow(expected, "UPDATE %s SET s = {'bar', 'foo'} WHERE k=0 IF " + condition);
+        assertRows(execute("SELECT * FROM %s"), row(0, set("bar", "foo")));
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.whole_map_conditional_test()
+     */
+    @Test
+    public void testWholeMap() throws Throwable
+    {
+        for (boolean frozen : new boolean[] {false, true})
+        {
+            createTable(String.format("CREATE TABLE %%s (k int PRIMARY KEY, m %s)",
+                                      frozen
+                                      ? "frozen<map<text, text>>"
+                                      : "map<text, text>"));
+
+            execute("INSERT INTO %s (k, m) VALUES (0, {'foo' : 'bar'})");
+
+            check_applies_map("m = {'foo': 'bar'}");
+            check_applies_map("m > {'a': 'a'}");
+            check_applies_map("m >= {'a': 'a'}");
+            check_applies_map("m < {'z': 'z'}");
+            check_applies_map("m <= {'z': 'z'}");
+            check_applies_map("m != {'a': 'a'}");
+            check_applies_map("m IN (null, {'a': 'a'}, {'foo': 'bar'})");
+
+            // multiple conditions
+            check_applies_map("m > {'a': 'a'} AND m < {'z': 'z'}");
+            check_applies_map("m != null AND m IN (null, {'a': 'a'}, {'foo': 'bar'})");
+
+            // should not apply
+            check_does_not_apply_map("m = {'a': 'a'}");
+            check_does_not_apply_map("m > {'z': 'z'}");
+            check_does_not_apply_map("m >= {'z': 'z'}");
+            check_does_not_apply_map("m < {'a': 'a'}");
+            check_does_not_apply_map("m <= {'a': 'a'}");
+            check_does_not_apply_map("m != {'foo': 'bar'}");
+            check_does_not_apply_map("m IN ({'a': 'a'}, null)");
+            check_does_not_apply_map("m IN ()");
+            check_does_not_apply_map("m = null AND m != null");
+
+            check_invalid_map("m = {null: null}", InvalidRequestException.class);
+            check_invalid_map("m = {'a': null}", InvalidRequestException.class);
+            check_invalid_map("m = {null: 'a'}", InvalidRequestException.class);
+            check_invalid_map("m < null", InvalidRequestException.class);
+            check_invalid_map("m IN null", SyntaxException.class);
+
+            // not supported yet
+            check_invalid_map("m CONTAINS 'bar'", SyntaxException.class);
+            check_invalid_map("m CONTAINS KEY 'foo'", SyntaxException.class);
+            check_invalid_map("m CONTAINS null", SyntaxException.class);
+            check_invalid_map("m CONTAINS KEY null", SyntaxException.class);
+        }
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.map_item_conditional_test()
+     */
+    @Test
+    public void testMapItem() throws Throwable
+    {
+        for (boolean frozen : new boolean[]{ false, true })
+        {
+            createTable(String.format("CREATE TABLE %%s (k int PRIMARY KEY, m %s)",
+                                      frozen
+                                      ? "frozen<map<text, text>>"
+                                      : "map<text, text>"));
+
+            execute("INSERT INTO %s (k, m) VALUES (0, {'foo' : 'bar'})");
+            assertInvalid("DELETE FROM %s WHERE k=0 IF m[null] = 'foo'");
+            assertRows(execute("DELETE FROM %s WHERE k=0 IF m['foo'] = 'foo'"), row(false, map("foo", "bar")));
+            assertRows(execute("DELETE FROM %s WHERE k=0 IF m['foo'] = null"), row(false, map("foo", "bar")));
+            assertRows(execute("SELECT * FROM %s"), row(0, map("foo", "bar")));
+
+            assertRows(execute("DELETE FROM %s WHERE k=0 IF m['foo'] = 'bar'"), row(true));
+            assertEmpty(execute("SELECT * FROM %s"));
+
+            execute("INSERT INTO %s(k, m) VALUES (1, null)");
+            if (frozen)
+                assertInvalid("UPDATE %s set m['foo'] = 'bar', m['bar'] = 'foo' WHERE k = 1 IF m['foo'] IN ('blah', null)");
+            else
+                assertRows(execute("UPDATE %s set m['foo'] = 'bar', m['bar'] = 'foo' WHERE k = 1 IF m['foo'] IN ('blah', null)"), row(true));
+        }
+    }
+
+    /**
+     * Test expanded functionality from CASSANDRA-6839,
+     * migrated from cql_tests.py:TestCQL.expanded_map_item_conditional_test()
+     */
+    @Test
+    public void testExpandedMapItem() throws Throwable
+    {
+        for (boolean frozen : new boolean[]{ false, true })
+        {
+            createTable(String.format("CREATE TABLE %%s (k int PRIMARY KEY, m %s)",
+                                      frozen
+                                      ? "frozen<map<text, text>>"
+                                      : "map<text, text>"));
+
+            execute("INSERT INTO %s (k, m) VALUES (0, {'foo' : 'bar'})");
+
+            check_applies_map("m['xxx'] = null");
+            check_applies_map("m['foo'] < 'zzz'");
+            check_applies_map("m['foo'] <= 'bar'");
+            check_applies_map("m['foo'] > 'aaa'");
+            check_applies_map("m['foo'] >= 'bar'");
+            check_applies_map("m['foo'] != 'xxx'");
+            check_applies_map("m['foo'] != null");
+            check_applies_map("m['foo'] IN (null, 'xxx', 'bar')");
+            check_applies_map("m['xxx'] IN (null, 'xxx', 'bar')"); // m['xxx'] is not set
+
+            // multiple conditions
+            check_applies_map("m['foo'] < 'zzz' AND m['foo'] > 'aaa'");
+
+            check_does_not_apply_map("m['foo'] < 'aaa'");
+            check_does_not_apply_map("m['foo'] <= 'aaa'");
+            check_does_not_apply_map("m['foo'] > 'zzz'");
+            check_does_not_apply_map("m['foo'] >= 'zzz'");
+            check_does_not_apply_map("m['foo'] != 'bar'");
+            check_does_not_apply_map("m['xxx'] != null");  // m['xxx'] is not set
+            check_does_not_apply_map("m['foo'] IN (null, 'xxx')");
+            check_does_not_apply_map("m['foo'] IN ()");
+            check_does_not_apply_map("m['foo'] != null AND m['foo'] = null");
+
+            check_invalid_map("m['foo'] < null", InvalidRequestException.class);
+            check_invalid_map("m['foo'] <= null", InvalidRequestException.class);
+            check_invalid_map("m['foo'] > null", InvalidRequestException.class);
+            check_invalid_map("m['foo'] >= null", InvalidRequestException.class);
+            check_invalid_map("m['foo'] IN null", SyntaxException.class);
+            check_invalid_map("m['foo'] IN 367", SyntaxException.class);
+            check_invalid_map("m['foo'] IN (1, 2, 3)", InvalidRequestException.class);
+            check_invalid_map("m['foo'] CONTAINS 367", SyntaxException.class);
+            check_invalid_map("m['foo'] CONTAINS KEY 367", SyntaxException.class);
+            check_invalid_map("m[null] = null", InvalidRequestException.class);
+        }
+    }
+
+    void check_applies_map(String condition) throws Throwable
+    {
+        assertRows(execute("UPDATE %s SET m = {'foo': 'bar'} WHERE k=0 IF " + condition), row(true));
+        assertRows(execute("SELECT * FROM %s"), row(0, map("foo", "bar")));
+    }
+
+    void check_does_not_apply_map(String condition) throws Throwable
+    {
+        assertRows(execute("UPDATE %s SET m = {'foo': 'bar'} WHERE k=0 IF " + condition), row(false, map("foo", "bar")));
+        assertRows(execute("SELECT * FROM %s"), row(0, map("foo", "bar")));
+    }
+
+    void check_invalid_map(String condition, Class<? extends Throwable> expected) throws Throwable
+    {
+        assertInvalidThrow(expected, "UPDATE %s SET m = {'foo': 'bar'} WHERE k=0 IF " + condition);
+        assertRows(execute("SELECT * FROM %s"), row(0, map("foo", "bar")));
+    }
+
+    /**
+     * Test for 7499,
+     * migrated from cql_tests.py:TestCQL.cas_and_list_index_test()
+     */
+    @Test
+    public void testCasAndListIndex() throws Throwable
+    {
+        createTable("CREATE TABLE %s ( k int PRIMARY KEY, v text, l list<text>)");
+
+        execute("INSERT INTO %s (k, v, l) VALUES(0, 'foobar', ['foi', 'bar'])");
+
+        assertRows(execute("UPDATE %s SET l[0] = 'foo' WHERE k = 0 IF v = 'barfoo'"), row(false, "foobar"));
+        assertRows(execute("UPDATE %s SET l[0] = 'foo' WHERE k = 0 IF v = 'foobar'"), row(true));
+
+        assertRows(execute("SELECT * FROM %s"), row(0, list("foo", "bar"), "foobar"));
+
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.conditional_ddl_keyspace_test()
+     */
+    @Test
+    public void testDropCreateKeyspaceIfNotExists() throws Throwable
+    {
+        String keyspace =  KEYSPACE_PER_TEST;
+
+        dropPerTestKeyspace();
+
+        // try dropping when doesn't exist
+        dropPerTestKeyspace();
+
+        // create and confirm
+        schemaChange("CREATE KEYSPACE IF NOT EXISTS " + keyspace + " WITH replication = { 'class':'SimpleStrategy', 'replication_factor':1} and durable_writes = true ");
+        assertRows(execute("select durable_writes from system.schema_keyspaces where keyspace_name = ?", keyspace), row(true));
+
+        // unsuccessful create since it's already there, confirm settings don't change
+        schemaChange("CREATE KEYSPACE IF NOT EXISTS " + keyspace + " WITH replication = {'class':'SimpleStrategy', 'replication_factor':1} and durable_writes = false ");
+
+        assertRows(execute("select durable_writes from system.schema_keyspaces where keyspace_name = ?", keyspace), row(true));
+
+        // drop and confirm
+        schemaChange("DROP KEYSPACE IF EXISTS " + keyspace);
+
+        assertEmpty(execute("select * from system.schema_keyspaces where keyspace_name = ?", keyspace));
+    }
+
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.conditional_ddl_table_test()
+     */
+    @Test
+    public void testDropCreateTableIfNotExists() throws Throwable
+    {
+        String tableName = createTableName();
+        String fullTableName = KEYSPACE + "." + tableName;
+
+        // try dropping when doesn't exist
+        schemaChange("DROP TABLE IF EXISTS " + fullTableName);
+
+        // create and confirm
+        schemaChange("CREATE TABLE IF NOT EXISTS " + fullTableName + " (id text PRIMARY KEY, value1 blob) with comment = 'foo'");
+
+        assertRows(execute("select comment from system.schema_columnfamilies where keyspace_name = ? and columnfamily_name = ?", KEYSPACE, tableName),
+                   row("foo"));
+
+        // unsuccessful create since it's already there, confirm settings don't change
+        schemaChange("CREATE TABLE IF NOT EXISTS " + fullTableName + " (id text PRIMARY KEY, value2 blob)with comment = 'bar'");
+
+        assertRows(execute("select comment from system.schema_columnfamilies where keyspace_name = ? and columnfamily_name = ?", KEYSPACE, tableName),
+                   row("foo"));
+
+        // drop and confirm
+        schemaChange("DROP TABLE IF EXISTS " + fullTableName);
+
+        assertEmpty(execute("select * from system.schema_columnfamilies where keyspace_name = ? and columnfamily_name = ?", KEYSPACE, tableName));
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.conditional_ddl_index_test()
+     */
+    @Test
+    public void testDropCreateIndexIfNotExists() throws Throwable
+    {
+        String tableName = createTable("CREATE TABLE %s (id text PRIMARY KEY, value1 blob, value2 blob)with comment = 'foo'");
+
+        execute("use " + KEYSPACE);
+
+        // try dropping when doesn't exist
+        schemaChange("DROP INDEX IF EXISTS myindex");
+
+        // create and confirm
+        createIndex("CREATE INDEX IF NOT EXISTS myindex ON %s (value1)");
+
+        assertTrue(waitForIndex(KEYSPACE, tableName, "myindex"));
+
+        // unsuccessful create since it's already there
+        execute("CREATE INDEX IF NOT EXISTS myindex ON %s (value1)");
+
+        // drop and confirm
+        execute("DROP INDEX IF EXISTS myindex");
+
+        Object[][] rows = getRows(execute("select index_name from system.\"IndexInfo\" where table_name = ?", tableName));
+        assertEquals(0, rows.length);
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.conditional_ddl_type_test()
+     */
+    @Test
+    public void testDropCreateTypeIfNotExists() throws Throwable
+    {
+        execute("use " + KEYSPACE);
+
+        // try dropping when doesn 't exist
+        execute("DROP TYPE IF EXISTS mytype");
+
+        // create and confirm
+        execute("CREATE TYPE IF NOT EXISTS mytype (somefield int)");
+        assertRows(execute("SELECT type_name from system.schema_usertypes where keyspace_name = ? and type_name = ?", KEYSPACE, "mytype"),
+                   row("mytype"));
+
+        // unsuccessful create since it 's already there
+        // TODO: confirm this create attempt doesn't alter type field from int to blob
+        execute("CREATE TYPE IF NOT EXISTS mytype (somefield blob)");
+
+        // drop and confirm
+        execute("DROP TYPE IF EXISTS mytype");
+        assertEmpty(execute("SELECT type_name from system.schema_usertypes where keyspace_name = ? and type_name = ?", KEYSPACE, "mytype"));
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/01115f72/test/unit/org/apache/cassandra/cql3/validation/operations/SelectLimitTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/SelectLimitTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/SelectLimitTest.java
new file mode 100644
index 0000000..275ff04
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/SelectLimitTest.java
@@ -0,0 +1,112 @@
+package org.apache.cassandra.cql3.validation.operations;
+
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.dht.ByteOrderedPartitioner;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+
+public class SelectLimitTest extends CQLTester
+{
+    @BeforeClass
+    public static void setUp()
+    {
+        DatabaseDescriptor.setPartitioner(ByteOrderedPartitioner.instance);
+    }
+
+    /**
+     * Test limit across a partition range, requires byte ordered partitioner,
+     * migrated from cql_tests.py:TestCQL.limit_range_test()
+     */
+    @Test
+    public void testPartitionRange() throws Throwable
+    {
+        createTable("CREATE TABLE %s (userid int, url text, time bigint, PRIMARY KEY (userid, url)) WITH COMPACT STORAGE");
+
+        for (int i = 0; i < 100; i++)
+            for (String tld : new String[] { "com", "org", "net" })
+                execute("INSERT INTO %s (userid, url, time) VALUES (?, ?, ?)", i, String.format("http://foo.%s", tld), 42L);
+
+        assertRows(execute("SELECT * FROM %s WHERE token(userid) >= token(2) LIMIT 1"),
+                   row(2, "http://foo.com", 42L));
+
+        assertRows(execute("SELECT * FROM %s WHERE token(userid) > token(2) LIMIT 1"),
+                   row(3, "http://foo.com", 42L));
+    }
+
+    /**
+     * Test limit across a column range,
+     * migrated from cql_tests.py:TestCQL.limit_multiget_test()
+     */
+    @Test
+    public void testColumnRange() throws Throwable
+    {
+        createTable("CREATE TABLE %s (userid int, url text, time bigint, PRIMARY KEY (userid, url)) WITH COMPACT STORAGE");
+
+        for (int i = 0; i < 100; i++)
+            for (String tld : new String[] { "com", "org", "net" })
+                execute("INSERT INTO %s (userid, url, time) VALUES (?, ?, ?)", i, String.format("http://foo.%s", tld), 42L);
+
+        // Check that we do limit the output to 1 *and* that we respect query
+        // order of keys (even though 48 is after 2)
+        assertRows(execute("SELECT * FROM %s WHERE userid IN (48, 2) LIMIT 1"),
+                   row(2, "http://foo.com", 42L));
+
+    }
+
+    /**
+     * Test limit queries on a sparse table,
+     * migrated from cql_tests.py:TestCQL.limit_sparse_test()
+     */
+    @Test
+    public void testSparseTable() throws Throwable
+    {
+        createTable("CREATE TABLE %s (userid int, url text, day int, month text, year int, PRIMARY KEY (userid, url))");
+
+        for (int i = 0; i < 100; i++)
+            for (String tld : new String[] { "com", "org", "net" })
+                execute("INSERT INTO %s (userid, url, day, month, year) VALUES (?, ?, 1, 'jan', 2012)", i, String.format("http://foo.%s", tld));
+
+        assertRowCount(execute("SELECT * FROM %s LIMIT 4"), 4);
+
+    }
+
+    /**
+     * Check for #7052 bug,
+     * migrated from cql_tests.py:TestCQL.limit_compact_table()
+     */
+    @Test
+    public void testLimitInCompactTable() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int, v int, PRIMARY KEY (k, v) ) WITH COMPACT STORAGE ");
+
+        for (int i = 0; i < 4; i++)
+            for (int j = 0; j < 4; j++)
+                execute("INSERT INTO %s(k, v) VALUES (?, ?)", i, j);
+
+        assertRows(execute("SELECT v FROM %s WHERE k=0 AND v > 0 AND v <= 4 LIMIT 2"),
+                   row(1),
+                   row(2));
+        assertRows(execute("SELECT v FROM %s WHERE k=0 AND v > -1 AND v <= 4 LIMIT 2"),
+                   row(0),
+                   row(1));
+        assertRows(execute("SELECT * FROM %s WHERE k IN (0, 1, 2) AND v > 0 AND v <= 4 LIMIT 2"),
+                   row(0, 1),
+                   row(0, 2));
+        assertRows(execute("SELECT * FROM %s WHERE k IN (0, 1, 2) AND v > -1 AND v <= 4 LIMIT 2"),
+                   row(0, 0),
+                   row(0, 1));
+        assertRows(execute("SELECT * FROM %s WHERE k IN (0, 1, 2) AND v > 0 AND v <= 4 LIMIT 6"),
+                   row(0, 1),
+                   row(0, 2),
+                   row(0, 3),
+                   row(1, 1),
+                   row(1, 2),
+                   row(1, 3));
+
+        // strict bound (v > 1) over a range of partitions is not supported for compact storage if limit is provided
+        assertInvalidThrow(InvalidRequestException.class, "SELECT * FROM %s WHERE v > 1 AND v <= 3 LIMIT 6 ALLOW FILTERING");
+    }
+}


[19/32] cassandra git commit: 2.2 commit for CASSANDRA-9160

Posted by jm...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/01115f72/test/unit/org/apache/cassandra/cql3/validation/entities/SecondaryIndexTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/SecondaryIndexTest.java b/test/unit/org/apache/cassandra/cql3/validation/entities/SecondaryIndexTest.java
new file mode 100644
index 0000000..0b812c6
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/SecondaryIndexTest.java
@@ -0,0 +1,645 @@
+/*
+ * 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.validation.entities;
+
+import java.nio.ByteBuffer;
+import java.util.HashMap;
+import java.util.Locale;
+import java.util.Map;
+import java.util.UUID;
+
+import org.apache.commons.lang3.StringUtils;
+
+import org.junit.Test;
+
+import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.exceptions.SyntaxException;
+import org.apache.cassandra.utils.FBUtilities;
+
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+public class SecondaryIndexTest extends CQLTester
+{
+    private static final int TOO_BIG = 1024 * 65;
+
+    @Test
+    public void testCreateAndDropIndex() throws Throwable
+    {
+        testCreateAndDropIndex("test", false);
+        testCreateAndDropIndex("test2", true);
+    }
+
+    @Test
+    public void testCreateAndDropIndexWithQuotedIdentifier() throws Throwable
+    {
+        testCreateAndDropIndex("\"quoted_ident\"", false);
+        testCreateAndDropIndex("\"quoted_ident2\"", true);
+    }
+
+    @Test
+    public void testCreateAndDropIndexWithCamelCaseIdentifier() throws Throwable
+    {
+        testCreateAndDropIndex("CamelCase", false);
+        testCreateAndDropIndex("CamelCase2", true);
+    }
+
+    /**
+     * Test creating and dropping an index with the specified name.
+     *
+     * @param indexName         the index name
+     * @param addKeyspaceOnDrop add the keyspace name in the drop statement
+     * @throws Throwable if an error occurs
+     */
+    private void testCreateAndDropIndex(String indexName, boolean addKeyspaceOnDrop) throws Throwable
+    {
+        execute("USE system");
+        assertInvalidMessage("Index '" + removeQuotes(indexName.toLowerCase(Locale.US)) + "' could not be found", "DROP INDEX " + indexName + ";");
+
+        createTable("CREATE TABLE %s (a int primary key, b int);");
+        createIndex("CREATE INDEX " + indexName + " ON %s(b);");
+        createIndex("CREATE INDEX IF NOT EXISTS " + indexName + " ON %s(b);");
+
+        assertInvalidMessage("Index already exists", "CREATE INDEX " + indexName + " ON %s(b)");
+
+        execute("INSERT INTO %s (a, b) values (?, ?);", 0, 0);
+        execute("INSERT INTO %s (a, b) values (?, ?);", 1, 1);
+        execute("INSERT INTO %s (a, b) values (?, ?);", 2, 2);
+        execute("INSERT INTO %s (a, b) values (?, ?);", 3, 1);
+
+        assertRows(execute("SELECT * FROM %s where b = ?", 1), row(1, 1), row(3, 1));
+        assertInvalidMessage("Index '" + removeQuotes(indexName.toLowerCase(Locale.US)) + "' could not be found in any of the tables of keyspace 'system'",
+                             "DROP INDEX " + indexName);
+
+        if (addKeyspaceOnDrop)
+        {
+            dropIndex("DROP INDEX " + KEYSPACE + "." + indexName);
+        }
+        else
+        {
+            execute("USE " + KEYSPACE);
+            execute("DROP INDEX " + indexName);
+        }
+
+        assertInvalidMessage("No secondary indexes on the restricted columns support the provided operators",
+                             "SELECT * FROM %s where b = ?", 1);
+        dropIndex("DROP INDEX IF EXISTS " + indexName);
+        assertInvalidMessage("Index '" + removeQuotes(indexName.toLowerCase(Locale.US)) + "' could not be found", "DROP INDEX " + indexName);
+    }
+
+    /**
+     * Removes the quotes from the specified index name.
+     *
+     * @param indexName the index name from which the quotes must be removed.
+     * @return the unquoted index name.
+     */
+    private static String removeQuotes(String indexName)
+    {
+        return StringUtils.remove(indexName, '\"');
+    }
+
+    /**
+     * Check that you can query for an indexed column even with a key EQ clause,
+     * migrated from cql_tests.py:TestCQL.static_cf_test()
+     */
+    @Test
+    public void testSelectWithEQ() throws Throwable
+    {
+        createTable("CREATE TABLE %s (userid uuid PRIMARY KEY, firstname text, lastname text, age int)");
+        createIndex("CREATE INDEX byAge ON %s(age)");
+
+        UUID id1 = UUID.fromString("550e8400-e29b-41d4-a716-446655440000");
+        UUID id2 = UUID.fromString("f47ac10b-58cc-4372-a567-0e02b2c3d479");
+
+        execute("INSERT INTO %s (userid, firstname, lastname, age) VALUES (?, 'Frodo', 'Baggins', 32)", id1);
+        execute("UPDATE %s SET firstname = 'Samwise', lastname = 'Gamgee', age = 33 WHERE userid = ?", id2);
+
+        assertEmpty(execute("SELECT firstname FROM %s WHERE userid = ? AND age = 33", id1));
+
+        assertRows(execute("SELECT firstname FROM %s WHERE userid = ? AND age = 33", id2),
+                   row("Samwise"));
+    }
+
+    /**
+     * Check CREATE INDEX without name and validate the index can be dropped,
+     * migrated from cql_tests.py:TestCQL.nameless_index_test()
+     */
+    @Test
+    public void testNamelessIndex() throws Throwable
+    {
+        createTable(" CREATE TABLE %s (id text PRIMARY KEY, birth_year int)");
+
+        createIndex("CREATE INDEX on %s (birth_year)");
+
+        execute("INSERT INTO %s (id, birth_year) VALUES ('Tom', 42)");
+        execute("INSERT INTO %s (id, birth_year) VALUES ('Paul', 24)");
+        execute("INSERT INTO %s (id, birth_year) VALUES ('Bob', 42)");
+
+        assertRows(execute("SELECT id FROM %s WHERE birth_year = 42"),
+                   row("Tom"),
+                   row("Bob"));
+
+        execute("DROP INDEX %s_birth_year_idx");
+
+        assertInvalid("SELECT id FROM users WHERE birth_year = 42");
+    }
+
+    /**
+     * Test range queries with 2ndary indexes (#4257),
+     * migrated from cql_tests.py:TestCQL.range_query_2ndary_test()
+     */
+    @Test
+    public void testRangeQuery() throws Throwable
+    {
+        createTable("CREATE TABLE %s (id int primary key, row int, setid int)");
+        createIndex("CREATE INDEX indextest_setid_idx ON %s (setid)");
+
+        execute("INSERT INTO %s (id, row, setid) VALUES (?, ?, ?)", 0, 0, 0);
+        execute("INSERT INTO %s (id, row, setid) VALUES (?, ?, ?)", 1, 1, 0);
+        execute("INSERT INTO %s (id, row, setid) VALUES (?, ?, ?)", 2, 2, 0);
+        execute("INSERT INTO %s (id, row, setid) VALUES (?, ?, ?)", 3, 3, 0);
+
+        assertInvalid("SELECT * FROM %s WHERE setid = 0 AND row < 1");
+
+        assertRows(execute("SELECT * FROM %s WHERE setid = 0 AND row < 1 ALLOW FILTERING"),
+                   row(0, 0, 0));
+    }
+
+    /**
+     * Check for unknown compression parameters options (#4266),
+     * migrated from cql_tests.py:TestCQL.compression_option_validation_test()
+     */
+    @Test
+    public void testUnknownCompressionOptions() throws Throwable
+    {
+        String tableName = createTableName();
+        assertInvalidThrow(SyntaxException.class, String.format(
+                                                               "CREATE TABLE %s (key varchar PRIMARY KEY, password varchar, gender varchar) WITH compression_parameters:sstable_compressor = 'DeflateCompressor'", tableName));
+
+
+        assertInvalidThrow(ConfigurationException.class, String.format(
+                                                                      "CREATE TABLE %s (key varchar PRIMARY KEY, password varchar, gender varchar) WITH compression = { 'sstable_compressor': 'DeflateCompressor' }", tableName));
+    }
+
+    /**
+     * Check one can use arbitrary name for datacenter when creating keyspace (#4278),
+     * migrated from cql_tests.py:TestCQL.keyspace_creation_options_test()
+     */
+    @Test
+    public void testDataCenterName() throws Throwable
+    {
+       execute("CREATE KEYSPACE Foo WITH replication = { 'class' : 'NetworkTopologyStrategy', 'us-east' : 1, 'us-west' : 1 };");
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.indexes_composite_test()
+     */
+    @Test
+    public void testIndexOnComposite() throws Throwable
+    {
+        String tableName = createTable("CREATE TABLE %s (blog_id int, timestamp int, author text, content text, PRIMARY KEY (blog_id, timestamp))");
+
+        execute("INSERT INTO %s (blog_id, timestamp, author, content) VALUES (?, ?, ?, ?)", 0, 0, "bob", "1st post");
+        execute("INSERT INTO %s (blog_id, timestamp, author, content) VALUES (?, ?, ?, ?)", 0, 1, "tom", "2nd post");
+        execute("INSERT INTO %s (blog_id, timestamp, author, content) VALUES (?, ?, ?, ?)", 0, 2, "bob", "3rd post");
+        execute("INSERT INTO %s (blog_id, timestamp, author, content) VALUES (?, ?, ?, ?)", 0, 3, "tom", "4th post");
+        execute("INSERT INTO %s (blog_id, timestamp, author, content) VALUES (?, ?, ?, ?)", 1, 0, "bob", "5th post");
+
+        createIndex("CREATE INDEX authoridx ON %s (author)");
+
+        assertTrue(waitForIndex(keyspace(), tableName, "authoridx"));
+
+        assertRows(execute("SELECT blog_id, timestamp FROM %s WHERE author = 'bob'"),
+                   row(1, 0),
+                   row(0, 0),
+                   row(0, 2));
+
+        execute("INSERT INTO %s (blog_id, timestamp, author, content) VALUES (?, ?, ?, ?)", 1, 1, "tom", "6th post");
+        execute("INSERT INTO %s (blog_id, timestamp, author, content) VALUES (?, ?, ?, ?)", 1, 2, "tom", "7th post");
+        execute("INSERT INTO %s (blog_id, timestamp, author, content) VALUES (?, ?, ?, ?)", 1, 3, "bob", "8th post");
+
+        assertRows(execute("SELECT blog_id, timestamp FROM %s WHERE author = 'bob'"),
+                   row(1, 0),
+                   row(1, 3),
+                   row(0, 0),
+                   row(0, 2));
+
+        execute("DELETE FROM %s WHERE blog_id = 0 AND timestamp = 2");
+
+        assertRows(execute("SELECT blog_id, timestamp FROM %s WHERE author = 'bob'"),
+                   row(1, 0),
+                   row(1, 3),
+                   row(0, 0));
+    }
+
+    /**
+     * Test for the validation bug of #4709,
+     * migrated from cql_tests.py:TestCQL.refuse_in_with_indexes_test()
+     */
+    @Test
+    public void testInvalidIndexSelect() throws Throwable
+    {
+        createTable("create table %s (pk varchar primary key, col1 varchar, col2 varchar)");
+        createIndex("create index on %s (col1)");
+        createIndex("create index on %s (col2)");
+
+        execute("insert into %s (pk, col1, col2) values ('pk1','foo1','bar1')");
+        execute("insert into %s (pk, col1, col2) values ('pk1a','foo1','bar1')");
+        execute("insert into %s (pk, col1, col2) values ('pk1b','foo1','bar1')");
+        execute("insert into %s (pk, col1, col2) values ('pk1c','foo1','bar1')");
+        execute("insert into %s (pk, col1, col2) values ('pk2','foo2','bar2')");
+        execute("insert into %s (pk, col1, col2) values ('pk3','foo3','bar3')");
+        assertInvalid("select * from %s where col2 in ('bar1', 'bar2')");
+
+        //Migrated from cql_tests.py:TestCQL.bug_6050_test()
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, a int, b int)");
+
+        createIndex("CREATE INDEX ON %s (a)");
+        assertInvalid("SELECT * FROM %s WHERE a = 3 AND b IN (1, 3)");
+
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.edge_2i_on_complex_pk_test()
+     */
+    @Test
+    public void testIndexesOnComplexPrimaryKey() throws Throwable
+    {
+        createTable("CREATE TABLE %s (pk0 int, pk1 int, ck0 int, ck1 int, ck2 int, value int, PRIMARY KEY ((pk0, pk1), ck0, ck1, ck2))");
+
+        execute("CREATE INDEX ON %s (pk0)");
+        execute("CREATE INDEX ON %s (ck0)");
+        execute("CREATE INDEX ON %s (ck1)");
+        execute("CREATE INDEX ON %s (ck2)");
+
+        execute("INSERT INTO %s (pk0, pk1, ck0, ck1, ck2, value) VALUES (0, 1, 2, 3, 4, 5)");
+        execute("INSERT INTO %s (pk0, pk1, ck0, ck1, ck2, value) VALUES (1, 2, 3, 4, 5, 0)");
+        execute("INSERT INTO %s (pk0, pk1, ck0, ck1, ck2, value) VALUES (2, 3, 4, 5, 0, 1)");
+        execute("INSERT INTO %s (pk0, pk1, ck0, ck1, ck2, value) VALUES (3, 4, 5, 0, 1, 2)");
+        execute("INSERT INTO %s (pk0, pk1, ck0, ck1, ck2, value) VALUES (4, 5, 0, 1, 2, 3)");
+        execute("INSERT INTO %s (pk0, pk1, ck0, ck1, ck2, value) VALUES (5, 0, 1, 2, 3, 4)");
+
+        assertRows(execute("SELECT value FROM %s WHERE pk0 = 2"),
+                   row(1));
+
+        assertRows(execute("SELECT value FROM %s WHERE ck0 = 0"),
+                   row(3));
+
+        assertRows(execute("SELECT value FROM %s WHERE pk0 = 3 AND pk1 = 4 AND ck1 = 0"),
+                   row(2));
+
+        assertRows(execute("SELECT value FROM %s WHERE pk0 = 5 AND pk1 = 0 AND ck0 = 1 AND ck2 = 3 ALLOW FILTERING"),
+                   row(4));
+    }
+
+    /**
+     * Test for CASSANDRA-5240,
+     * migrated from cql_tests.py:TestCQL.bug_5240_test()
+     */
+    @Test
+    public void testIndexOnCompoundRowKey() throws Throwable
+    {
+        createTable("CREATE TABLE %s (interval text, seq int, id int, severity int, PRIMARY KEY ((interval, seq), id) ) WITH CLUSTERING ORDER BY (id DESC)");
+
+        execute("CREATE INDEX ON %s (severity)");
+
+        execute("insert into %s (interval, seq, id , severity) values('t',1, 1, 1)");
+        execute("insert into %s (interval, seq, id , severity) values('t',1, 2, 1)");
+        execute("insert into %s (interval, seq, id , severity) values('t',1, 3, 2)");
+        execute("insert into %s (interval, seq, id , severity) values('t',1, 4, 3)");
+        execute("insert into %s (interval, seq, id , severity) values('t',2, 1, 3)");
+        execute("insert into %s (interval, seq, id , severity) values('t',2, 2, 3)");
+        execute("insert into %s (interval, seq, id , severity) values('t',2, 3, 1)");
+        execute("insert into %s (interval, seq, id , severity) values('t',2, 4, 2)");
+
+        assertRows(execute("select * from %s where severity = 3 and interval = 't' and seq =1"),
+                   row("t", 1, 4, 3));
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.secondary_index_counters()
+     */
+    @Test
+    public void testIndexOnCountersInvalid() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, c counter)");
+        assertInvalid("CREATE INDEX ON test(c)");
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.collection_indexing_test()
+     */
+    @Test
+    public void testIndexOnCollections() throws Throwable
+    {
+        createTable(" CREATE TABLE %s ( k int, v int, l list<int>, s set<text>, m map<text, int>, PRIMARY KEY (k, v))");
+
+        createIndex("CREATE INDEX ON %s (l)");
+        createIndex("CREATE INDEX ON %s (s)");
+        createIndex("CREATE INDEX ON %s (m)");
+
+        execute("INSERT INTO %s (k, v, l, s, m) VALUES (0, 0, [1, 2],    {'a'},      {'a' : 1})");
+        execute("INSERT INTO %s (k, v, l, s, m) VALUES (0, 1, [3, 4],    {'b', 'c'}, {'a' : 1, 'b' : 2})");
+        execute("INSERT INTO %s (k, v, l, s, m) VALUES (0, 2, [1],       {'a', 'c'}, {'c' : 3})");
+        execute("INSERT INTO %s (k, v, l, s, m) VALUES (1, 0, [1, 2, 4], {},         {'b' : 1})");
+        execute("INSERT INTO %s (k, v, l, s, m) VALUES (1, 1, [4, 5],    {'d'},      {'a' : 1, 'b' : 3})");
+
+        // lists
+        assertRows(execute("SELECT k, v FROM %s WHERE l CONTAINS 1"), row(1, 0), row(0, 0), row(0, 2));
+        assertRows(execute("SELECT k, v FROM %s WHERE k = 0 AND l CONTAINS 1"), row(0, 0), row(0, 2));
+        assertRows(execute("SELECT k, v FROM %s WHERE l CONTAINS 2"), row(1, 0), row(0, 0));
+        assertEmpty(execute("SELECT k, v FROM %s WHERE l CONTAINS 6"));
+
+        // sets
+        assertRows(execute("SELECT k, v FROM %s WHERE s CONTAINS 'a'"), row(0, 0), row(0, 2));
+        assertRows(execute("SELECT k, v FROM %s WHERE k = 0 AND s CONTAINS 'a'"), row(0, 0), row(0, 2));
+        assertRows(execute("SELECT k, v FROM %s WHERE s CONTAINS 'd'"), row(1, 1));
+        assertEmpty(execute("SELECT k, v FROM %s  WHERE s CONTAINS 'e'"));
+
+        // maps
+        assertRows(execute("SELECT k, v FROM %s WHERE m CONTAINS 1"), row(1, 0), row(1, 1), row(0, 0), row(0, 1));
+        assertRows(execute("SELECT k, v FROM %s WHERE k = 0 AND m CONTAINS 1"), row(0, 0), row(0, 1));
+        assertRows(execute("SELECT k, v FROM %s WHERE m CONTAINS 2"), row(0, 1));
+        assertEmpty(execute("SELECT k, v FROM %s  WHERE m CONTAINS 4"));
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.map_keys_indexing()
+     */
+    @Test
+    public void testIndexOnMapKeys() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int, v int, m map<text, int>, PRIMARY KEY (k, v))");
+
+        createIndex("CREATE INDEX ON %s(keys(m))");
+
+        execute("INSERT INTO %s (k, v, m) VALUES (0, 0, {'a' : 1})");
+        execute("INSERT INTO %s (k, v, m) VALUES (0, 1, {'a' : 1, 'b' : 2})");
+        execute("INSERT INTO %s (k, v, m) VALUES (0, 2, {'c' : 3})");
+        execute("INSERT INTO %s (k, v, m) VALUES (1, 0, {'b' : 1})");
+        execute("INSERT INTO %s (k, v, m) VALUES (1, 1, {'a' : 1, 'b' : 3})");
+
+        // maps
+        assertRows(execute("SELECT k, v FROM %s WHERE m CONTAINS KEY 'a'"), row(1, 1), row(0, 0), row(0, 1));
+        assertRows(execute("SELECT k, v FROM %s WHERE k = 0 AND m CONTAINS KEY 'a'"), row(0, 0), row(0, 1));
+        assertRows(execute("SELECT k, v FROM %s WHERE m CONTAINS KEY 'c'"), row(0, 2));
+        assertEmpty(execute("SELECT k, v FROM %s  WHERE m CONTAINS KEY 'd'"));
+
+        // we're not allowed to create a value index if we already have a key one
+        assertInvalid("CREATE INDEX ON %s(m)");
+    }
+
+    /**
+     * Test for #6950 bug,
+     * migrated from cql_tests.py:TestCQL.key_index_with_reverse_clustering()
+     */
+    @Test
+    public void testIndexOnKeyWithReverseClustering() throws Throwable
+    {
+        createTable(" CREATE TABLE %s (k1 int, k2 int, v int, PRIMARY KEY ((k1, k2), v) ) WITH CLUSTERING ORDER BY (v DESC)");
+
+        createIndex("CREATE INDEX ON %s (k2)");
+
+        execute("INSERT INTO %s (k1, k2, v) VALUES (0, 0, 1)");
+        execute("INSERT INTO %s (k1, k2, v) VALUES (0, 1, 2)");
+        execute("INSERT INTO %s (k1, k2, v) VALUES (0, 0, 3)");
+        execute("INSERT INTO %s (k1, k2, v) VALUES (1, 0, 4)");
+        execute("INSERT INTO %s (k1, k2, v) VALUES (1, 1, 5)");
+        execute("INSERT INTO %s (k1, k2, v) VALUES (2, 0, 7)");
+        execute("INSERT INTO %s (k1, k2, v) VALUES (2, 1, 8)");
+        execute("INSERT INTO %s (k1, k2, v) VALUES (3, 0, 1)");
+
+        assertRows(execute("SELECT * FROM %s WHERE k2 = 0 AND v >= 2 ALLOW FILTERING"),
+                   row(2, 0, 7),
+                   row(0, 0, 3),
+                   row(1, 0, 4));
+    }
+
+    /**
+     * Test for CASSANDRA-6612,
+     * migrated from cql_tests.py:TestCQL.bug_6612_test()
+     */
+    @Test
+    public void testSelectCountOnIndexedColumn() throws Throwable
+    {
+        createTable("CREATE TABLE %s (username text, session_id text, app_name text, account text, last_access timestamp, created_on timestamp, PRIMARY KEY (username, session_id, app_name, account))");
+
+        createIndex("create index ON %s (app_name)");
+        createIndex("create index ON %s (last_access)");
+
+        assertRows(execute("select count(*) from %s where app_name='foo' and account='bar' and last_access > 4 allow filtering"), row(0L));
+
+        execute("insert into %s (username, session_id, app_name, account, last_access, created_on) values ('toto', 'foo', 'foo', 'bar', 12, 13)");
+
+        assertRows(execute("select count(*) from %s where app_name='foo' and account='bar' and last_access > 4 allow filtering"), row(1L));
+    }
+
+    /**
+     * Test for CASSANDRA-5732, Can not query secondary index
+     * migrated from cql_tests.py:TestCQL.bug_5732_test(),
+     * which was executing with a row cache size of 100 MB
+     * and restarting the node, here we just cleanup the cache.
+     */
+    @Test
+    public void testCanQuerySecondaryIndex() throws Throwable
+    {
+        String tableName = createTable("CREATE TABLE %s (k int PRIMARY KEY, v int,)");
+
+        execute("ALTER TABLE %s WITH CACHING='ALL'");
+        execute("INSERT INTO %s (k,v) VALUES (0,0)");
+        execute("INSERT INTO %s (k,v) VALUES (1,1)");
+
+        createIndex("CREATE INDEX testindex on %s (v)");
+        assertTrue(waitForIndex(keyspace(), tableName, "testindex"));
+
+        assertRows(execute("SELECT k FROM %s WHERE v = 0"), row(0));
+        cleanupCache();
+        assertRows(execute("SELECT k FROM %s WHERE v = 0"), row(0));
+    }
+
+    // CASSANDRA-8280/8081
+    // reject updates with indexed values where value > 64k
+    @Test
+    public void testIndexOnCompositeValueOver64k() throws Throwable
+    {
+        createTable("CREATE TABLE %s(a int, b int, c blob, PRIMARY KEY (a))");
+        createIndex("CREATE INDEX ON %s(c)");
+        failInsert("INSERT INTO %s (a, b, c) VALUES (0, 0, ?)", ByteBuffer.allocate(TOO_BIG));
+    }
+
+    @Test
+    public void testIndexOnClusteringColumnInsertPartitionKeyAndClusteringsOver64k() throws Throwable
+    {
+        createTable("CREATE TABLE %s(a blob, b blob, c blob, d int, PRIMARY KEY (a, b, c))");
+        createIndex("CREATE INDEX ON %s(b)");
+
+        // CompositeIndexOnClusteringKey creates index entries composed of the
+        // PK plus all of the non-indexed clustering columns from the primary row
+        // so we should reject where len(a) + len(c) > 65560 as this will form the
+        // total clustering in the index table
+        ByteBuffer a = ByteBuffer.allocate(100);
+        ByteBuffer b = ByteBuffer.allocate(10);
+        ByteBuffer c = ByteBuffer.allocate(FBUtilities.MAX_UNSIGNED_SHORT - 99);
+
+        failInsert("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, 0)", a, b, c);
+    }
+
+    @Test
+    public void testCompactTableWithValueOver64k() throws Throwable
+    {
+        createTable("CREATE TABLE %s(a int, b blob, PRIMARY KEY (a)) WITH COMPACT STORAGE");
+        createIndex("CREATE INDEX ON %s(b)");
+        failInsert("INSERT INTO %s (a, b) VALUES (0, ?)", ByteBuffer.allocate(TOO_BIG));
+    }
+
+    @Test
+    public void testIndexOnCollectionValueInsertPartitionKeyAndCollectionKeyOver64k() throws Throwable
+    {
+        createTable("CREATE TABLE %s(a blob , b map<blob, int>, PRIMARY KEY (a))");
+        createIndex("CREATE INDEX ON %s(b)");
+
+        // A collection key > 64k by itself will be rejected from
+        // the primary table.
+        // To test index validation we need to ensure that
+        // len(b) < 64k, but len(a) + len(b) > 64k as that will
+        // form the clustering in the index table
+        ByteBuffer a = ByteBuffer.allocate(100);
+        ByteBuffer b = ByteBuffer.allocate(FBUtilities.MAX_UNSIGNED_SHORT - 100);
+
+        failInsert("UPDATE %s SET b[?] = 0 WHERE a = ?", b, a);
+    }
+
+    @Test
+    public void testIndexOnCollectionKeyInsertPartitionKeyAndClusteringOver64k() throws Throwable
+    {
+        createTable("CREATE TABLE %s(a blob, b blob, c map<blob, int>, PRIMARY KEY (a, b))");
+        createIndex("CREATE INDEX ON %s(KEYS(c))");
+
+        // Basically the same as the case with non-collection clustering
+        // CompositeIndexOnCollectionKeyy creates index entries composed of the
+        // PK plus all of the clustering columns from the primary row, except the
+        // collection element - which becomes the partition key in the index table
+        ByteBuffer a = ByteBuffer.allocate(100);
+        ByteBuffer b = ByteBuffer.allocate(FBUtilities.MAX_UNSIGNED_SHORT - 100);
+        ByteBuffer c = ByteBuffer.allocate(10);
+
+        failInsert("UPDATE %s SET c[?] = 0 WHERE a = ? and b = ?", c, a, b);
+    }
+
+    @Test
+    public void testIndexOnPartitionKeyInsertValueOver64k() throws Throwable
+    {
+        createTable("CREATE TABLE %s(a int, b int, c blob, PRIMARY KEY ((a, b)))");
+        createIndex("CREATE INDEX ON %s(a)");
+        succeedInsert("INSERT INTO %s (a, b, c) VALUES (0, 0, ?)", ByteBuffer.allocate(TOO_BIG));
+    }
+
+    @Test
+    public void testIndexOnClusteringColumnInsertValueOver64k() throws Throwable
+    {
+        createTable("CREATE TABLE %s(a int, b int, c blob, PRIMARY KEY (a, b))");
+        createIndex("CREATE INDEX ON %s(b)");
+        succeedInsert("INSERT INTO %s (a, b, c) VALUES (0, 0, ?)", ByteBuffer.allocate(TOO_BIG));
+    }
+
+    @Test
+    public void testIndexOnFullCollectionEntryInsertCollectionValueOver64k() throws Throwable
+    {
+        createTable("CREATE TABLE %s(a int, b frozen<map<int, blob>>, PRIMARY KEY (a))");
+        createIndex("CREATE INDEX ON %s(full(b))");
+        Map<Integer, ByteBuffer> map = new HashMap();
+        map.put(0, ByteBuffer.allocate(1024 * 65));
+        failInsert("INSERT INTO %s (a, b) VALUES (0, ?)", map);
+    }
+
+    public void failInsert(String insertCQL, Object...args) throws Throwable
+    {
+        try
+        {
+            execute(insertCQL, args);
+            fail("Expected statement to fail validation");
+        }
+        catch (Exception e)
+        {
+            // as expected
+        }
+    }
+
+    public void succeedInsert(String insertCQL, Object...args) throws Throwable
+    {
+        execute(insertCQL, args);
+        flush();
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.clustering_indexing_test()
+     */
+    @Test
+    public void testIndexesOnClustering() throws Throwable
+    {
+        createTable("CREATE TABLE %s ( id1 int, id2 int, author text, time bigint, v1 text, v2 text, PRIMARY KEY ((id1, id2), author, time))");
+
+        createIndex("CREATE INDEX ON %s (time)");
+        execute("CREATE INDEX ON %s (id2)");
+
+        execute("INSERT INTO %s (id1, id2, author, time, v1, v2) VALUES(0, 0, 'bob', 0, 'A', 'A')");
+        execute("INSERT INTO %s (id1, id2, author, time, v1, v2) VALUES(0, 0, 'bob', 1, 'B', 'B')");
+        execute("INSERT INTO %s (id1, id2, author, time, v1, v2) VALUES(0, 1, 'bob', 2, 'C', 'C')");
+        execute("INSERT INTO %s (id1, id2, author, time, v1, v2) VALUES(0, 0, 'tom', 0, 'D', 'D')");
+        execute("INSERT INTO %s (id1, id2, author, time, v1, v2) VALUES(0, 1, 'tom', 1, 'E', 'E')");
+
+        assertRows(execute("SELECT v1 FROM %s WHERE time = 1"),
+                   row("B"), row("E"));
+
+        assertRows(execute("SELECT v1 FROM %s WHERE id2 = 1"),
+                   row("C"), row("E"));
+
+        assertRows(execute("SELECT v1 FROM %s WHERE id1 = 0 AND id2 = 0 AND author = 'bob' AND time = 0"),
+                   row("A"));
+
+        // Test for CASSANDRA-8206
+        execute("UPDATE %s SET v2 = null WHERE id1 = 0 AND id2 = 0 AND author = 'bob' AND time = 1");
+
+        assertRows(execute("SELECT v1 FROM %s WHERE id2 = 0"),
+                   row("A"), row("B"), row("D"));
+
+        assertRows(execute("SELECT v1 FROM %s WHERE time = 1"),
+                   row("B"), row("E"));
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.invalid_clustering_indexing_test()
+     */
+    @Test
+    public void testIndexesOnClusteringInvalid() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int, b int, c int, d int, PRIMARY KEY ((a, b))) WITH COMPACT STORAGE");
+        assertInvalid("CREATE INDEX ON %s (a)");
+        assertInvalid("CREATE INDEX ON %s (b)");
+
+        createTable("CREATE TABLE %s (a int, b int, c int, PRIMARY KEY (a, b)) WITH COMPACT STORAGE");
+        assertInvalid("CREATE INDEX ON %s (a)");
+        assertInvalid("CREATE INDEX ON %s (b)");
+        assertInvalid("CREATE INDEX ON %s (c)");
+
+        createTable("CREATE TABLE %s (a int, b int, c int static , PRIMARY KEY (a, b))");
+        assertInvalid("CREATE INDEX ON %s (c)");
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/01115f72/test/unit/org/apache/cassandra/cql3/validation/entities/StaticColumnsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/StaticColumnsTest.java b/test/unit/org/apache/cassandra/cql3/validation/entities/StaticColumnsTest.java
new file mode 100644
index 0000000..cef6f1f
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/StaticColumnsTest.java
@@ -0,0 +1,271 @@
+/*
+ * 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.validation.entities;
+
+import java.util.Arrays;
+
+import org.junit.Test;
+
+import org.apache.cassandra.cql3.CQLTester;
+
+import static junit.framework.Assert.assertNull;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+public class StaticColumnsTest extends CQLTester
+{
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.static_columns_test()
+     */
+    @Test
+    public void testStaticColumns() throws Throwable
+    {
+        createTable("CREATE TABLE %s ( k int, p int, s int static, v int, PRIMARY KEY (k, p))");
+
+        execute("INSERT INTO %s(k, s) VALUES (0, 42)");
+
+        assertRows(execute("SELECT * FROM %s"), row(0, null, 42, null));
+
+        // Check that writetime works (//7081) -- we can't predict the exact value easily so
+        // we just check that it's non zero
+        Object[][] row = getRows(execute("SELECT s, writetime(s) FROM %s WHERE k=0"));
+        assertEquals(42, row[0][0]);
+        assertTrue((Long)row[0][1] > 0);
+
+        execute("INSERT INTO %s (k, p, s, v) VALUES (0, 0, 12, 0)");
+        execute("INSERT INTO %s (k, p, s, v) VALUES (0, 1, 24, 1)");
+
+        // Check the static columns in indeed "static"
+        assertRows(execute("SELECT * FROM %s"), row(0, 0, 24, 0), row(0, 1, 24, 1));
+
+        // Check we do correctly get the static column value with a SELECT *, even
+        // if we're only slicing part of the partition
+        assertRows(execute("SELECT * FROM %s WHERE k=0 AND p=0"), row(0, 0, 24, 0));
+        assertRows(execute("SELECT * FROM %s WHERE k=0 AND p=1"), row(0, 1, 24, 1));
+
+        // Test for IN on the clustering key (//6769)
+        assertRows(execute("SELECT * FROM %s WHERE k=0 AND p IN (0, 1)"), row(0, 0, 24, 0), row(0, 1, 24, 1));
+
+        // Check things still work if we don't select the static column. We also want
+        // this to not request the static columns internally at all, though that part
+        // require debugging to assert
+        assertRows(execute("SELECT p, v FROM %s WHERE k=0 AND p=1"), row(1, 1));
+
+        // Check selecting only a static column with distinct only yield one value
+        // (as we only query the static columns)
+        assertRows(execute("SELECT DISTINCT s FROM %s WHERE k=0"), row(24));
+        // But without DISTINCT, we still get one result per row
+        assertRows(execute("SELECT s FROM %s WHERE k=0"),row(24),row(24));
+        // but that querying other columns does correctly yield the full partition
+        assertRows(execute("SELECT s, v FROM %s WHERE k=0"),row(24, 0),row(24, 1));
+        assertRows(execute("SELECT s, v FROM %s WHERE k=0 AND p=1"),row(24, 1));
+        assertRows(execute("SELECT p, s FROM %s WHERE k=0 AND p=1"), row(1, 24));
+        assertRows(execute("SELECT k, p, s FROM %s WHERE k=0 AND p=1"),row(0, 1, 24));
+
+        // Check that deleting a row don't implicitely deletes statics
+        execute("DELETE FROM %s WHERE k=0 AND p=0");
+        assertRows(execute("SELECT * FROM %s"),row(0, 1, 24, 1));
+
+        // But that explicitely deleting the static column does remove it
+        execute("DELETE s FROM %s WHERE k=0");
+        assertRows(execute("SELECT * FROM %s"), row(0, 1, null, 1));
+
+        // Check we can add a static column ...
+        execute("ALTER TABLE %s ADD s2 int static");
+        assertRows(execute("SELECT * FROM %s"), row(0, 1, null, null, 1));
+        execute("INSERT INTO %s (k, p, s2, v) VALUES(0, 2, 42, 2)");
+        assertRows(execute("SELECT * FROM %s"), row(0, 1, null, 42, 1), row(0, 2, null, 42, 2));
+        // ... and that we can drop it
+        execute("ALTER TABLE %s DROP s2");
+        assertRows(execute("SELECT * FROM %s"), row(0, 1, null, 1), row(0, 2, null, 2));
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.static_columns_with_2i_test()
+     */
+    @Test
+    public void testStaticColumnsWithSecondaryIndex() throws Throwable
+    {
+        createTable(" CREATE TABLE %s (k int, p int, s int static, v int, PRIMARY KEY (k, p) ) ");
+
+        createIndex("CREATE INDEX ON %s (v)");
+
+        execute("INSERT INTO %s (k, p, s, v) VALUES (0, 0, 42, 1)");
+        execute("INSERT INTO %s (k, p, v) VALUES (0, 1, 1)");
+        execute("INSERT INTO %s (k, p, v) VALUES (0, 2, 2)");
+
+        assertRows(execute("SELECT * FROM %s WHERE v = 1"), row(0, 0, 42, 1), row(0, 1, 42, 1));
+        assertRows(execute("SELECT p, s FROM %s WHERE v = 1"), row(0, 42), row(1, 42));
+        assertRows(execute("SELECT p FROM %s WHERE v = 1"), row(0), row(1));
+        // We don't support that
+        assertInvalid("SELECT s FROM %s WHERE v = 1");
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.static_columns_with_distinct_test()
+     */
+    @Test
+    public void testStaticColumnsWithDistinct() throws Throwable
+    {
+        createTable("CREATE TABLE %s( k int, p int, s int static, PRIMARY KEY (k, p) ) ");
+
+        execute("INSERT INTO %s (k, p) VALUES (1, 1)");
+        execute("INSERT INTO %s (k, p) VALUES (1, 2)");
+
+        assertRows(execute("SELECT k, s FROM %s"), row(1, null), row(1, null));
+        assertRows(execute("SELECT DISTINCT k, s FROM %s"), row(1, null));
+
+        Object[][] rows = getRows(execute("SELECT DISTINCT s FROM %s WHERE k=1"));
+        assertNull(rows[0][0]);
+
+        assertEmpty(execute("SELECT DISTINCT s FROM %s WHERE k=2"));
+
+        execute("INSERT INTO %s (k, p, s) VALUES (2, 1, 3)");
+        execute("INSERT INTO %s (k, p) VALUES (2, 2)");
+
+        assertRows(execute("SELECT k, s FROM %s"), row(1, null), row(1, null), row(2, 3), row(2, 3));
+        assertRows(execute("SELECT DISTINCT k, s FROM %s"), row(1, null), row(2, 3));
+        rows = getRows(execute("SELECT DISTINCT s FROM %s WHERE k=1"));
+        assertNull(rows[0][0]);
+        assertRows(execute("SELECT DISTINCT s FROM %s WHERE k=2"), row(3));
+
+        assertInvalid("SELECT DISTINCT s FROM %s");
+
+        // paging to test for CASSANDRA-8108
+        execute("TRUNCATE %s");
+        for (int i = 0; i < 10; i++)
+            for (int j = 0; j < 10; j++)
+                execute("INSERT INTO %s (k, p, s) VALUES (?, ?, ?)", i, j, i);
+
+        rows = getRows(execute("SELECT DISTINCT k, s FROM %s"));
+        checkDistinctRows(rows, true, 0, 10, 0, 10);
+
+        String keys = "0, 1, 2, 3, 4, 5, 6, 7, 8, 9";
+        rows = getRows(execute("SELECT DISTINCT k, s FROM %s WHERE k IN (" + keys + ")"));
+        checkDistinctRows(rows, false, 0, 10, 0, 10);
+
+        // additional testing for CASSANRA-8087
+        createTable("CREATE TABLE %s( k int, c1 int, c2 int, s1 int static, s2 int static, PRIMARY KEY (k, c1, c2))");
+
+        for (int i = 0; i < 10; i++)
+            for (int j = 0; j < 5; j++)
+                for (int k = 0; k < 5; k++)
+                    execute("INSERT INTO %s (k, c1, c2, s1, s2) VALUES (?, ?, ?, ?, ?)", i, j, k, i, i + 1);
+
+        rows = getRows(execute("SELECT DISTINCT k, s1 FROM %s"));
+        checkDistinctRows(rows, true, 0, 10, 0, 10);
+
+        rows = getRows(execute("SELECT DISTINCT k, s2 FROM %s"));
+        checkDistinctRows(rows, true, 0, 10, 1, 11);
+
+        rows = getRows(execute("SELECT DISTINCT k, s1 FROM %s LIMIT 10"));
+        checkDistinctRows(rows, true, 0, 10, 0, 10);
+
+        rows = getRows(execute("SELECT DISTINCT k, s1 FROM %s WHERE k IN (" + keys + ")"));
+        checkDistinctRows(rows, false, 0, 10, 0, 10);
+
+        rows = getRows(execute("SELECT DISTINCT k, s2 FROM %s WHERE k IN (" + keys + ")"));
+        checkDistinctRows(rows, false, 0, 10, 1, 11);
+
+        rows = getRows(execute("SELECT DISTINCT k, s1 FROM %s WHERE k IN (" + keys + ")"));
+        checkDistinctRows(rows, true, 0, 10, 0, 10);
+    }
+
+    void checkDistinctRows(Object[][] rows, boolean sort, int... ranges)
+    {
+        assertTrue(ranges.length % 2 == 0);
+
+        int numdim = ranges.length / 2;
+        int[] from = new int[numdim];
+        int[] to = new int[numdim];
+
+        for (int i = 0, j = 0; i < ranges.length && j < numdim; i+= 2, j++)
+        {
+            from[j] = ranges[i];
+            to[j] = ranges[i+1];
+        }
+
+        //sort the rows
+        for (int i = 0; i < numdim; i++)
+        {
+            int[] vals = new int[rows.length];
+            for (int j = 0; j < rows.length; j++)
+                vals[j] = (Integer)rows[j][i];
+
+            if (sort)
+                Arrays.sort(vals);
+
+            for (int j = from[i]; j < to[i]; j++)
+                assertEquals(j, vals[j - from[i]]);
+        }
+    }
+
+    /**
+     * Test LIMIT when static columns are present (#6956),
+     * migrated from cql_tests.py:TestCQL.static_with_limit_test()
+     */
+    @Test
+    public void testStaticColumnsWithLimit() throws Throwable
+    {
+        createTable(" CREATE TABLE %s (k int, s int static, v int, PRIMARY KEY (k, v))");
+
+        execute("INSERT INTO %s (k, s) VALUES(0, 42)");
+        for (int i = 0; i < 4; i++)
+            execute("INSERT INTO %s(k, v) VALUES(0, ?)", i);
+
+        assertRows(execute("SELECT * FROM %s WHERE k = 0 LIMIT 1"),
+                   row(0, 0, 42));
+        assertRows(execute("SELECT * FROM %s WHERE k = 0 LIMIT 2"),
+                   row(0, 0, 42),
+                   row(0, 1, 42));
+        assertRows(execute("SELECT * FROM %s WHERE k = 0 LIMIT 3"),
+                   row(0, 0, 42),
+                   row(0, 1, 42),
+                   row(0, 2, 42));
+    }
+
+    /**
+     * Test for bug of #7455,
+     * migrated from cql_tests.py:TestCQL.static_with_empty_clustering_test()
+     */
+    @Test
+    public void testStaticColumnsWithEmptyClustering() throws Throwable
+    {
+        createTable("CREATE TABLE %s (pkey text, ckey text, value text, static_value text static, PRIMARY KEY(pkey, ckey))");
+
+        execute("INSERT INTO %s (pkey, static_value) VALUES ('partition1', 'static value')");
+        execute("INSERT INTO %s (pkey, ckey, value) VALUES('partition1', '', 'value')");
+
+        assertRows(execute("SELECT * FROM %s"),
+                   row("partition1", "", "static value", "value"));
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.alter_clustering_and_static_test()
+     */
+    @Test
+    public void testAlterClusteringAndStatic() throws Throwable
+    {
+        createTable("CREATE TABLE %s (bar int, PRIMARY KEY (bar))");
+
+        // We shouldn 't allow static when there is not clustering columns
+        assertInvalid("ALTER TABLE %s ADD bar2 text static");
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/01115f72/test/unit/org/apache/cassandra/cql3/validation/entities/TimestampTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/TimestampTest.java b/test/unit/org/apache/cassandra/cql3/validation/entities/TimestampTest.java
new file mode 100644
index 0000000..3e70cd0
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/TimestampTest.java
@@ -0,0 +1,155 @@
+/*
+ * 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.validation.entities;
+
+import org.junit.Test;
+
+import junit.framework.Assert;
+import org.apache.cassandra.cql3.CQLTester;
+
+import static junit.framework.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+public class TimestampTest extends CQLTester
+{
+    @Test
+    public void testNegativeTimestamps() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, v int)");
+
+        execute("INSERT INTO %s (k, v) VALUES (?, ?) USING TIMESTAMP ?", 1, 1, -42L);
+        assertRows(execute("SELECT writetime(v) FROM %s WHERE k = ?", 1),
+            row(-42L)
+        );
+
+        assertInvalid("INSERT INTO %s (k, v) VALUES (?, ?) USING TIMESTAMP ?", 2, 2, Long.MIN_VALUE);
+    }
+
+    /**
+     * Test timestmp and ttl
+     * migrated from cql_tests.py:TestCQL.timestamp_and_ttl_test()
+     */
+    @Test
+    public void testTimestampTTL() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, c text, d text)");
+
+        execute("INSERT INTO %s (k, c) VALUES (1, 'test')");
+        execute("INSERT INTO %s (k, c) VALUES (2, 'test') USING TTL 400");
+
+        Object[][] res = getRows(execute("SELECT k, c, writetime(c), ttl(c) FROM %s"));
+        Assert.assertEquals(2, res.length);
+
+        for (Object[] r : res)
+        {
+            assertTrue(r[2] instanceof Integer || r[2] instanceof Long);
+            if (r[0].equals(1))
+                assertNull(r[3]);
+            else
+                assertTrue(r[3] instanceof Integer || r[2] instanceof Long);
+        }
+
+
+        // wrap writetime(), ttl() in other functions (test for CASSANDRA-8451)
+        res = getRows(execute("SELECT k, c, blobAsBigint(bigintAsBlob(writetime(c))), ttl(c) FROM %s"));
+        Assert.assertEquals(2, res.length);
+
+        for (Object[] r : res)
+        {
+            assertTrue(r[2] instanceof Integer || r[2] instanceof Long);
+            if (r[0].equals(1))
+                assertNull(r[3]);
+            else
+                assertTrue(r[3] instanceof Integer || r[2] instanceof Long);
+        }
+
+        res = getRows(execute("SELECT k, c, writetime(c), blobAsInt(intAsBlob(ttl(c))) FROM %s"));
+        Assert.assertEquals(2, res.length);
+
+
+        for (Object[] r : res)
+        {
+            assertTrue(r[2] instanceof Integer || r[2] instanceof Long);
+            if (r[0].equals(1))
+                assertNull(r[3]);
+            else
+                assertTrue(r[3] instanceof Integer || r[2] instanceof Long);
+        }
+
+        assertInvalid("SELECT k, c, writetime(k) FROM %s");
+
+        assertRows(execute("SELECT k, d, writetime(d) FROM %s WHERE k = 1"),
+                   row(1, null, null));
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.invalid_custom_timestamp_test()
+     */
+    @Test
+    public void testInvalidCustomTimestamp() throws Throwable
+    {
+        // Conditional updates
+        createTable("CREATE TABLE %s (k int, v int, PRIMARY KEY (k, v))");
+
+        execute("BEGIN BATCH " +
+                "INSERT INTO %1$s (k, v) VALUES(0, 0) IF NOT EXISTS; " +
+                "INSERT INTO %1$s (k, v) VALUES(0, 1) IF NOT EXISTS; " +
+                "APPLY BATCH");
+
+        assertInvalid("BEGIN BATCH " +
+                      "INSERT INTO %1$s (k, v) VALUES(0, 2) IF NOT EXISTS USING TIMESTAMP 1; " +
+                      "INSERT INTO %1$s (k, v) VALUES(0, 3) IF NOT EXISTS; " +
+                      "APPLY BATCH");
+        assertInvalid("BEGIN BATCH " +
+                      "USING TIMESTAMP 1 INSERT INTO %1$s (k, v) VALUES(0, 4) IF NOT EXISTS; " +
+                      "INSERT INTO %1$s (k, v) VALUES(0, 1) IF NOT EXISTS; " +
+                      "APPLY BATCH");
+
+        execute("INSERT INTO %s (k, v) VALUES(1, 0) IF NOT EXISTS");
+        assertInvalid("INSERT INTO %s (k, v) VALUES(1, 1) IF NOT EXISTS USING TIMESTAMP 5");
+
+        // Counters
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, c counter)");
+
+        execute("UPDATE %s SET c = c + 1 WHERE k = 0");
+        assertInvalid("UPDATE %s USING TIMESTAMP 10 SET c = c + 1 WHERE k = 0");
+
+        execute("BEGIN COUNTER BATCH " +
+                "UPDATE %1$s SET c = c + 1 WHERE k = 0; " +
+                "UPDATE %1$s SET c = c + 1 WHERE k = 0; " +
+                "APPLY BATCH");
+
+        assertInvalid("BEGIN COUNTER BATCH " +
+                      "UPDATE %1$s USING TIMESTAMP 3 SET c = c + 1 WHERE k = 0; " +
+                      "UPDATE %1$s SET c = c + 1 WHERE k = 0; " +
+                      "APPLY BATCH");
+
+        assertInvalid("BEGIN COUNTER BATCH " +
+                      "USING TIMESTAMP 3 UPDATE %1$s SET c = c + 1 WHERE k = 0; " +
+                      "UPDATE %1$s SET c = c + 1 WHERE k = 0; " +
+                      "APPLY BATCH");
+    }
+
+    @Test
+    public void testInsertTimestampWithUnset() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, i int)");
+        execute("INSERT INTO %s (k, i) VALUES (1, 1) USING TIMESTAMP ?", unset()); // treat as 'now'
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/01115f72/test/unit/org/apache/cassandra/cql3/validation/entities/TimeuuidTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/TimeuuidTest.java b/test/unit/org/apache/cassandra/cql3/validation/entities/TimeuuidTest.java
new file mode 100644
index 0000000..0f1f8f0
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/TimeuuidTest.java
@@ -0,0 +1,81 @@
+/*
+ * 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.validation.entities;
+
+import java.util.Date;
+import java.util.UUID;
+
+import org.junit.Test;
+
+import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.exceptions.SyntaxException;
+import org.apache.cassandra.utils.UUIDGen;
+
+import static org.junit.Assert.assertEquals;
+
+public class TimeuuidTest extends CQLTester
+{
+    /**
+     * Migrated from cql_tests.py:TestCQL.timeuuid_test()
+     */
+    @Test
+    public void testTimeuuid() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int, t timeuuid, PRIMARY KEY(k, t))");
+
+        assertInvalidThrow(SyntaxException.class, "INSERT INTO %s (k, t) VALUES (0, 2012-11-07 18:18:22-0800)");
+
+        for (int i = 0; i < 4; i++)
+            execute("INSERT INTO %s (k, t) VALUES (0, now())");
+
+        Object[][] rows = getRows(execute("SELECT * FROM %s"));
+        assertEquals(4, rows.length);
+
+        assertRowCount(execute("SELECT * FROM %s WHERE k = 0 AND t >= ?", rows[0][1]), 4);
+
+        assertEmpty(execute("SELECT * FROM %s WHERE k = 0 AND t < ?", rows[0][1]));
+
+        assertRowCount(execute("SELECT * FROM %s WHERE k = 0 AND t > ? AND t <= ?", rows[0][1], rows[2][1]), 2);
+
+        assertRowCount(execute("SELECT * FROM %s WHERE k = 0 AND t = ?", rows[0][1]), 1);
+
+        assertInvalid("SELECT dateOf(k) FROM %s WHERE k = 0 AND t = ?", rows[0][1]);
+
+        for (int i = 0; i < 4; i++)
+        {
+            long timestamp = UUIDGen.unixTimestamp((UUID) rows[i][1]);
+            assertRows(execute("SELECT dateOf(t), unixTimestampOf(t) FROM %s WHERE k = 0 AND t = ?", rows[i][1]),
+                       row(new Date(timestamp), timestamp));
+        }
+
+        assertEmpty(execute("SELECT t FROM %s WHERE k = 0 AND t > maxTimeuuid(1234567) AND t < minTimeuuid('2012-11-07 18:18:22-0800')"));
+    }
+
+    /**
+     * Test for 5386,
+     * migrated from cql_tests.py:TestCQL.function_and_reverse_type_test()
+     */
+    @Test
+    public void testDescClusteringOnTimeuuid() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int, c timeuuid, v int, PRIMARY KEY (k, c)) WITH CLUSTERING ORDER BY (c DESC)");
+
+        execute("INSERT INTO %s (k, c, v) VALUES (0, now(), 0)");
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/01115f72/test/unit/org/apache/cassandra/cql3/validation/entities/TupleTypeTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/TupleTypeTest.java b/test/unit/org/apache/cassandra/cql3/validation/entities/TupleTypeTest.java
new file mode 100644
index 0000000..362756b
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/TupleTypeTest.java
@@ -0,0 +1,171 @@
+/*
+ * 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.validation.entities;
+
+import org.junit.Test;
+
+import org.apache.cassandra.cql3.CQLTester;
+
+public class TupleTypeTest extends CQLTester
+{
+    @Test
+    public void testTuplePutAndGet() throws Throwable
+    {
+        String[] valueTypes = {"frozen<tuple<int, text, double>>", "tuple<int, text, double>"};
+        for (String valueType : valueTypes)
+        {
+            createTable("CREATE TABLE %s (k int PRIMARY KEY, t " + valueType + ")");
+
+            execute("INSERT INTO %s (k, t) VALUES (?, ?)", 0, tuple(3, "foo", 3.4));
+            execute("INSERT INTO %s (k, t) VALUES (?, ?)", 1, tuple(8, "bar", 0.2));
+            assertAllRows(row(1, tuple(8, "bar", 0.2)),
+                          row(0, tuple(3, "foo", 3.4))
+            );
+
+            // nulls
+            execute("INSERT INTO %s (k, t) VALUES (?, ?)", 2, tuple(5, null, 3.4));
+            assertRows(execute("SELECT * FROM %s WHERE k=?", 2),
+                       row(2, tuple(5, null, 3.4))
+            );
+
+            // incomplete tuple
+            execute("INSERT INTO %s (k, t) VALUES (?, ?)", 3, tuple(5, "bar"));
+            assertRows(execute("SELECT * FROM %s WHERE k=?", 3),
+                       row(3, tuple(5, "bar"))
+            );
+        }
+    }
+
+    @Test
+    public void testNestedTuple() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, t frozen<tuple<int, tuple<text, double>>>)");
+
+        execute("INSERT INTO %s (k, t) VALUES (?, ?)", 0, tuple(3, tuple("foo", 3.4)));
+        execute("INSERT INTO %s (k, t) VALUES (?, ?)", 1, tuple(8, tuple("bar", 0.2)));
+        assertAllRows(
+            row(1, tuple(8, tuple("bar", 0.2))),
+            row(0, tuple(3, tuple("foo", 3.4)))
+        );
+    }
+
+    @Test
+    public void testTupleInPartitionKey() throws Throwable
+    {
+        createTable("CREATE TABLE %s (t frozen<tuple<int, text>> PRIMARY KEY)");
+
+        execute("INSERT INTO %s (t) VALUES (?)", tuple(3, "foo"));
+        assertAllRows(row(tuple(3, "foo")));
+    }
+
+    @Test
+    public void testTupleInClusteringKey() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int, t frozen<tuple<int, text>>, PRIMARY KEY (k, t))");
+
+        execute("INSERT INTO %s (k, t) VALUES (?, ?)", 0, tuple(5, "bar"));
+        execute("INSERT INTO %s (k, t) VALUES (?, ?)", 0, tuple(3, "foo"));
+        execute("INSERT INTO %s (k, t) VALUES (?, ?)", 0, tuple(6, "bar"));
+        execute("INSERT INTO %s (k, t) VALUES (?, ?)", 0, tuple(5, "foo"));
+
+        assertAllRows(
+            row(0, tuple(3, "foo")),
+            row(0, tuple(5, "bar")),
+            row(0, tuple(5, "foo")),
+            row(0, tuple(6, "bar"))
+        );
+    }
+
+    @Test
+    public void testInvalidQueries() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, t frozen<tuple<int, text, double>>)");
+
+        assertInvalidSyntax("INSERT INTO %s (k, t) VALUES (0, ())");
+        assertInvalid("INSERT INTO %s (k, t) VALUES (0, (2, 'foo', 3.1, 'bar'))");
+    }
+
+    @Test
+    public void testTupleWithUnsetValues() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, t tuple<int, text, double>)");
+        // invalid positional field substitution
+        assertInvalidMessage("Invalid unset value for tuple field number 1",
+                             "INSERT INTO %s (k, t) VALUES(0, (3, ?, 2.1))", unset());
+
+        createIndex("CREATE INDEX tuple_index ON %s (t)");
+        // select using unset
+        assertInvalidMessage("Invalid unset value for tuple field number 0", "SELECT * FROM %s WHERE k = ? and t = (?,?,?)", unset(), unset(), unset(), unset());
+    }
+	
+    /**
+     * Test the syntax introduced by #4851,
+     * migrated from cql_tests.py:TestCQL.tuple_notation_test()
+     */
+    @Test
+    public void testTupleNotation() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int, v1 int, v2 int, v3 int, PRIMARY KEY (k, v1, v2, v3))");
+        for (int i = 0; i < 2; i++)
+            for (int j = 0; j < 2; j++)
+                for (int k = 0; k < 2; k++)
+                    execute("INSERT INTO %s (k, v1, v2, v3) VALUES (0, ?, ?, ?)", i, j, k);
+
+        assertRows(execute("SELECT v1, v2, v3 FROM %s WHERE k = 0"),
+                   row(0, 0, 0),
+                   row(0, 0, 1),
+                   row(0, 1, 0),
+                   row(0, 1, 1),
+                   row(1, 0, 0),
+                   row(1, 0, 1),
+                   row(1, 1, 0),
+                   row(1, 1, 1));
+
+        assertRows(execute("SELECT v1, v2, v3 FROM %s WHERE k = 0 AND (v1, v2, v3) >= (1, 0, 1)"),
+                   row(1, 0, 1),
+                   row(1, 1, 0),
+                   row(1, 1, 1));
+        assertRows(execute("SELECT v1, v2, v3 FROM %s WHERE k = 0 AND (v1, v2) >= (1, 1)"),
+                   row(1, 1, 0),
+                   row(1, 1, 1));
+
+        assertRows(execute("SELECT v1, v2, v3 FROM %s WHERE k = 0 AND (v1, v2) > (0, 1) AND (v1, v2, v3) <= (1, 1, 0)"),
+                   row(1, 0, 0),
+                   row(1, 0, 1),
+                   row(1, 1, 0));
+
+        assertInvalid("SELECT v1, v2, v3 FROM %s WHERE k = 0 AND (v1, v3) > (1, 0)");
+    }
+
+    /**
+     * Test for CASSANDRA-8062,
+     * migrated from cql_tests.py:TestCQL.test_v2_protocol_IN_with_tuples()
+     */
+    @Test
+    public void testSelectInStatementWithTuples() throws Throwable
+    {   // TODO - the dtest was using v2 protocol
+        createTable("CREATE TABLE %s (k int, c1 int, c2 text, PRIMARY KEY (k, c1, c2))");
+        execute("INSERT INTO %s (k, c1, c2) VALUES (0, 0, 'a')");
+        execute("INSERT INTO %s (k, c1, c2) VALUES (0, 0, 'b')");
+        execute("INSERT INTO %s (k, c1, c2) VALUES (0, 0, 'c')");
+
+        assertRows(execute("SELECT * FROM %s WHERE k=0 AND (c1, c2) IN ((0, 'b'), (0, 'c'))"),
+                   row(0, 0, "b"),
+                   row(0, 0, "c"));
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/01115f72/test/unit/org/apache/cassandra/cql3/validation/entities/TypeTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/TypeTest.java b/test/unit/org/apache/cassandra/cql3/validation/entities/TypeTest.java
new file mode 100644
index 0000000..f3aa22b
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/TypeTest.java
@@ -0,0 +1,92 @@
+/*
+ * 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.validation.entities;
+
+import org.apache.cassandra.cql3.UntypedResultSet;
+import org.apache.cassandra.cql3.CQLTester;
+
+import org.junit.Test;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+public class TypeTest extends CQLTester
+{
+    @Test
+    public void testNowToUUIDCompatibility() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int, b uuid, PRIMARY KEY (a, b))");
+        execute("INSERT INTO %s (a, b) VALUES (0, now())");
+        UntypedResultSet results = execute("SELECT * FROM %s WHERE a=0 AND b < now()");
+        assertEquals(1, results.size());
+    }
+
+    @Test
+    public void testDateCompatibility() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int, b timestamp, c bigint, d varint, PRIMARY KEY (a, b, c, d))");
+
+        execute("INSERT INTO %s (a, b, c, d) VALUES (0, toUnixTimestamp(now()), toTimestamp(now()), toTimestamp(now()))");
+        UntypedResultSet results = execute("SELECT * FROM %s WHERE a=0 AND b < toUnixTimestamp(now())");
+        assertEquals(1, results.size());
+
+        execute("INSERT INTO %s (a, b, c, d) VALUES (1, unixTimestampOf(now()), dateOf(now()), dateOf(now()))");
+        results = execute("SELECT * FROM %s WHERE a=1 AND b < toUnixTimestamp(now())");
+        assertEquals(1, results.size());
+    }
+
+    @Test
+    public void testReversedTypeCompatibility() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int, b timeuuid, PRIMARY KEY (a, b)) WITH CLUSTERING ORDER BY (b DESC)");
+        execute("INSERT INTO %s (a, b) VALUES (0, now())");
+        UntypedResultSet results = execute("SELECT * FROM %s WHERE a=0 AND b < now()");
+        assertEquals(1, results.size());
+    }
+
+    @Test
+    // tests CASSANDRA-7797
+    public void testAlterReversedColumn() throws Throwable
+    {
+        createTable("CREATE TABLE IF NOT EXISTS %s (a int, b 'org.apache.cassandra.db.marshal.DateType', PRIMARY KEY (a, b)) WITH CLUSTERING ORDER BY (b DESC)");
+        alterTable("ALTER TABLE %s ALTER b TYPE 'org.apache.cassandra.db.marshal.ReversedType(org.apache.cassandra.db.marshal.TimestampType)'");
+    }
+
+    @Test
+    public void testIncompatibleReversedTypes() throws Throwable
+    {
+        createTable("CREATE TABLE IF NOT EXISTS %s (a int, b 'org.apache.cassandra.db.marshal.DateType', PRIMARY KEY (a, b)) WITH CLUSTERING ORDER BY (b DESC)");
+        try
+        {
+            alterTable("ALTER TABLE %s ALTER b TYPE 'org.apache.cassandra.db.marshal.ReversedType(org.apache.cassandra.db.marshal.TimeUUIDType)'");
+            fail("Expected error for ALTER statement");
+        }
+        catch (RuntimeException e) { }
+    }
+
+    @Test
+    public void testReversedAndNonReversed() throws Throwable
+    {
+        createTable("CREATE TABLE IF NOT EXISTS %s (a int, b 'org.apache.cassandra.db.marshal.DateType', PRIMARY KEY (a, b))");
+        try
+        {
+            alterTable("ALTER TABLE %s ALTER b TYPE 'org.apache.cassandra.db.marshal.ReversedType(org.apache.cassandra.db.marshal.DateType)'");
+            fail("Expected error for ALTER statement");
+        }
+        catch (RuntimeException e) { }
+    }
+}


[29/32] cassandra git commit: 2.2 commit for CASSANDRA-9160

Posted by jm...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/01115f72/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 d47b9d2..4b4631e 100644
--- a/test/unit/org/apache/cassandra/cql3/CQLTester.java
+++ b/test/unit/org/apache/cassandra/cql3/CQLTester.java
@@ -35,11 +35,18 @@ import org.junit.*;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.datastax.driver.core.*;
+import static junit.framework.Assert.assertNotNull;
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ColumnDefinitions;
+import com.datastax.driver.core.DataType;
+import com.datastax.driver.core.ProtocolVersion;
 import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Row;
+import com.datastax.driver.core.Session;
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.concurrent.ScheduledExecutors;
 import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.cql3.functions.FunctionName;
 import org.apache.cassandra.cql3.statements.ParsedStatement;
@@ -51,6 +58,7 @@ import org.apache.cassandra.db.marshal.TupleType;
 import org.apache.cassandra.exceptions.CassandraException;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.exceptions.SyntaxException;
+import org.apache.cassandra.dht.Murmur3Partitioner;
 import org.apache.cassandra.io.util.FileUtils;
 import org.apache.cassandra.serializers.TypeSerializer;
 import org.apache.cassandra.service.ClientState;
@@ -71,6 +79,7 @@ public abstract class CQLTester
     public static final String KEYSPACE = "cql_test_keyspace";
     public static final String KEYSPACE_PER_TEST = "cql_test_keyspace_alt";
     protected static final boolean USE_PREPARED_VALUES = Boolean.valueOf(System.getProperty("cassandra.test.use_prepared", "true"));
+    protected static final long ROW_CACHE_SIZE_IN_MB = Integer.valueOf(System.getProperty("cassandra.test.row_cache_size_in_mb", "0"));
     private static final AtomicInteger seqNumber = new AtomicInteger();
 
     private static org.apache.cassandra.transport.Server server;
@@ -79,7 +88,7 @@ public abstract class CQLTester
     private static final Cluster[] cluster;
     private static final Session[] session;
 
-    static int maxProtocolVersion;
+    public static int maxProtocolVersion;
     static {
         int version;
         for (version = 1; version <= Server.CURRENT_VERSION; )
@@ -129,9 +138,12 @@ public abstract class CQLTester
     private boolean usePrepared = USE_PREPARED_VALUES;
 
     @BeforeClass
-    public static void setUpClass() throws Throwable
+    public static void setUpClass()
     {
-        schemaChange(String.format("CREATE KEYSPACE IF NOT EXISTS %s WITH replication = {'class': 'SimpleStrategy', 'replication_factor': '1'}", KEYSPACE));
+        if (ROW_CACHE_SIZE_IN_MB > 0)
+            DatabaseDescriptor.setRowCacheSizeInMB(ROW_CACHE_SIZE_IN_MB);
+
+        DatabaseDescriptor.setPartitioner(Murmur3Partitioner.instance);
     }
 
     @AfterClass
@@ -151,6 +163,7 @@ public abstract class CQLTester
     @Before
     public void beforeTest() throws Throwable
     {
+        schemaChange(String.format("CREATE KEYSPACE IF NOT EXISTS %s WITH replication = {'class': 'SimpleStrategy', 'replication_factor': '1'}", KEYSPACE));
         schemaChange(String.format("CREATE KEYSPACE IF NOT EXISTS %s WITH replication = {'class': 'SimpleStrategy', 'replication_factor': '1'}", KEYSPACE_PER_TEST));
     }
 
@@ -178,16 +191,16 @@ public abstract class CQLTester
             {
                 try
                 {
-                    for (int i = tablesToDrop.size() - 1; i >=0; i--)
+                    for (int i = tablesToDrop.size() - 1; i >= 0; i--)
                         schemaChange(String.format("DROP TABLE IF EXISTS %s.%s", KEYSPACE, tablesToDrop.get(i)));
 
-                    for (int i = aggregatesToDrop.size() - 1; i >=0; i--)
+                    for (int i = aggregatesToDrop.size() - 1; i >= 0; i--)
                         schemaChange(String.format("DROP AGGREGATE IF EXISTS %s", aggregatesToDrop.get(i)));
 
-                    for (int i = functionsToDrop.size() - 1; i >=0; i--)
+                    for (int i = functionsToDrop.size() - 1; i >= 0; i--)
                         schemaChange(String.format("DROP FUNCTION IF EXISTS %s", functionsToDrop.get(i)));
 
-                    for (int i = typesToDrop.size() - 1; i >=0; i--)
+                    for (int i = typesToDrop.size() - 1; i >= 0; i--)
                         schemaChange(String.format("DROP TYPE IF EXISTS %s.%s", KEYSPACE, typesToDrop.get(i)));
 
                     // Dropping doesn't delete the sstables. It's not a huge deal but it's cleaner to cleanup after us
@@ -198,10 +211,10 @@ public abstract class CQLTester
                     final CountDownLatch latch = new CountDownLatch(1);
                     ScheduledExecutors.nonPeriodicTasks.execute(new Runnable()
                     {
-                            public void run()
-                            {
-                                latch.countDown();
-                            }
+                        public void run()
+                        {
+                            latch.countDown();
+                        }
                     });
                     latch.await(2, TimeUnit.SECONDS);
 
@@ -272,9 +285,25 @@ public abstract class CQLTester
         }
     }
 
-    public boolean usePrepared()
+    public void compact()
     {
-        return USE_PREPARED_VALUES;
+        try
+        {
+            String currentTable = currentTable();
+            if (currentTable != null)
+                Keyspace.open(KEYSPACE).getColumnFamilyStore(currentTable).forceMajorCompaction();
+        }
+        catch (InterruptedException | ExecutionException e)
+        {
+            throw new RuntimeException(e);
+        }
+    }
+
+    public void cleanupCache()
+    {
+        String currentTable = currentTable();
+        if (currentTable != null)
+            Keyspace.open(KEYSPACE).getColumnFamilyStore(currentTable).cleanupCache();
     }
 
     public static FunctionName parseFunctionName(String qualifiedName)
@@ -379,14 +408,20 @@ public abstract class CQLTester
 
     protected String createTable(String query)
     {
-        String currentTable = "table_" + seqNumber.getAndIncrement();
-        tables.add(currentTable);
+        String currentTable = createTableName();
         String fullQuery = formatQuery(query);
         logger.info(fullQuery);
         schemaChange(fullQuery);
         return currentTable;
     }
 
+    protected String createTableName()
+    {
+        String currentTable = "table_" + seqNumber.getAndIncrement();
+        tables.add(currentTable);
+        return currentTable;
+    }
+
     protected void createTableMayThrow(String query) throws Throwable
     {
         String currentTable = "table_" + seqNumber.getAndIncrement();
@@ -424,6 +459,37 @@ public abstract class CQLTester
         schemaChange(fullQuery);
     }
 
+    /**
+     * Index creation is asynchronous, this method searches in the system table IndexInfo
+     * for the specified index and returns true if it finds it, which indicates the
+     * index was built. If we haven't found it after 5 seconds we give-up.
+     */
+    protected boolean waitForIndex(String keyspace, String table, String index) throws Throwable
+    {
+        long start = System.currentTimeMillis();
+        boolean indexCreated = false;
+        String indedName = String.format("%s.%s", table, index);
+        while (!indexCreated)
+        {
+            Object[][] results = getRows(execute("select index_name from system.\"IndexInfo\" where table_name = ?", keyspace));
+            for(int i = 0; i < results.length; i++)
+            {
+                if (indedName.equals(results[i][0]))
+                {
+                    indexCreated = true;
+                    break;
+                }
+            }
+
+            if (System.currentTimeMillis() - start > 5000)
+                break;
+
+            Thread.sleep(10);
+        }
+
+        return indexCreated;
+    }
+
     protected void createIndexMayThrow(String query) throws Throwable
     {
         String fullQuery = formatQuery(query);
@@ -593,7 +659,7 @@ public abstract class CQLTester
             Object[] expected = rows[i];
             UntypedResultSet.Row actual = iter.next();
 
-            Assert.assertEquals(String.format("Invalid number of (expected) values provided for row %d", i), meta.size(), expected.length);
+            Assert.assertEquals(String.format("Invalid number of (expected) values provided for row %d", i), expected.length, meta.size());
 
             for (int j = 0; j < meta.size(); j++)
             {
@@ -602,8 +668,17 @@ public abstract class CQLTester
                 ByteBuffer actualValue = actual.getBytes(column.name.toString());
 
                 if (!Objects.equal(expectedByteValue, actualValue))
-                    Assert.fail(String.format("Invalid value for row %d column %d (%s of type %s), expected <%s> but got <%s>",
-                                              i, j, column.name, column.type.asCQL3Type(), formatValue(expectedByteValue, column.type), formatValue(actualValue, column.type)));
+                {
+                    Object actualValueDecoded = column.type.getSerializer().deserialize(actualValue);
+                    if (!actualValueDecoded.equals(expected[j]))
+                        Assert.fail(String.format("Invalid value for row %d column %d (%s of type %s), expected <%s> but got <%s>",
+                                                  i,
+                                                  j,
+                                                  column.name,
+                                                  column.type.asCQL3Type(),
+                                                  formatValue(expectedByteValue, column.type),
+                                                  formatValue(actualValue, column.type)));
+                }
             }
             i++;
         }
@@ -621,6 +696,65 @@ public abstract class CQLTester
         Assert.assertTrue(String.format("Got %s rows than expected. Expected %d but got %d", rows.length>i ? "less" : "more", rows.length, i), i == rows.length);
     }
 
+    protected void assertRowCount(UntypedResultSet result, int numExpectedRows)
+    {
+        if (result == null)
+        {
+            if (numExpectedRows > 0)
+                Assert.fail(String.format("No rows returned by query but %d expected", numExpectedRows));
+            return;
+        }
+
+        List<ColumnSpecification> meta = result.metadata();
+        Iterator<UntypedResultSet.Row> iter = result.iterator();
+        int i = 0;
+        while (iter.hasNext() && i < numExpectedRows)
+        {
+            UntypedResultSet.Row actual = iter.next();
+            assertNotNull(actual);
+            i++;
+        }
+
+        if (iter.hasNext())
+        {
+            while (iter.hasNext())
+            {
+                iter.next();
+                i++;
+            }
+            Assert.fail(String.format("Got less rows than expected. Expected %d but got %d.", numExpectedRows, i));
+        }
+
+        Assert.assertTrue(String.format("Got %s rows than expected. Expected %d but got %d", numExpectedRows>i ? "less" : "more", numExpectedRows, i), i == numExpectedRows);
+    }
+
+    protected Object[][] getRows(UntypedResultSet result)
+    {
+        if (result == null)
+            return new Object[0][];
+
+        List<Object[]> ret = new ArrayList<>();
+        List<ColumnSpecification> meta = result.metadata();
+
+        Iterator<UntypedResultSet.Row> iter = result.iterator();
+        while (iter.hasNext())
+        {
+            UntypedResultSet.Row rowVal = iter.next();
+            Object[] row = new Object[meta.size()];
+            for (int j = 0; j < meta.size(); j++)
+            {
+                ColumnSpecification column = meta.get(j);
+                ByteBuffer val = rowVal.getBytes(column.name.toString());
+                row[j] = val == null ? null : column.type.getSerializer().deserialize(val);
+            }
+
+            ret.add(row);
+        }
+
+        Object[][] a = new Object[ret.size()][];
+        return ret.toArray(a);
+    }
+
     protected void assertColumnNames(UntypedResultSet result, String... expectedColumnNames)
     {
         if (result == null)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/01115f72/test/unit/org/apache/cassandra/cql3/CollectionsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/CollectionsTest.java b/test/unit/org/apache/cassandra/cql3/CollectionsTest.java
deleted file mode 100644
index c9c6d31..0000000
--- a/test/unit/org/apache/cassandra/cql3/CollectionsTest.java
+++ /dev/null
@@ -1,340 +0,0 @@
-/*
- * 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;
-
-import org.junit.Test;
-
-public class CollectionsTest extends CQLTester
-{
-    @Test
-    public void testMapBulkRemoval() throws Throwable
-    {
-        createTable("CREATE TABLE %s (k int PRIMARY KEY, m map<text, text>)");
-
-        execute("INSERT INTO %s(k, m) VALUES (?, ?)", 0, map("k1", "v1", "k2", "v2", "k3", "v3"));
-
-        assertRows(execute("SELECT * FROM %s"),
-            row(0, map("k1", "v1", "k2", "v2", "k3", "v3"))
-        );
-
-        execute("UPDATE %s SET m = m - ? WHERE k = ?", set("k2"), 0);
-
-        assertRows(execute("SELECT * FROM %s"),
-            row(0, map("k1", "v1", "k3", "v3"))
-        );
-
-        execute("UPDATE %s SET m = m + ?, m = m - ? WHERE k = ?", map("k4", "v4"), set("k3"), 0);
-
-        assertRows(execute("SELECT * FROM %s"),
-            row(0, map("k1", "v1", "k4", "v4"))
-        );
-    }
-
-    @Test
-    public void testInvalidCollectionsMix() throws Throwable
-    {
-        createTable("CREATE TABLE %s (k int PRIMARY KEY, l list<text>, s set<text>, m map<text, text>)");
-
-        // Note: we force the non-prepared form for some of those tests because a list and a set
-        // have the same serialized format in practice and CQLTester don't validate that the type
-        // of what's passed as a value in the prepared case, so the queries would work (which is ok,
-        // CQLTester is just a "dumb" client).
-
-        assertInvalid("UPDATE %s SET l = l + { 'a', 'b' } WHERE k = 0");
-        assertInvalid("UPDATE %s SET l = l - { 'a', 'b' } WHERE k = 0");
-        assertInvalid("UPDATE %s SET l = l + ? WHERE k = 0", map("a", "b", "c", "d"));
-        assertInvalid("UPDATE %s SET l = l - ? WHERE k = 0", map("a", "b", "c", "d"));
-
-        assertInvalid("UPDATE %s SET s = s + [ 'a', 'b' ] WHERE k = 0");
-        assertInvalid("UPDATE %s SET s = s - [ 'a', 'b' ] WHERE k = 0");
-        assertInvalid("UPDATE %s SET s = s + ? WHERE k = 0", map("a", "b", "c", "d"));
-        assertInvalid("UPDATE %s SET s = s - ? WHERE k = 0", map("a", "b", "c", "d"));
-
-        assertInvalid("UPDATE %s SET m = m + ? WHERE k = 0", list("a", "b"));
-        assertInvalid("UPDATE %s SET m = m - [ 'a', 'b' ] WHERE k = 0");
-        assertInvalid("UPDATE %s SET m = m + ? WHERE k = 0", set("a", "b"));
-        assertInvalid("UPDATE %s SET m = m - ? WHERE k = 0", map("a", "b", "c", "d"));
-    }
-
-    @Test
-    public void testSets() throws Throwable
-    {
-        createTable("CREATE TABLE %s (k int PRIMARY KEY, s set<text>)");
-
-        execute("INSERT INTO %s(k, s) VALUES (0, ?)", set("v1", "v2", "v3", "v4"));
-
-        assertRows(execute("SELECT s FROM %s WHERE k = 0"),
-            row(set("v1", "v2", "v3", "v4"))
-        );
-
-        execute("DELETE s[?] FROM %s WHERE k = 0", "v1");
-
-        assertRows(execute("SELECT s FROM %s WHERE k = 0"),
-            row(set("v2", "v3", "v4"))
-        );
-
-        // Full overwrite
-        execute("UPDATE %s SET s = ? WHERE k = 0", set("v6", "v5"));
-
-        assertRows(execute("SELECT s FROM %s WHERE k = 0"),
-            row(set("v5", "v6"))
-        );
-
-        execute("UPDATE %s SET s = s + ? WHERE k = 0", set("v7"));
-
-        assertRows(execute("SELECT s FROM %s WHERE k = 0"),
-            row(set("v5", "v6", "v7"))
-        );
-
-        execute("UPDATE %s SET s = s - ? WHERE k = 0", set("v6", "v5"));
-
-        assertRows(execute("SELECT s FROM %s WHERE k = 0"),
-            row(set("v7"))
-        );
-
-        execute("DELETE s[?] FROM %s WHERE k = 0", set("v7"));
-
-        // Deleting an element that does not exist will succeed
-        execute("DELETE s[?] FROM %s WHERE k = 0", set("v7"));
-
-        execute("DELETE s FROM %s WHERE k = 0");
-
-        assertRows(execute("SELECT s FROM %s WHERE k = 0"),
-            row((Object)null)
-        );
-    }
-
-    @Test
-    public void testMaps() throws Throwable
-    {
-        createTable("CREATE TABLE %s (k int PRIMARY KEY, m map<text, int>)");
-
-        execute("INSERT INTO %s(k, m) VALUES (0, ?)", map("v1", 1, "v2", 2));
-
-        assertRows(execute("SELECT m FROM %s WHERE k = 0"),
-            row(map("v1", 1, "v2", 2))
-        );
-
-        execute("UPDATE %s SET m[?] = ?, m[?] = ? WHERE k = 0", "v3", 3, "v4", 4);
-
-        assertRows(execute("SELECT m FROM %s WHERE k = 0"),
-            row(map("v1", 1, "v2", 2, "v3", 3, "v4", 4))
-        );
-
-        execute("DELETE m[?] FROM %s WHERE k = 0", "v1");
-
-        assertRows(execute("SELECT m FROM %s WHERE k = 0"),
-            row(map("v2", 2, "v3", 3, "v4", 4))
-        );
-
-        // Full overwrite
-        execute("UPDATE %s SET m = ? WHERE k = 0", map("v6", 6, "v5", 5));
-
-        assertRows(execute("SELECT m FROM %s WHERE k = 0"),
-            row(map("v5", 5, "v6", 6))
-        );
-
-        execute("UPDATE %s SET m = m + ? WHERE k = 0", map("v7", 7));
-
-        assertRows(execute("SELECT m FROM %s WHERE k = 0"),
-            row(map("v5", 5, "v6", 6, "v7", 7))
-        );
-
-        execute("DELETE m[?] FROM %s WHERE k = 0", "v7");
-
-        assertRows(execute("SELECT m FROM %s WHERE k = 0"),
-            row(map("v5", 5, "v6", 6))
-        );
-
-        execute("DELETE m[?] FROM %s WHERE k = 0", "v6");
-
-        assertRows(execute("SELECT m FROM %s WHERE k = 0"),
-            row(map("v5", 5))
-        );
-
-        execute("DELETE m[?] FROM %s WHERE k = 0", "v5");
-
-        assertRows(execute("SELECT m FROM %s WHERE k = 0"),
-            row((Object)null)
-        );
-
-        // Deleting a non-existing key should succeed
-        execute("DELETE m[?] FROM %s WHERE k = 0", "v5");
-
-        assertRows(execute("SELECT m FROM %s WHERE k = 0"),
-            row((Object) null)
-        );
-
-        // The empty map is parsed as an empty set (because we don't have enough info at parsing
-        // time when we see a {}) and special cased later. This test checks this work properly
-        execute("UPDATE %s SET m = {} WHERE k = 0");
-
-        assertRows(execute("SELECT m FROM %s WHERE k = 0"),
-            row((Object)null)
-        );
-    }
-
-    @Test
-    public void testLists() throws Throwable
-    {
-        createTable("CREATE TABLE %s (k int PRIMARY KEY, l list<text>)");
-
-        execute("INSERT INTO %s(k, l) VALUES (0, ?)", list("v1", "v2", "v3"));
-
-        assertRows(execute("SELECT l FROM %s WHERE k = 0"), row(list("v1", "v2", "v3")));
-
-        execute("DELETE l[?] FROM %s WHERE k = 0", 1);
-
-        assertRows(execute("SELECT l FROM %s WHERE k = 0"), row(list("v1", "v3")));
-
-        execute("UPDATE %s SET l[?] = ? WHERE k = 0", 1, "v4");
-
-        assertRows(execute("SELECT l FROM %s WHERE k = 0"), row(list("v1", "v4")));
-
-        // Full overwrite
-        execute("UPDATE %s SET l = ? WHERE k = 0", list("v6", "v5"));
-
-        assertRows(execute("SELECT l FROM %s WHERE k = 0"), row(list("v6", "v5")));
-
-        execute("UPDATE %s SET l = l + ? WHERE k = 0", list("v7", "v8"));
-
-        assertRows(execute("SELECT l FROM %s WHERE k = 0"), row(list("v6", "v5", "v7", "v8")));
-
-        execute("UPDATE %s SET l = ? + l WHERE k = 0", list("v9"));
-
-        assertRows(execute("SELECT l FROM %s WHERE k = 0"), row(list("v9", "v6", "v5", "v7", "v8")));
-
-        execute("UPDATE %s SET l = l - ? WHERE k = 0", list("v5", "v8"));
-
-        assertRows(execute("SELECT l FROM %s WHERE k = 0"), row(list("v9", "v6", "v7")));
-
-        execute("DELETE l FROM %s WHERE k = 0");
-
-        assertRows(execute("SELECT l FROM %s WHERE k = 0"), row((Object) null));
-
-        assertInvalidMessage("Attempted to delete an element from a list which is null",
-                             "DELETE l[0] FROM %s WHERE k=0 ");
-
-        assertInvalidMessage("Attempted to set an element on a list which is null",
-                             "UPDATE %s SET l[0] = ? WHERE k=0", list("v10"));
-
-        execute("UPDATE %s SET l = l - ? WHERE k=0 ", list("v11"));
-
-        assertRows(execute("SELECT l FROM %s WHERE k = 0"), row((Object) null));
-    }
-
-    @Test
-    public void testMapWithUnsetValues() throws Throwable
-    {
-        createTable("CREATE TABLE %s (k int PRIMARY KEY, m map<text,text>)");
-        // set up
-        Object m = map("k", "v");
-        execute("INSERT INTO %s (k, m) VALUES (10, ?)", m);
-        assertRows(execute("SELECT m FROM %s WHERE k = 10"),
-                row(m)
-        );
-
-        // test putting an unset map, should not delete the contents
-        execute("INSERT INTO %s (k, m) VALUES (10, ?)", unset());
-        assertRows(execute("SELECT m FROM %s WHERE k = 10"),
-                row(m)
-        );
-        // test unset variables in a map update operaiotn, should not delete the contents
-        execute("UPDATE %s SET m['k'] = ? WHERE k = 10", unset());
-        assertRows(execute("SELECT m FROM %s WHERE k = 10"),
-                row(m)
-        );
-        assertInvalidMessage("Invalid unset map key", "UPDATE %s SET m[?] = 'foo' WHERE k = 10", unset());
-
-        // test unset value for map key
-        assertInvalidMessage("Invalid unset map key", "DELETE m[?] FROM %s WHERE k = 10", unset());
-    }
-
-    @Test
-    public void testListWithUnsetValues() throws Throwable
-    {
-        createTable("CREATE TABLE %s (k int PRIMARY KEY, l list<text>)");
-        // set up
-        Object l = list("foo", "foo");
-        execute("INSERT INTO %s (k, l) VALUES (10, ?)", l);
-        assertRows(execute("SELECT l FROM %s WHERE k = 10"),
-                row(l)
-        );
-
-        // replace list with unset value
-        execute("INSERT INTO %s (k, l) VALUES (10, ?)", unset());
-        assertRows(execute("SELECT l FROM %s WHERE k = 10"),
-                row(l)
-        );
-
-        // add to position
-        execute("UPDATE %s SET l[1] = ? WHERE k = 10", unset());
-        assertRows(execute("SELECT l FROM %s WHERE k = 10"),
-                row(l)
-        );
-
-        // set in index
-        assertInvalidMessage("Invalid unset value for list index", "UPDATE %s SET l[?] = 'foo' WHERE k = 10", unset());
-
-        // remove element by index
-        execute("DELETE l[?] FROM %s WHERE k = 10", unset());
-        assertRows(execute("SELECT l FROM %s WHERE k = 10"),
-                row(l)
-        );
-
-        // remove all occurrences of element
-        execute("UPDATE %s SET l = l - ? WHERE k = 10", unset());
-        assertRows(execute("SELECT l FROM %s WHERE k = 10"),
-                row(l)
-        );
-
-        // select with in clause
-        assertInvalidMessage("Invalid unset value for column k", "SELECT * FROM %s WHERE k IN ?", unset());
-        assertInvalidMessage("Invalid unset value for column k", "SELECT * FROM %s WHERE k IN (?)", unset());
-    }
-
-    @Test
-    public void testSetWithUnsetValues() throws Throwable
-    {
-        createTable("CREATE TABLE %s (k int PRIMARY KEY, s set<text>)");
-
-        Object s = set("bar", "baz", "foo");
-        execute("INSERT INTO %s (k, s) VALUES (10, ?)", s);
-        assertRows(execute("SELECT s FROM %s WHERE k = 10"),
-                row(s)
-        );
-
-        // replace set with unset value
-        execute("INSERT INTO %s (k, s) VALUES (10, ?)", unset());
-        assertRows(execute("SELECT s FROM %s WHERE k = 10"),
-                row(s)
-        );
-
-        // add to set
-        execute("UPDATE %s SET s = s + ? WHERE k = 10", unset());
-        assertRows(execute("SELECT s FROM %s WHERE k = 10"),
-                row(s)
-        );
-
-        // remove all occurrences of element
-        execute("UPDATE %s SET s = s - ? WHERE k = 10", unset());
-        assertRows(execute("SELECT s FROM %s WHERE k = 10"),
-                row(s)
-        );
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/01115f72/test/unit/org/apache/cassandra/cql3/ContainsRelationTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/ContainsRelationTest.java b/test/unit/org/apache/cassandra/cql3/ContainsRelationTest.java
deleted file mode 100644
index a987bb2..0000000
--- a/test/unit/org/apache/cassandra/cql3/ContainsRelationTest.java
+++ /dev/null
@@ -1,283 +0,0 @@
-package org.apache.cassandra.cql3;
-
-import org.junit.Test;
-
-public class ContainsRelationTest extends CQLTester
-{
-    @Test
-    public void testSetContains() throws Throwable
-    {
-        createTable("CREATE TABLE %s (account text, id int, categories set<text>, PRIMARY KEY (account, id))");
-        createIndex("CREATE INDEX ON %s(categories)");
-
-        execute("INSERT INTO %s (account, id , categories) VALUES (?, ?, ?)", "test", 5, set("lmn"));
-
-        assertEmpty(execute("SELECT * FROM %s WHERE account = ? AND categories CONTAINS ?", "xyz", "lmn"));
-
-        assertRows(execute("SELECT * FROM %s WHERE categories CONTAINS ?", "lmn"),
-            row("test", 5, set("lmn"))
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE account = ? AND categories CONTAINS ?", "test", "lmn"),
-            row("test", 5, set("lmn"))
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE account = ? AND id = ? AND categories CONTAINS ?", "test", 5, "lmn"),
-                   row("test", 5, set("lmn"))
-        );
-
-        assertInvalidMessage("Unsupported null value for indexed column categories",
-                             "SELECT * FROM %s WHERE account = ? AND id = ? AND categories CONTAINS ?", "test", 5, null);
-
-        assertInvalidMessage("Unsupported unset value for indexed column categories",
-                             "SELECT * FROM %s WHERE account = ? AND id = ? AND categories CONTAINS ?", "test", 5, unset());
-
-        assertInvalidMessage("Cannot execute this query as it might involve data filtering and thus may have unpredictable performance. If you want to execute this query despite the performance unpredictability, use ALLOW FILTERING",
-                             "SELECT * FROM %s WHERE account = ? AND categories CONTAINS ? AND categories CONTAINS ?", "xyz", "lmn", "notPresent");
-        assertEmpty(execute("SELECT * FROM %s WHERE account = ? AND categories CONTAINS ? AND categories CONTAINS ? ALLOW FILTERING", "xyz", "lmn", "notPresent"));
-    }
-
-    @Test
-    public void testListContains() throws Throwable
-    {
-        createTable("CREATE TABLE %s (account text, id int, categories list<text>, PRIMARY KEY (account, id))");
-        createIndex("CREATE INDEX ON %s(categories)");
-
-        execute("INSERT INTO %s (account, id , categories) VALUES (?, ?, ?)", "test", 5, list("lmn"));
-
-        assertEmpty(execute("SELECT * FROM %s WHERE account = ? AND categories CONTAINS ?", "xyz", "lmn"));
-
-        assertRows(execute("SELECT * FROM %s WHERE account = ? AND categories CONTAINS ?;", "test", "lmn"),
-            row("test", 5, list("lmn"))
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE categories CONTAINS ?", "lmn"),
-            row("test", 5, list("lmn"))
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE account = ? AND id = ? AND categories CONTAINS ?;", "test", 5, "lmn"),
-                   row("test", 5, list("lmn"))
-        );
-
-        assertInvalidMessage("Unsupported null value for indexed column categories",
-                             "SELECT * FROM %s WHERE account = ? AND id = ? AND categories CONTAINS ?", "test", 5, null);
-
-        assertInvalidMessage("Unsupported unset value for indexed column categories",
-                             "SELECT * FROM %s WHERE account = ? AND id = ? AND categories CONTAINS ?", "test", 5, unset());
-
-        assertInvalidMessage("Cannot execute this query as it might involve data filtering and thus may have unpredictable performance. If you want to execute this query despite the performance unpredictability, use ALLOW FILTERING",
-                             "SELECT * FROM %s WHERE account = ? AND id = ? AND categories CONTAINS ? AND categories CONTAINS ?",
-                             "test", 5, "lmn", "notPresent");
-        assertEmpty(execute("SELECT * FROM %s WHERE account = ? AND id = ? AND categories CONTAINS ? AND categories CONTAINS ? ALLOW FILTERING",
-                            "test", 5, "lmn", "notPresent"));
-    }
-
-    @Test
-    public void testListContainsWithFiltering() throws Throwable
-    {
-        createTable("CREATE TABLE %s (e int PRIMARY KEY, f list<text>, s int)");
-        createIndex("CREATE INDEX ON %s(f)");
-        for(int i = 0; i < 3; i++)
-        {
-            execute("INSERT INTO %s (e, f, s) VALUES (?, ?, ?)", i, list("Dubai"), 4);
-        }
-        for(int i = 3; i < 5; i++)
-        {
-            execute("INSERT INTO %s (e, f, s) VALUES (?, ?, ?)", i, list("Dubai"), 3);
-        }
-        assertRows(execute("SELECT * FROM %s WHERE f CONTAINS ? AND s=? allow filtering", "Dubai", 3),
-                   row(3, list("Dubai"), 3),
-                   row(4, list("Dubai"), 3));
-    }
-
-    @Test
-    public void testMapKeyContains() throws Throwable
-    {
-        createTable("CREATE TABLE %s (account text, id int, categories map<text,text>, PRIMARY KEY (account, id))");
-        createIndex("CREATE INDEX ON %s(keys(categories))");
-
-        execute("INSERT INTO %s (account, id , categories) VALUES (?, ?, ?)", "test", 5, map("lmn", "foo"));
-
-        assertEmpty(execute("SELECT * FROM %s WHERE account = ? AND categories CONTAINS KEY ?", "xyz", "lmn"));
-
-        assertRows(execute("SELECT * FROM %s WHERE account = ? AND categories CONTAINS KEY ?", "test", "lmn"),
-            row("test", 5, map("lmn", "foo"))
-        );
-        assertRows(execute("SELECT * FROM %s WHERE categories CONTAINS KEY ?", "lmn"),
-            row("test", 5, map("lmn", "foo"))
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE account = ? AND id = ? AND categories CONTAINS KEY ?", "test", 5, "lmn"),
-                   row("test", 5, map("lmn", "foo"))
-        );
-
-        assertInvalidMessage("Unsupported null value for indexed column categories",
-                             "SELECT * FROM %s WHERE account = ? AND id = ? AND categories CONTAINS KEY ?", "test", 5, null);
-
-        assertInvalidMessage("Unsupported unset value for indexed column categories",
-                             "SELECT * FROM %s WHERE account = ? AND id = ? AND categories CONTAINS KEY ?", "test", 5, unset());
-
-        assertInvalidMessage("Cannot execute this query as it might involve data filtering and thus may have unpredictable performance. If you want to execute this query despite the performance unpredictability, use ALLOW FILTERING",
-                             "SELECT * FROM %s WHERE account = ? AND id = ? AND categories CONTAINS KEY ? AND categories CONTAINS KEY ?",
-                             "test", 5, "lmn", "notPresent");
-        assertEmpty(execute("SELECT * FROM %s WHERE account = ? AND id = ? AND categories CONTAINS KEY ? AND categories CONTAINS KEY ? ALLOW FILTERING",
-                            "test", 5, "lmn", "notPresent"));
-
-        assertInvalidMessage("Cannot execute this query as it might involve data filtering and thus may have unpredictable performance. If you want to execute this query despite the performance unpredictability, use ALLOW FILTERING",
-                             "SELECT * FROM %s WHERE account = ? AND id = ? AND categories CONTAINS KEY ? AND categories CONTAINS ?",
-                             "test", 5, "lmn", "foo");
-    }
-
-    @Test
-    public void testMapValueContains() throws Throwable
-    {
-        createTable("CREATE TABLE %s (account text, id int, categories map<text,text>, PRIMARY KEY (account, id))");
-        createIndex("CREATE INDEX ON %s(categories)");
-
-        execute("INSERT INTO %s (account, id , categories) VALUES (?, ?, ?)", "test", 5, map("lmn", "foo"));
-
-        assertEmpty(execute("SELECT * FROM %s WHERE account = ? AND categories CONTAINS ?", "xyz", "foo"));
-
-        assertRows(execute("SELECT * FROM %s WHERE account = ? AND categories CONTAINS ?", "test", "foo"),
-            row("test", 5, map("lmn", "foo"))
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE categories CONTAINS ?", "foo"),
-            row("test", 5, map("lmn", "foo"))
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE account = ? AND id = ? AND categories CONTAINS ?", "test", 5, "foo"),
-                   row("test", 5, map("lmn", "foo"))
-        );
-
-        assertInvalidMessage("Unsupported null value for indexed column categories",
-                             "SELECT * FROM %s WHERE account = ? AND id = ? AND categories CONTAINS ?", "test", 5, null);
-
-        assertInvalidMessage("Unsupported unset value for indexed column categories",
-                             "SELECT * FROM %s WHERE account = ? AND id = ? AND categories CONTAINS ?", "test", 5, unset());
-
-        assertInvalidMessage("Cannot execute this query as it might involve data filtering and thus may have unpredictable performance. If you want to execute this query despite the performance unpredictability, use ALLOW FILTERING",
-                             "SELECT * FROM %s WHERE account = ? AND id = ? AND categories CONTAINS ? AND categories CONTAINS ?"
-                           , "test", 5, "foo", "notPresent");
-
-        assertEmpty(execute("SELECT * FROM %s WHERE account = ? AND id = ? AND categories CONTAINS ? AND categories CONTAINS ? ALLOW FILTERING"
-                           , "test", 5, "foo", "notPresent"));
-    }
-
-    // See CASSANDRA-7525
-    @Test
-    public void testQueryMultipleIndexTypes() throws Throwable
-    {
-        createTable("CREATE TABLE %s (account text, id int, categories map<text,text>, PRIMARY KEY (account, id))");
-
-        // create an index on
-        createIndex("CREATE INDEX id_index ON %s(id)");
-        createIndex("CREATE INDEX categories_values_index ON %s(categories)");
-
-        execute("INSERT INTO %s (account, id , categories) VALUES (?, ?, ?)", "test", 5, map("lmn", "foo"));
-
-        assertRows(execute("SELECT * FROM %s WHERE categories CONTAINS ? AND id = ? ALLOW FILTERING", "foo", 5),
-                row("test", 5, map("lmn", "foo"))
-        );
-
-        assertRows(
-            execute("SELECT * FROM %s WHERE account = ? AND categories CONTAINS ? AND id = ? ALLOW FILTERING", "test", "foo", 5),
-            row("test", 5, map("lmn", "foo"))
-        );
-    }
-
-    // See CASSANDRA-8033
-    @Test
-    public void testFilterForContains() throws Throwable
-    {
-        createTable("CREATE TABLE %s (k1 int, k2 int, v set<int>, PRIMARY KEY ((k1, k2)))");
-        createIndex("CREATE INDEX ON %s(k2)");
-
-        execute("INSERT INTO %s (k1, k2, v) VALUES (?, ?, ?)", 0, 0, set(1, 2, 3));
-        execute("INSERT INTO %s (k1, k2, v) VALUES (?, ?, ?)", 0, 1, set(2, 3, 4));
-        execute("INSERT INTO %s (k1, k2, v) VALUES (?, ?, ?)", 1, 0, set(3, 4, 5));
-        execute("INSERT INTO %s (k1, k2, v) VALUES (?, ?, ?)", 1, 1, set(4, 5, 6));
-
-        assertRows(execute("SELECT * FROM %s WHERE k2 = ?", 1),
-            row(0, 1, set(2, 3, 4)),
-            row(1, 1, set(4, 5, 6))
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE k2 = ? AND v CONTAINS ? ALLOW FILTERING", 1, 6),
-            row(1, 1, set(4, 5, 6))
-        );
-
-        assertEmpty(execute("SELECT * FROM %s WHERE k2 = ? AND v CONTAINS ? ALLOW FILTERING", 1, 7));
-    }
-
-    // See CASSANDRA-8073
-    @Test
-    public void testIndexLookupWithClusteringPrefix() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a int, b int, c int, d set<int>, PRIMARY KEY (a, b, c))");
-        createIndex("CREATE INDEX ON %s(d)");
-        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 0, set(1, 2, 3));
-        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 1, set(3, 4, 5));
-        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 0, set(1, 2, 3));
-        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 1, set(3, 4, 5));
-
-        assertRows(execute("SELECT * FROM %s WHERE a=? AND b=? AND d CONTAINS ?", 0, 1, 3),
-            row(0, 1, 0, set(1, 2, 3)),
-            row(0, 1, 1, set(3, 4, 5))
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE a=? AND b=? AND d CONTAINS ?", 0, 1, 2),
-            row(0, 1, 0, set(1, 2, 3))
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE a=? AND b=? AND d CONTAINS ?", 0, 1, 5),
-            row(0, 1, 1, set(3, 4, 5))
-        );
-    }
-
-    @Test
-    public void testContainsKeyAndContainsWithIndexOnMapKey() throws Throwable
-    {
-        createTable("CREATE TABLE %s (account text, id int, categories map<text,text>, PRIMARY KEY (account, id))");
-        createIndex("CREATE INDEX ON %s(keys(categories))");
-
-        execute("INSERT INTO %s (account, id , categories) VALUES (?, ?, ?)", "test", 5, map("lmn", "foo"));
-        execute("INSERT INTO %s (account, id , categories) VALUES (?, ?, ?)", "test", 6, map("lmn", "foo2"));
-
-        assertInvalidMessage("No secondary indexes on the restricted columns support the provided operators: 'categories CONTAINS <value>'",
-                             "SELECT * FROM %s WHERE account = ? AND categories CONTAINS ?", "test", "foo");
-
-        assertRows(execute("SELECT * FROM %s WHERE account = ? AND categories CONTAINS KEY ?", "test", "lmn"),
-                   row("test", 5, map("lmn", "foo")),
-                   row("test", 6, map("lmn", "foo2")));
-        assertRows(execute("SELECT * FROM %s WHERE account = ? AND categories CONTAINS KEY ? AND categories CONTAINS ? ALLOW FILTERING",
-                           "test", "lmn", "foo"),
-                   row("test", 5, map("lmn", "foo")));
-        assertRows(execute("SELECT * FROM %s WHERE account = ? AND categories CONTAINS ? AND categories CONTAINS KEY ? ALLOW FILTERING",
-                           "test", "foo", "lmn"),
-                   row("test", 5, map("lmn", "foo")));
-    }
-
-    @Test
-    public void testContainsKeyAndContainsWithIndexOnMapValue() throws Throwable
-    {
-        createTable("CREATE TABLE %s (account text, id int, categories map<text,text>, PRIMARY KEY (account, id))");
-        createIndex("CREATE INDEX ON %s(categories)");
-
-        execute("INSERT INTO %s (account, id , categories) VALUES (?, ?, ?)", "test", 5, map("lmn", "foo"));
-        execute("INSERT INTO %s (account, id , categories) VALUES (?, ?, ?)", "test", 6, map("lmn2", "foo"));
-
-        assertInvalidMessage("No secondary indexes on the restricted columns support the provided operators: 'categories CONTAINS KEY <value>'",
-                             "SELECT * FROM %s WHERE account = ? AND categories CONTAINS KEY ?", "test", "lmn");
-
-        assertRows(execute("SELECT * FROM %s WHERE account = ? AND categories CONTAINS ?", "test", "foo"),
-                   row("test", 5, map("lmn", "foo")),
-                   row("test", 6, map("lmn2", "foo")));
-        assertRows(execute("SELECT * FROM %s WHERE account = ? AND categories CONTAINS KEY ? AND categories CONTAINS ? ALLOW FILTERING",
-                           "test", "lmn", "foo"),
-                   row("test", 5, map("lmn", "foo")));
-        assertRows(execute("SELECT * FROM %s WHERE account = ? AND categories CONTAINS ? AND categories CONTAINS KEY ? ALLOW FILTERING",
-                           "test", "foo", "lmn"),
-                   row("test", 5, map("lmn", "foo")));
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/01115f72/test/unit/org/apache/cassandra/cql3/CrcCheckChanceTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/CrcCheckChanceTest.java b/test/unit/org/apache/cassandra/cql3/CrcCheckChanceTest.java
deleted file mode 100644
index ac3ffbc..0000000
--- a/test/unit/org/apache/cassandra/cql3/CrcCheckChanceTest.java
+++ /dev/null
@@ -1,159 +0,0 @@
-/*
- * 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;
-
-import java.util.List;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.Future;
-
-import junit.framework.Assert;
-import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.db.ColumnFamilyStore;
-import org.apache.cassandra.db.Keyspace;
-import org.apache.cassandra.db.compaction.CompactionInterruptedException;
-import org.apache.cassandra.db.compaction.CompactionManager;
-import org.apache.cassandra.utils.FBUtilities;
-
-import org.junit.Test;
-
-
-public class CrcCheckChanceTest extends CQLTester
-{
-    @Test
-    public void testChangingCrcCheckChance() throws Throwable
-    {
-        //Start with crc_check_chance of 99%
-        createTable("CREATE TABLE %s (p text, c text, v text, s text static, PRIMARY KEY (p, c)) WITH compression = {'sstable_compression': 'LZ4Compressor', 'crc_check_chance' : 0.99}");
-
-        execute("CREATE INDEX foo ON %s(v)");
-
-        execute("INSERT INTO %s(p, c, v, s) values (?, ?, ?, ?)", "p1", "k1", "v1", "sv1");
-        execute("INSERT INTO %s(p, c, v) values (?, ?, ?)", "p1", "k2", "v2");
-        execute("INSERT INTO %s(p, s) values (?, ?)", "p2", "sv2");
-
-
-        ColumnFamilyStore cfs = Keyspace.open(CQLTester.KEYSPACE).getColumnFamilyStore(currentTable());
-        ColumnFamilyStore indexCfs = cfs.indexManager.getIndexesBackedByCfs().iterator().next();
-        cfs.forceBlockingFlush();
-
-        Assert.assertEquals(0.99, cfs.metadata.compressionParameters.getCrcCheckChance());
-        Assert.assertEquals(0.99, cfs.getSSTables().iterator().next().getCompressionMetadata().parameters.getCrcCheckChance());
-        Assert.assertEquals(0.99, indexCfs.metadata.compressionParameters.getCrcCheckChance());
-        Assert.assertEquals(0.99, indexCfs.getSSTables().iterator().next().getCompressionMetadata().parameters.getCrcCheckChance());
-
-        //Test for stack overflow
-        cfs.setCrcCheckChance(0.99);
-
-        assertRows(execute("SELECT * FROM %s WHERE p=?", "p1"),
-                row("p1", "k1", "sv1", "v1"),
-                row("p1", "k2", "sv1", "v2")
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE v=?", "v1"),
-                row("p1", "k1", "sv1", "v1")
-        );
-
-
-
-        //Write a few SSTables then Compact
-
-        execute("INSERT INTO %s(p, c, v, s) values (?, ?, ?, ?)", "p1", "k1", "v1", "sv1");
-        execute("INSERT INTO %s(p, c, v) values (?, ?, ?)", "p1", "k2", "v2");
-        execute("INSERT INTO %s(p, s) values (?, ?)", "p2", "sv2");
-
-        cfs.forceBlockingFlush();
-
-
-        execute("INSERT INTO %s(p, c, v, s) values (?, ?, ?, ?)", "p1", "k1", "v1", "sv1");
-        execute("INSERT INTO %s(p, c, v) values (?, ?, ?)", "p1", "k2", "v2");
-        execute("INSERT INTO %s(p, s) values (?, ?)", "p2", "sv2");
-
-        cfs.forceBlockingFlush();
-
-        execute("INSERT INTO %s(p, c, v, s) values (?, ?, ?, ?)", "p1", "k1", "v1", "sv1");
-        execute("INSERT INTO %s(p, c, v) values (?, ?, ?)", "p1", "k2", "v2");
-        execute("INSERT INTO %s(p, s) values (?, ?)", "p2", "sv2");
-
-        cfs.forceBlockingFlush();
-
-        cfs.forceMajorCompaction();
-
-        //Verify when we alter the value the live sstable readers hold the new one
-        alterTable("ALTER TABLE %s WITH compression = {'sstable_compression': 'LZ4Compressor', 'crc_check_chance': 0.01}");
-
-        Assert.assertEquals( 0.01, cfs.metadata.compressionParameters.getCrcCheckChance());
-        Assert.assertEquals( 0.01, cfs.getSSTables().iterator().next().getCompressionMetadata().parameters.getCrcCheckChance());
-        Assert.assertEquals( 0.01, indexCfs.metadata.compressionParameters.getCrcCheckChance());
-        Assert.assertEquals( 0.01, indexCfs.getSSTables().iterator().next().getCompressionMetadata().parameters.getCrcCheckChance());
-
-        assertRows(execute("SELECT * FROM %s WHERE p=?", "p1"),
-                row("p1", "k1", "sv1", "v1"),
-                row("p1", "k2", "sv1", "v2")
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE v=?", "v1"),
-                row("p1", "k1", "sv1", "v1")
-        );
-
-
-        //Verify the call used by JMX still works
-        cfs.setCrcCheckChance(0.03);
-        Assert.assertEquals( 0.03, cfs.metadata.compressionParameters.getCrcCheckChance());
-        Assert.assertEquals( 0.03, cfs.getSSTables().iterator().next().getCompressionMetadata().parameters.getCrcCheckChance());
-        Assert.assertEquals( 0.03, indexCfs.metadata.compressionParameters.getCrcCheckChance());
-        Assert.assertEquals( 0.03, indexCfs.getSSTables().iterator().next().getCompressionMetadata().parameters.getCrcCheckChance());
-
-    }
-
-
-    @Test
-    public void testDropDuringCompaction() throws Throwable
-    {
-        CompactionManager.instance.disableAutoCompaction();
-
-        //Start with crc_check_chance of 99%
-        createTable("CREATE TABLE %s (p text, c text, v text, s text static, PRIMARY KEY (p, c)) WITH compression = {'sstable_compression': 'LZ4Compressor', 'crc_check_chance' : 0.99}");
-
-        ColumnFamilyStore cfs = Keyspace.open(CQLTester.KEYSPACE).getColumnFamilyStore(currentTable());
-
-        //Write a few SSTables then Compact, and drop
-        for (int i = 0; i < 100; i++)
-        {
-            execute("INSERT INTO %s(p, c, v, s) values (?, ?, ?, ?)", "p1", "k1", "v1", "sv1");
-            execute("INSERT INTO %s(p, c, v) values (?, ?, ?)", "p1", "k2", "v2");
-            execute("INSERT INTO %s(p, s) values (?, ?)", "p2", "sv2");
-
-            cfs.forceBlockingFlush();
-        }
-
-        DatabaseDescriptor.setCompactionThroughputMbPerSec(1);
-        List<Future<?>> futures = CompactionManager.instance.submitMaximal(cfs, CompactionManager.getDefaultGcBefore(cfs), false); 
-        execute("DROP TABLE %s");
-
-        try
-        {
-            FBUtilities.waitOnFutures(futures);
-        }
-        catch (Throwable t)
-        {
-            if (!(t.getCause() instanceof ExecutionException) || !(t.getCause().getCause() instanceof CompactionInterruptedException))
-                throw t;
-        }
-    }
-}
-

http://git-wip-us.apache.org/repos/asf/cassandra/blob/01115f72/test/unit/org/apache/cassandra/cql3/CreateAndAlterKeyspaceTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/CreateAndAlterKeyspaceTest.java b/test/unit/org/apache/cassandra/cql3/CreateAndAlterKeyspaceTest.java
deleted file mode 100644
index 9e0ca21..0000000
--- a/test/unit/org/apache/cassandra/cql3/CreateAndAlterKeyspaceTest.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * 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;
-
-import org.junit.Test;
-
-public class CreateAndAlterKeyspaceTest extends CQLTester
-{
-    @Test
-    // tests CASSANDRA-9565
-    public void testCreateAndAlterWithDoubleWith() throws Throwable
-    {
-        String[] stmts = new String[] {"ALTER KEYSPACE WITH WITH DURABLE_WRITES = true",
-                                       "ALTER KEYSPACE ks WITH WITH DURABLE_WRITES = true",
-                                       "CREATE KEYSPACE WITH WITH DURABLE_WRITES = true",
-                                       "CREATE KEYSPACE ks WITH WITH DURABLE_WRITES = true"};
-
-        for (String stmt : stmts) {
-            assertInvalidSyntaxMessage("no viable alternative at input 'WITH'", stmt);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/01115f72/test/unit/org/apache/cassandra/cql3/CreateIndexStatementTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/CreateIndexStatementTest.java b/test/unit/org/apache/cassandra/cql3/CreateIndexStatementTest.java
deleted file mode 100644
index 1e2e084..0000000
--- a/test/unit/org/apache/cassandra/cql3/CreateIndexStatementTest.java
+++ /dev/null
@@ -1,101 +0,0 @@
-/*
- * 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;
-
-import java.util.Locale;
-
-import org.apache.commons.lang3.StringUtils;
-
-import org.junit.Test;
-
-public class CreateIndexStatementTest extends CQLTester
-{
-    @Test
-    public void testCreateAndDropIndex() throws Throwable
-    {
-        testCreateAndDropIndex("test", false);
-        testCreateAndDropIndex("test2", true);
-    }
-
-    @Test
-    public void testCreateAndDropIndexWithQuotedIdentifier() throws Throwable
-    {
-        testCreateAndDropIndex("\"quoted_ident\"", false);
-        testCreateAndDropIndex("\"quoted_ident2\"", true);
-    }
-
-    @Test
-    public void testCreateAndDropIndexWithCamelCaseIdentifier() throws Throwable
-    {
-        testCreateAndDropIndex("CamelCase", false);
-        testCreateAndDropIndex("CamelCase2", true);
-    }
-
-    /**
-     * Test creating and dropping an index with the specified name.
-     *
-     * @param indexName the index name
-     * @param addKeyspaceOnDrop add the keyspace name in the drop statement
-     * @throws Throwable if an error occurs
-     */
-    private void testCreateAndDropIndex(String indexName, boolean addKeyspaceOnDrop) throws Throwable
-    {
-        execute("USE system");
-        assertInvalidMessage("Index '" + removeQuotes(indexName.toLowerCase(Locale.US)) + "' could not be found", "DROP INDEX " + indexName + ";");
-
-        createTable("CREATE TABLE %s (a int primary key, b int);");
-        createIndex("CREATE INDEX " + indexName + " ON %s(b);");
-        createIndex("CREATE INDEX IF NOT EXISTS " + indexName + " ON %s(b);");
-
-        assertInvalidMessage("Index already exists", "CREATE INDEX " + indexName + " ON %s(b)");
-
-        execute("INSERT INTO %s (a, b) values (?, ?);", 0, 0);
-        execute("INSERT INTO %s (a, b) values (?, ?);", 1, 1);
-        execute("INSERT INTO %s (a, b) values (?, ?);", 2, 2);
-        execute("INSERT INTO %s (a, b) values (?, ?);", 3, 1);
-
-        assertRows(execute("SELECT * FROM %s where b = ?", 1), row(1, 1), row(3, 1));
-        assertInvalidMessage("Index '" + removeQuotes(indexName.toLowerCase(Locale.US)) + "' could not be found in any of the tables of keyspace 'system'", "DROP INDEX " + indexName);
-
-        if (addKeyspaceOnDrop)
-        {
-            dropIndex("DROP INDEX " + KEYSPACE + "." + indexName);
-        }
-        else
-        {
-            execute("USE " + KEYSPACE);
-            execute("DROP INDEX " + indexName);
-        }
-
-        assertInvalidMessage("No secondary indexes on the restricted columns support the provided operators",
-                             "SELECT * FROM %s where b = ?", 1);
-        dropIndex("DROP INDEX IF EXISTS " + indexName);
-        assertInvalidMessage("Index '" + removeQuotes(indexName.toLowerCase(Locale.US)) + "' could not be found", "DROP INDEX " + indexName);
-    }
-
-    /**
-     * Removes the quotes from the specified index name.
-     *
-     * @param indexName the index name from which the quotes must be removed.
-     * @return the unquoted index name.
-     */
-    private static String removeQuotes(String indexName)
-    {
-        return StringUtils.remove(indexName, '\"');
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/01115f72/test/unit/org/apache/cassandra/cql3/CreateTableTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/CreateTableTest.java b/test/unit/org/apache/cassandra/cql3/CreateTableTest.java
deleted file mode 100644
index d14e87b..0000000
--- a/test/unit/org/apache/cassandra/cql3/CreateTableTest.java
+++ /dev/null
@@ -1,69 +0,0 @@
-/*
- * 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;
-
-import org.junit.Test;
-
-import org.apache.cassandra.utils.ByteBufferUtil;
-
-import static junit.framework.Assert.assertFalse;
-
-public class CreateTableTest extends CQLTester
-{
-    @Test
-    public void testCQL3PartitionKeyOnlyTable()
-    {
-        createTable("CREATE TABLE %s (id text PRIMARY KEY);");
-        assertFalse(currentTableMetadata().isThriftCompatible());
-    }
-
-    @Test
-    public void testCreateTableWithSmallintColumns() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a text, b smallint, c smallint, primary key (a, b));");
-        execute("INSERT INTO %s (a, b, c) VALUES ('1', 1, 2)");
-        execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", "2", Short.MAX_VALUE, Short.MIN_VALUE);
-
-        assertRows(execute("SELECT * FROM %s"),
-                   row("1", (short) 1, (short) 2),
-                   row("2", Short.MAX_VALUE, Short.MIN_VALUE));
-
-        assertInvalidMessage("Expected 2 bytes for a smallint (4)",
-                             "INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", "3", 1, 2);
-        assertInvalidMessage("Expected 2 bytes for a smallint (0)",
-                             "INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", "3", (short) 1, ByteBufferUtil.EMPTY_BYTE_BUFFER);
-     }
-
-    @Test
-    public void testCreateTinyintColumns() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a text, b tinyint, c tinyint, primary key (a, b));");
-        execute("INSERT INTO %s (a, b, c) VALUES ('1', 1, 2)");
-        execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", "2", Byte.MAX_VALUE, Byte.MIN_VALUE);
-
-        assertRows(execute("SELECT * FROM %s"),
-                   row("1", (byte) 1, (byte) 2),
-                   row("2", Byte.MAX_VALUE, Byte.MIN_VALUE));
-
-        assertInvalidMessage("Expected 1 byte for a tinyint (4)",
-                             "INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", "3", 1, 2);
-
-        assertInvalidMessage("Expected 1 byte for a tinyint (0)",
-                             "INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", "3", (byte) 1, ByteBufferUtil.EMPTY_BYTE_BUFFER);
-     }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/01115f72/test/unit/org/apache/cassandra/cql3/CreateTriggerStatementTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/CreateTriggerStatementTest.java b/test/unit/org/apache/cassandra/cql3/CreateTriggerStatementTest.java
deleted file mode 100644
index 6557c16..0000000
--- a/test/unit/org/apache/cassandra/cql3/CreateTriggerStatementTest.java
+++ /dev/null
@@ -1,121 +0,0 @@
-/*
- * 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;
-
-import java.nio.ByteBuffer;
-import java.util.Collection;
-import java.util.Collections;
-
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.Schema;
-import org.apache.cassandra.config.TriggerDefinition;
-import org.apache.cassandra.db.ColumnFamily;
-import org.apache.cassandra.db.Mutation;
-import org.apache.cassandra.triggers.ITrigger;
-import org.junit.Test;
-
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-public class CreateTriggerStatementTest extends CQLTester
-{
-    @Test
-    public void testCreateTrigger() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a int, b int, c int, PRIMARY KEY (a))");
-        execute("CREATE TRIGGER trigger_1 ON %s USING '" + TestTrigger.class.getName() + "'");
-        assertTriggerExists("trigger_1", TestTrigger.class);
-        execute("CREATE TRIGGER trigger_2 ON %s USING '" + TestTrigger.class.getName() + "'");
-        assertTriggerExists("trigger_2", TestTrigger.class);
-        assertInvalid("CREATE TRIGGER trigger_1 ON %s USING '" + TestTrigger.class.getName() + "'");
-        execute("CREATE TRIGGER \"Trigger 3\" ON %s USING '" + TestTrigger.class.getName() + "'");
-        assertTriggerExists("Trigger 3", TestTrigger.class);
-    }
-
-    @Test
-    public void testCreateTriggerIfNotExists() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a int, b int, c int, PRIMARY KEY (a, b))");
-
-        execute("CREATE TRIGGER IF NOT EXISTS trigger_1 ON %s USING '" + TestTrigger.class.getName() + "'");
-        assertTriggerExists("trigger_1", TestTrigger.class);
-
-        execute("CREATE TRIGGER IF NOT EXISTS trigger_1 ON %s USING '" + TestTrigger.class.getName() + "'");
-        assertTriggerExists("trigger_1", TestTrigger.class);
-    }
-
-    @Test
-    public void testDropTrigger() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a int, b int, c int, PRIMARY KEY (a))");
-
-        execute("CREATE TRIGGER trigger_1 ON %s USING '" + TestTrigger.class.getName() + "'");
-        assertTriggerExists("trigger_1", TestTrigger.class);
-
-        execute("DROP TRIGGER trigger_1 ON %s");
-        assertTriggerDoesNotExists("trigger_1", TestTrigger.class);
-
-        execute("CREATE TRIGGER trigger_1 ON %s USING '" + TestTrigger.class.getName() + "'");
-        assertTriggerExists("trigger_1", TestTrigger.class);
-
-        assertInvalid("DROP TRIGGER trigger_2 ON %s");
-        
-        execute("CREATE TRIGGER \"Trigger 3\" ON %s USING '" + TestTrigger.class.getName() + "'");
-        assertTriggerExists("Trigger 3", TestTrigger.class);
-
-        execute("DROP TRIGGER \"Trigger 3\" ON %s");
-        assertTriggerDoesNotExists("Trigger 3", TestTrigger.class);
-    }
-
-    @Test
-    public void testDropTriggerIfExists() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a int, b int, c int, PRIMARY KEY (a))");
-
-        execute("DROP TRIGGER IF EXISTS trigger_1 ON %s");
-        assertTriggerDoesNotExists("trigger_1", TestTrigger.class);
-
-        execute("CREATE TRIGGER trigger_1 ON %s USING '" + TestTrigger.class.getName() + "'");
-        assertTriggerExists("trigger_1", TestTrigger.class);
-
-        execute("DROP TRIGGER IF EXISTS trigger_1 ON %s");
-        assertTriggerDoesNotExists("trigger_1", TestTrigger.class);
-    }
-
-    private void assertTriggerExists(String name, Class<?> clazz)
-    {
-        CFMetaData cfm = Schema.instance.getCFMetaData(keyspace(), currentTable()).copy();
-        assertTrue("the trigger does not exist", cfm.containsTriggerDefinition(TriggerDefinition.create(name,
-                clazz.getName())));
-    }
-
-    private void assertTriggerDoesNotExists(String name, Class<?> clazz)
-    {
-        CFMetaData cfm = Schema.instance.getCFMetaData(keyspace(), currentTable()).copy();
-        assertFalse("the trigger exists", cfm.containsTriggerDefinition(TriggerDefinition.create(name,
-                clazz.getName())));
-    }
-
-    public static class TestTrigger implements ITrigger
-    {
-        public Collection<Mutation> augment(ByteBuffer key, ColumnFamily update)
-        {
-            return Collections.emptyList();
-        }
-    }
-}


[04/32] cassandra git commit: Migrate CQL tests from dtest to unit tests

Posted by jm...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/f797bfa4/test/unit/org/apache/cassandra/cql3/validation/entities/StaticColumnsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/StaticColumnsTest.java b/test/unit/org/apache/cassandra/cql3/validation/entities/StaticColumnsTest.java
new file mode 100644
index 0000000..cef6f1f
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/StaticColumnsTest.java
@@ -0,0 +1,271 @@
+/*
+ * 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.validation.entities;
+
+import java.util.Arrays;
+
+import org.junit.Test;
+
+import org.apache.cassandra.cql3.CQLTester;
+
+import static junit.framework.Assert.assertNull;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+public class StaticColumnsTest extends CQLTester
+{
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.static_columns_test()
+     */
+    @Test
+    public void testStaticColumns() throws Throwable
+    {
+        createTable("CREATE TABLE %s ( k int, p int, s int static, v int, PRIMARY KEY (k, p))");
+
+        execute("INSERT INTO %s(k, s) VALUES (0, 42)");
+
+        assertRows(execute("SELECT * FROM %s"), row(0, null, 42, null));
+
+        // Check that writetime works (//7081) -- we can't predict the exact value easily so
+        // we just check that it's non zero
+        Object[][] row = getRows(execute("SELECT s, writetime(s) FROM %s WHERE k=0"));
+        assertEquals(42, row[0][0]);
+        assertTrue((Long)row[0][1] > 0);
+
+        execute("INSERT INTO %s (k, p, s, v) VALUES (0, 0, 12, 0)");
+        execute("INSERT INTO %s (k, p, s, v) VALUES (0, 1, 24, 1)");
+
+        // Check the static columns in indeed "static"
+        assertRows(execute("SELECT * FROM %s"), row(0, 0, 24, 0), row(0, 1, 24, 1));
+
+        // Check we do correctly get the static column value with a SELECT *, even
+        // if we're only slicing part of the partition
+        assertRows(execute("SELECT * FROM %s WHERE k=0 AND p=0"), row(0, 0, 24, 0));
+        assertRows(execute("SELECT * FROM %s WHERE k=0 AND p=1"), row(0, 1, 24, 1));
+
+        // Test for IN on the clustering key (//6769)
+        assertRows(execute("SELECT * FROM %s WHERE k=0 AND p IN (0, 1)"), row(0, 0, 24, 0), row(0, 1, 24, 1));
+
+        // Check things still work if we don't select the static column. We also want
+        // this to not request the static columns internally at all, though that part
+        // require debugging to assert
+        assertRows(execute("SELECT p, v FROM %s WHERE k=0 AND p=1"), row(1, 1));
+
+        // Check selecting only a static column with distinct only yield one value
+        // (as we only query the static columns)
+        assertRows(execute("SELECT DISTINCT s FROM %s WHERE k=0"), row(24));
+        // But without DISTINCT, we still get one result per row
+        assertRows(execute("SELECT s FROM %s WHERE k=0"),row(24),row(24));
+        // but that querying other columns does correctly yield the full partition
+        assertRows(execute("SELECT s, v FROM %s WHERE k=0"),row(24, 0),row(24, 1));
+        assertRows(execute("SELECT s, v FROM %s WHERE k=0 AND p=1"),row(24, 1));
+        assertRows(execute("SELECT p, s FROM %s WHERE k=0 AND p=1"), row(1, 24));
+        assertRows(execute("SELECT k, p, s FROM %s WHERE k=0 AND p=1"),row(0, 1, 24));
+
+        // Check that deleting a row don't implicitely deletes statics
+        execute("DELETE FROM %s WHERE k=0 AND p=0");
+        assertRows(execute("SELECT * FROM %s"),row(0, 1, 24, 1));
+
+        // But that explicitely deleting the static column does remove it
+        execute("DELETE s FROM %s WHERE k=0");
+        assertRows(execute("SELECT * FROM %s"), row(0, 1, null, 1));
+
+        // Check we can add a static column ...
+        execute("ALTER TABLE %s ADD s2 int static");
+        assertRows(execute("SELECT * FROM %s"), row(0, 1, null, null, 1));
+        execute("INSERT INTO %s (k, p, s2, v) VALUES(0, 2, 42, 2)");
+        assertRows(execute("SELECT * FROM %s"), row(0, 1, null, 42, 1), row(0, 2, null, 42, 2));
+        // ... and that we can drop it
+        execute("ALTER TABLE %s DROP s2");
+        assertRows(execute("SELECT * FROM %s"), row(0, 1, null, 1), row(0, 2, null, 2));
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.static_columns_with_2i_test()
+     */
+    @Test
+    public void testStaticColumnsWithSecondaryIndex() throws Throwable
+    {
+        createTable(" CREATE TABLE %s (k int, p int, s int static, v int, PRIMARY KEY (k, p) ) ");
+
+        createIndex("CREATE INDEX ON %s (v)");
+
+        execute("INSERT INTO %s (k, p, s, v) VALUES (0, 0, 42, 1)");
+        execute("INSERT INTO %s (k, p, v) VALUES (0, 1, 1)");
+        execute("INSERT INTO %s (k, p, v) VALUES (0, 2, 2)");
+
+        assertRows(execute("SELECT * FROM %s WHERE v = 1"), row(0, 0, 42, 1), row(0, 1, 42, 1));
+        assertRows(execute("SELECT p, s FROM %s WHERE v = 1"), row(0, 42), row(1, 42));
+        assertRows(execute("SELECT p FROM %s WHERE v = 1"), row(0), row(1));
+        // We don't support that
+        assertInvalid("SELECT s FROM %s WHERE v = 1");
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.static_columns_with_distinct_test()
+     */
+    @Test
+    public void testStaticColumnsWithDistinct() throws Throwable
+    {
+        createTable("CREATE TABLE %s( k int, p int, s int static, PRIMARY KEY (k, p) ) ");
+
+        execute("INSERT INTO %s (k, p) VALUES (1, 1)");
+        execute("INSERT INTO %s (k, p) VALUES (1, 2)");
+
+        assertRows(execute("SELECT k, s FROM %s"), row(1, null), row(1, null));
+        assertRows(execute("SELECT DISTINCT k, s FROM %s"), row(1, null));
+
+        Object[][] rows = getRows(execute("SELECT DISTINCT s FROM %s WHERE k=1"));
+        assertNull(rows[0][0]);
+
+        assertEmpty(execute("SELECT DISTINCT s FROM %s WHERE k=2"));
+
+        execute("INSERT INTO %s (k, p, s) VALUES (2, 1, 3)");
+        execute("INSERT INTO %s (k, p) VALUES (2, 2)");
+
+        assertRows(execute("SELECT k, s FROM %s"), row(1, null), row(1, null), row(2, 3), row(2, 3));
+        assertRows(execute("SELECT DISTINCT k, s FROM %s"), row(1, null), row(2, 3));
+        rows = getRows(execute("SELECT DISTINCT s FROM %s WHERE k=1"));
+        assertNull(rows[0][0]);
+        assertRows(execute("SELECT DISTINCT s FROM %s WHERE k=2"), row(3));
+
+        assertInvalid("SELECT DISTINCT s FROM %s");
+
+        // paging to test for CASSANDRA-8108
+        execute("TRUNCATE %s");
+        for (int i = 0; i < 10; i++)
+            for (int j = 0; j < 10; j++)
+                execute("INSERT INTO %s (k, p, s) VALUES (?, ?, ?)", i, j, i);
+
+        rows = getRows(execute("SELECT DISTINCT k, s FROM %s"));
+        checkDistinctRows(rows, true, 0, 10, 0, 10);
+
+        String keys = "0, 1, 2, 3, 4, 5, 6, 7, 8, 9";
+        rows = getRows(execute("SELECT DISTINCT k, s FROM %s WHERE k IN (" + keys + ")"));
+        checkDistinctRows(rows, false, 0, 10, 0, 10);
+
+        // additional testing for CASSANRA-8087
+        createTable("CREATE TABLE %s( k int, c1 int, c2 int, s1 int static, s2 int static, PRIMARY KEY (k, c1, c2))");
+
+        for (int i = 0; i < 10; i++)
+            for (int j = 0; j < 5; j++)
+                for (int k = 0; k < 5; k++)
+                    execute("INSERT INTO %s (k, c1, c2, s1, s2) VALUES (?, ?, ?, ?, ?)", i, j, k, i, i + 1);
+
+        rows = getRows(execute("SELECT DISTINCT k, s1 FROM %s"));
+        checkDistinctRows(rows, true, 0, 10, 0, 10);
+
+        rows = getRows(execute("SELECT DISTINCT k, s2 FROM %s"));
+        checkDistinctRows(rows, true, 0, 10, 1, 11);
+
+        rows = getRows(execute("SELECT DISTINCT k, s1 FROM %s LIMIT 10"));
+        checkDistinctRows(rows, true, 0, 10, 0, 10);
+
+        rows = getRows(execute("SELECT DISTINCT k, s1 FROM %s WHERE k IN (" + keys + ")"));
+        checkDistinctRows(rows, false, 0, 10, 0, 10);
+
+        rows = getRows(execute("SELECT DISTINCT k, s2 FROM %s WHERE k IN (" + keys + ")"));
+        checkDistinctRows(rows, false, 0, 10, 1, 11);
+
+        rows = getRows(execute("SELECT DISTINCT k, s1 FROM %s WHERE k IN (" + keys + ")"));
+        checkDistinctRows(rows, true, 0, 10, 0, 10);
+    }
+
+    void checkDistinctRows(Object[][] rows, boolean sort, int... ranges)
+    {
+        assertTrue(ranges.length % 2 == 0);
+
+        int numdim = ranges.length / 2;
+        int[] from = new int[numdim];
+        int[] to = new int[numdim];
+
+        for (int i = 0, j = 0; i < ranges.length && j < numdim; i+= 2, j++)
+        {
+            from[j] = ranges[i];
+            to[j] = ranges[i+1];
+        }
+
+        //sort the rows
+        for (int i = 0; i < numdim; i++)
+        {
+            int[] vals = new int[rows.length];
+            for (int j = 0; j < rows.length; j++)
+                vals[j] = (Integer)rows[j][i];
+
+            if (sort)
+                Arrays.sort(vals);
+
+            for (int j = from[i]; j < to[i]; j++)
+                assertEquals(j, vals[j - from[i]]);
+        }
+    }
+
+    /**
+     * Test LIMIT when static columns are present (#6956),
+     * migrated from cql_tests.py:TestCQL.static_with_limit_test()
+     */
+    @Test
+    public void testStaticColumnsWithLimit() throws Throwable
+    {
+        createTable(" CREATE TABLE %s (k int, s int static, v int, PRIMARY KEY (k, v))");
+
+        execute("INSERT INTO %s (k, s) VALUES(0, 42)");
+        for (int i = 0; i < 4; i++)
+            execute("INSERT INTO %s(k, v) VALUES(0, ?)", i);
+
+        assertRows(execute("SELECT * FROM %s WHERE k = 0 LIMIT 1"),
+                   row(0, 0, 42));
+        assertRows(execute("SELECT * FROM %s WHERE k = 0 LIMIT 2"),
+                   row(0, 0, 42),
+                   row(0, 1, 42));
+        assertRows(execute("SELECT * FROM %s WHERE k = 0 LIMIT 3"),
+                   row(0, 0, 42),
+                   row(0, 1, 42),
+                   row(0, 2, 42));
+    }
+
+    /**
+     * Test for bug of #7455,
+     * migrated from cql_tests.py:TestCQL.static_with_empty_clustering_test()
+     */
+    @Test
+    public void testStaticColumnsWithEmptyClustering() throws Throwable
+    {
+        createTable("CREATE TABLE %s (pkey text, ckey text, value text, static_value text static, PRIMARY KEY(pkey, ckey))");
+
+        execute("INSERT INTO %s (pkey, static_value) VALUES ('partition1', 'static value')");
+        execute("INSERT INTO %s (pkey, ckey, value) VALUES('partition1', '', 'value')");
+
+        assertRows(execute("SELECT * FROM %s"),
+                   row("partition1", "", "static value", "value"));
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.alter_clustering_and_static_test()
+     */
+    @Test
+    public void testAlterClusteringAndStatic() throws Throwable
+    {
+        createTable("CREATE TABLE %s (bar int, PRIMARY KEY (bar))");
+
+        // We shouldn 't allow static when there is not clustering columns
+        assertInvalid("ALTER TABLE %s ADD bar2 text static");
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f797bfa4/test/unit/org/apache/cassandra/cql3/validation/entities/TimestampTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/TimestampTest.java b/test/unit/org/apache/cassandra/cql3/validation/entities/TimestampTest.java
new file mode 100644
index 0000000..95d194d
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/TimestampTest.java
@@ -0,0 +1,147 @@
+/*
+ * 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.validation.entities;
+
+import org.junit.Test;
+
+import junit.framework.Assert;
+import org.apache.cassandra.cql3.CQLTester;
+
+import static junit.framework.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+public class TimestampTest extends CQLTester
+{
+    @Test
+    public void testNegativeTimestamps() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, v int)");
+
+        execute("INSERT INTO %s (k, v) VALUES (?, ?) USING TIMESTAMP ?", 1, 1, -42L);
+        assertRows(execute("SELECT writetime(v) FROM %s WHERE k = ?", 1),
+            row(-42L)
+        );
+
+        assertInvalid("INSERT INTO %s (k, v) VALUES (?, ?) USING TIMESTAMP ?", 2, 2, Long.MIN_VALUE);
+    }
+
+    /**
+     * Test timestmp and ttl
+     * migrated from cql_tests.py:TestCQL.timestamp_and_ttl_test()
+     */
+    @Test
+    public void testTimestampTTL() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, c text, d text)");
+
+        execute("INSERT INTO %s (k, c) VALUES (1, 'test')");
+        execute("INSERT INTO %s (k, c) VALUES (2, 'test') USING TTL 400");
+
+        Object[][] res = getRows(execute("SELECT k, c, writetime(c), ttl(c) FROM %s"));
+        Assert.assertEquals(2, res.length);
+
+        for (Object[] r : res)
+        {
+            assertTrue(r[2] instanceof Integer || r[2] instanceof Long);
+            if (r[0].equals(1))
+                assertNull(r[3]);
+            else
+                assertTrue(r[3] instanceof Integer || r[2] instanceof Long);
+        }
+
+
+        // wrap writetime(), ttl() in other functions (test for CASSANDRA-8451)
+        res = getRows(execute("SELECT k, c, blobAsBigint(bigintAsBlob(writetime(c))), ttl(c) FROM %s"));
+        Assert.assertEquals(2, res.length);
+
+        for (Object[] r : res)
+        {
+            assertTrue(r[2] instanceof Integer || r[2] instanceof Long);
+            if (r[0].equals(1))
+                assertNull(r[3]);
+            else
+                assertTrue(r[3] instanceof Integer || r[2] instanceof Long);
+        }
+
+        res = getRows(execute("SELECT k, c, writetime(c), blobAsInt(intAsBlob(ttl(c))) FROM %s"));
+        Assert.assertEquals(2, res.length);
+
+
+        for (Object[] r : res)
+        {
+            assertTrue(r[2] instanceof Integer || r[2] instanceof Long);
+            if (r[0].equals(1))
+                assertNull(r[3]);
+            else
+                assertTrue(r[3] instanceof Integer || r[2] instanceof Long);
+        }
+
+        assertInvalid("SELECT k, c, writetime(k) FROM %s");
+
+        assertRows(execute("SELECT k, d, writetime(d) FROM %s WHERE k = 1"),
+                   row(1, null, null));
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.invalid_custom_timestamp_test()
+     */
+    @Test
+    public void testInvalidCustomTimestamp() throws Throwable
+    {
+        // Conditional updates
+        createTable("CREATE TABLE %s (k int, v int, PRIMARY KEY (k, v))");
+
+        execute("BEGIN BATCH " +
+                "INSERT INTO %1$s (k, v) VALUES(0, 0) IF NOT EXISTS; " +
+                "INSERT INTO %1$s (k, v) VALUES(0, 1) IF NOT EXISTS; " +
+                "APPLY BATCH");
+
+        assertInvalid("BEGIN BATCH " +
+                      "INSERT INTO %1$s (k, v) VALUES(0, 2) IF NOT EXISTS USING TIMESTAMP 1; " +
+                      "INSERT INTO %1$s (k, v) VALUES(0, 3) IF NOT EXISTS; " +
+                      "APPLY BATCH");
+        assertInvalid("BEGIN BATCH " +
+                      "USING TIMESTAMP 1 INSERT INTO %1$s (k, v) VALUES(0, 4) IF NOT EXISTS; " +
+                      "INSERT INTO %1$s (k, v) VALUES(0, 1) IF NOT EXISTS; " +
+                      "APPLY BATCH");
+
+        execute("INSERT INTO %s (k, v) VALUES(1, 0) IF NOT EXISTS");
+        assertInvalid("INSERT INTO %s (k, v) VALUES(1, 1) IF NOT EXISTS USING TIMESTAMP 5");
+
+        // Counters
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, c counter)");
+
+        execute("UPDATE %s SET c = c + 1 WHERE k = 0");
+        assertInvalid("UPDATE %s USING TIMESTAMP 10 SET c = c + 1 WHERE k = 0");
+
+        execute("BEGIN COUNTER BATCH " +
+                "UPDATE %1$s SET c = c + 1 WHERE k = 0; " +
+                "UPDATE %1$s SET c = c + 1 WHERE k = 0; " +
+                "APPLY BATCH");
+
+        assertInvalid("BEGIN COUNTER BATCH " +
+                      "UPDATE %1$s USING TIMESTAMP 3 SET c = c + 1 WHERE k = 0; " +
+                      "UPDATE %1$s SET c = c + 1 WHERE k = 0; " +
+                      "APPLY BATCH");
+
+        assertInvalid("BEGIN COUNTER BATCH " +
+                      "USING TIMESTAMP 3 UPDATE %1$s SET c = c + 1 WHERE k = 0; " +
+                      "UPDATE %1$s SET c = c + 1 WHERE k = 0; " +
+                      "APPLY BATCH");
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f797bfa4/test/unit/org/apache/cassandra/cql3/validation/entities/TimeuuidTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/TimeuuidTest.java b/test/unit/org/apache/cassandra/cql3/validation/entities/TimeuuidTest.java
new file mode 100644
index 0000000..0f1f8f0
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/TimeuuidTest.java
@@ -0,0 +1,81 @@
+/*
+ * 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.validation.entities;
+
+import java.util.Date;
+import java.util.UUID;
+
+import org.junit.Test;
+
+import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.exceptions.SyntaxException;
+import org.apache.cassandra.utils.UUIDGen;
+
+import static org.junit.Assert.assertEquals;
+
+public class TimeuuidTest extends CQLTester
+{
+    /**
+     * Migrated from cql_tests.py:TestCQL.timeuuid_test()
+     */
+    @Test
+    public void testTimeuuid() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int, t timeuuid, PRIMARY KEY(k, t))");
+
+        assertInvalidThrow(SyntaxException.class, "INSERT INTO %s (k, t) VALUES (0, 2012-11-07 18:18:22-0800)");
+
+        for (int i = 0; i < 4; i++)
+            execute("INSERT INTO %s (k, t) VALUES (0, now())");
+
+        Object[][] rows = getRows(execute("SELECT * FROM %s"));
+        assertEquals(4, rows.length);
+
+        assertRowCount(execute("SELECT * FROM %s WHERE k = 0 AND t >= ?", rows[0][1]), 4);
+
+        assertEmpty(execute("SELECT * FROM %s WHERE k = 0 AND t < ?", rows[0][1]));
+
+        assertRowCount(execute("SELECT * FROM %s WHERE k = 0 AND t > ? AND t <= ?", rows[0][1], rows[2][1]), 2);
+
+        assertRowCount(execute("SELECT * FROM %s WHERE k = 0 AND t = ?", rows[0][1]), 1);
+
+        assertInvalid("SELECT dateOf(k) FROM %s WHERE k = 0 AND t = ?", rows[0][1]);
+
+        for (int i = 0; i < 4; i++)
+        {
+            long timestamp = UUIDGen.unixTimestamp((UUID) rows[i][1]);
+            assertRows(execute("SELECT dateOf(t), unixTimestampOf(t) FROM %s WHERE k = 0 AND t = ?", rows[i][1]),
+                       row(new Date(timestamp), timestamp));
+        }
+
+        assertEmpty(execute("SELECT t FROM %s WHERE k = 0 AND t > maxTimeuuid(1234567) AND t < minTimeuuid('2012-11-07 18:18:22-0800')"));
+    }
+
+    /**
+     * Test for 5386,
+     * migrated from cql_tests.py:TestCQL.function_and_reverse_type_test()
+     */
+    @Test
+    public void testDescClusteringOnTimeuuid() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int, c timeuuid, v int, PRIMARY KEY (k, c)) WITH CLUSTERING ORDER BY (c DESC)");
+
+        execute("INSERT INTO %s (k, c, v) VALUES (0, now(), 0)");
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f797bfa4/test/unit/org/apache/cassandra/cql3/validation/entities/TupleTypeTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/TupleTypeTest.java b/test/unit/org/apache/cassandra/cql3/validation/entities/TupleTypeTest.java
new file mode 100644
index 0000000..2d145de
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/TupleTypeTest.java
@@ -0,0 +1,158 @@
+/*
+ * 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.validation.entities;
+
+import org.junit.Test;
+
+import org.apache.cassandra.cql3.CQLTester;
+
+public class TupleTypeTest extends CQLTester
+{
+    @Test
+    public void testTuplePutAndGet() throws Throwable
+    {
+        String[] valueTypes = {"frozen<tuple<int, text, double>>", "tuple<int, text, double>"};
+        for (String valueType : valueTypes)
+        {
+            createTable("CREATE TABLE %s (k int PRIMARY KEY, t " + valueType + ")");
+
+            execute("INSERT INTO %s (k, t) VALUES (?, ?)", 0, tuple(3, "foo", 3.4));
+            execute("INSERT INTO %s (k, t) VALUES (?, ?)", 1, tuple(8, "bar", 0.2));
+            assertAllRows(row(1, tuple(8, "bar", 0.2)),
+                          row(0, tuple(3, "foo", 3.4))
+            );
+
+            // nulls
+            execute("INSERT INTO %s (k, t) VALUES (?, ?)", 2, tuple(5, null, 3.4));
+            assertRows(execute("SELECT * FROM %s WHERE k=?", 2),
+                       row(2, tuple(5, null, 3.4))
+            );
+
+            // incomplete tuple
+            execute("INSERT INTO %s (k, t) VALUES (?, ?)", 3, tuple(5, "bar"));
+            assertRows(execute("SELECT * FROM %s WHERE k=?", 3),
+                       row(3, tuple(5, "bar"))
+            );
+        }
+    }
+
+    @Test
+    public void testNestedTuple() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, t frozen<tuple<int, tuple<text, double>>>)");
+
+        execute("INSERT INTO %s (k, t) VALUES (?, ?)", 0, tuple(3, tuple("foo", 3.4)));
+        execute("INSERT INTO %s (k, t) VALUES (?, ?)", 1, tuple(8, tuple("bar", 0.2)));
+        assertAllRows(
+            row(1, tuple(8, tuple("bar", 0.2))),
+            row(0, tuple(3, tuple("foo", 3.4)))
+        );
+    }
+
+    @Test
+    public void testTupleInPartitionKey() throws Throwable
+    {
+        createTable("CREATE TABLE %s (t frozen<tuple<int, text>> PRIMARY KEY)");
+
+        execute("INSERT INTO %s (t) VALUES (?)", tuple(3, "foo"));
+        assertAllRows(row(tuple(3, "foo")));
+    }
+
+    @Test
+    public void testTupleInClusteringKey() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int, t frozen<tuple<int, text>>, PRIMARY KEY (k, t))");
+
+        execute("INSERT INTO %s (k, t) VALUES (?, ?)", 0, tuple(5, "bar"));
+        execute("INSERT INTO %s (k, t) VALUES (?, ?)", 0, tuple(3, "foo"));
+        execute("INSERT INTO %s (k, t) VALUES (?, ?)", 0, tuple(6, "bar"));
+        execute("INSERT INTO %s (k, t) VALUES (?, ?)", 0, tuple(5, "foo"));
+
+        assertAllRows(
+            row(0, tuple(3, "foo")),
+            row(0, tuple(5, "bar")),
+            row(0, tuple(5, "foo")),
+            row(0, tuple(6, "bar"))
+        );
+    }
+
+    @Test
+    public void testInvalidQueries() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, t frozen<tuple<int, text, double>>)");
+
+        assertInvalidSyntax("INSERT INTO %s (k, t) VALUES (0, ())");
+        assertInvalid("INSERT INTO %s (k, t) VALUES (0, (2, 'foo', 3.1, 'bar'))");
+    }
+
+    /**
+     * Test the syntax introduced by #4851,
+     * migrated from cql_tests.py:TestCQL.tuple_notation_test()
+     */
+    @Test
+    public void testTupleNotation() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int, v1 int, v2 int, v3 int, PRIMARY KEY (k, v1, v2, v3))");
+        for (int i = 0; i < 2; i++)
+            for (int j = 0; j < 2; j++)
+                for (int k = 0; k < 2; k++)
+                    execute("INSERT INTO %s (k, v1, v2, v3) VALUES (0, ?, ?, ?)", i, j, k);
+
+        assertRows(execute("SELECT v1, v2, v3 FROM %s WHERE k = 0"),
+                   row(0, 0, 0),
+                   row(0, 0, 1),
+                   row(0, 1, 0),
+                   row(0, 1, 1),
+                   row(1, 0, 0),
+                   row(1, 0, 1),
+                   row(1, 1, 0),
+                   row(1, 1, 1));
+
+        assertRows(execute("SELECT v1, v2, v3 FROM %s WHERE k = 0 AND (v1, v2, v3) >= (1, 0, 1)"),
+                   row(1, 0, 1),
+                   row(1, 1, 0),
+                   row(1, 1, 1));
+        assertRows(execute("SELECT v1, v2, v3 FROM %s WHERE k = 0 AND (v1, v2) >= (1, 1)"),
+                   row(1, 1, 0),
+                   row(1, 1, 1));
+
+        assertRows(execute("SELECT v1, v2, v3 FROM %s WHERE k = 0 AND (v1, v2) > (0, 1) AND (v1, v2, v3) <= (1, 1, 0)"),
+                   row(1, 0, 0),
+                   row(1, 0, 1),
+                   row(1, 1, 0));
+
+        assertInvalid("SELECT v1, v2, v3 FROM %s WHERE k = 0 AND (v1, v3) > (1, 0)");
+    }
+
+    /**
+     * Test for CASSANDRA-8062,
+     * migrated from cql_tests.py:TestCQL.test_v2_protocol_IN_with_tuples()
+     */
+    @Test
+    public void testSelectInStatementWithTuples() throws Throwable
+    {   // TODO - the dtest was using v2 protocol
+        createTable("CREATE TABLE %s (k int, c1 int, c2 text, PRIMARY KEY (k, c1, c2))");
+        execute("INSERT INTO %s (k, c1, c2) VALUES (0, 0, 'a')");
+        execute("INSERT INTO %s (k, c1, c2) VALUES (0, 0, 'b')");
+        execute("INSERT INTO %s (k, c1, c2) VALUES (0, 0, 'c')");
+
+        assertRows(execute("SELECT * FROM %s WHERE k=0 AND (c1, c2) IN ((0, 'b'), (0, 'c'))"),
+                   row(0, 0, "b"),
+                   row(0, 0, "c"));
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f797bfa4/test/unit/org/apache/cassandra/cql3/validation/entities/TypeTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/TypeTest.java b/test/unit/org/apache/cassandra/cql3/validation/entities/TypeTest.java
new file mode 100644
index 0000000..f27cca8
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/TypeTest.java
@@ -0,0 +1,88 @@
+/*
+ * 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.validation.entities;
+
+import org.apache.cassandra.cql3.UntypedResultSet;
+import org.apache.cassandra.cql3.CQLTester;
+
+import org.junit.Test;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+public class TypeTest extends CQLTester
+{
+    @Test
+    public void testNowToUUIDCompatibility() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int, b uuid, PRIMARY KEY (a, b))");
+        execute("INSERT INTO %s (a, b) VALUES (0, now())");
+        UntypedResultSet results = execute("SELECT * FROM %s WHERE a=0 AND b < now()");
+        assertEquals(1, results.size());
+    }
+
+    @Test
+    public void testDateCompatibility() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int, b timestamp, c bigint, d varint, PRIMARY KEY (a, b, c, d))");
+        execute("INSERT INTO %s (a, b, c, d) VALUES (0, unixTimestampOf(now()), dateOf(now()), dateOf(now()))");
+        UntypedResultSet results = execute("SELECT * FROM %s WHERE a=0 AND b < unixTimestampOf(now())");
+        assertEquals(1, results.size());
+    }
+
+    @Test
+    public void testReversedTypeCompatibility() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int, b timeuuid, PRIMARY KEY (a, b)) WITH CLUSTERING ORDER BY (b DESC)");
+        execute("INSERT INTO %s (a, b) VALUES (0, now())");
+        UntypedResultSet results = execute("SELECT * FROM %s WHERE a=0 AND b < now()");
+        assertEquals(1, results.size());
+    }
+
+    @Test
+    // tests CASSANDRA-7797
+    public void testAlterReversedColumn() throws Throwable
+    {
+        createTable("CREATE TABLE IF NOT EXISTS %s (a int, b 'org.apache.cassandra.db.marshal.DateType', PRIMARY KEY (a, b)) WITH CLUSTERING ORDER BY (b DESC)");
+        alterTable("ALTER TABLE %s ALTER b TYPE 'org.apache.cassandra.db.marshal.ReversedType(org.apache.cassandra.db.marshal.TimestampType)'");
+    }
+
+    @Test
+    public void testIncompatibleReversedTypes() throws Throwable
+    {
+        createTable("CREATE TABLE IF NOT EXISTS %s (a int, b 'org.apache.cassandra.db.marshal.DateType', PRIMARY KEY (a, b)) WITH CLUSTERING ORDER BY (b DESC)");
+        try
+        {
+            alterTable("ALTER TABLE %s ALTER b TYPE 'org.apache.cassandra.db.marshal.ReversedType(org.apache.cassandra.db.marshal.TimeUUIDType)'");
+            fail("Expected error for ALTER statement");
+        }
+        catch (RuntimeException e) { }
+    }
+
+    @Test
+    public void testReversedAndNonReversed() throws Throwable
+    {
+        createTable("CREATE TABLE IF NOT EXISTS %s (a int, b 'org.apache.cassandra.db.marshal.DateType', PRIMARY KEY (a, b))");
+        try
+        {
+            alterTable("ALTER TABLE %s ALTER b TYPE 'org.apache.cassandra.db.marshal.ReversedType(org.apache.cassandra.db.marshal.DateType)'");
+            fail("Expected error for ALTER statement");
+        }
+        catch (RuntimeException e) { }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f797bfa4/test/unit/org/apache/cassandra/cql3/validation/entities/UserTypesTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/UserTypesTest.java b/test/unit/org/apache/cassandra/cql3/validation/entities/UserTypesTest.java
new file mode 100644
index 0000000..ee647d6
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/UserTypesTest.java
@@ -0,0 +1,389 @@
+/*
+ * 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.validation.entities;
+
+import java.util.UUID;
+
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.dht.ByteOrderedPartitioner;
+
+public class UserTypesTest extends CQLTester
+{
+    @BeforeClass
+    public static void setUpClass()
+    {
+        DatabaseDescriptor.setPartitioner(new ByteOrderedPartitioner());
+    }
+
+    @Test
+    public void testInvalidField() throws Throwable
+    {
+        String myType = createType("CREATE TYPE %s (f int)");
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, v frozen<" + myType + ">)");
+
+        // 's' is not a field of myType
+        assertInvalid("INSERT INTO %s (k, v) VALUES (?, {s : ?})", 0, 1);
+    }
+
+    @Test
+    public void testCassandra8105() throws Throwable
+    {
+        String ut1 = createType("CREATE TYPE %s (a int, b int)");
+        String ut2 = createType("CREATE TYPE %s (j frozen<" + KEYSPACE + "." + ut1 + ">, k int)");
+        createTable("CREATE TABLE %s (x int PRIMARY KEY, y set<frozen<" + KEYSPACE + "." + ut2 + ">>)");
+        execute("INSERT INTO %s (x, y) VALUES (1, { { k: 1 } })");
+
+        String ut3 = createType("CREATE TYPE %s (a int, b int)");
+        String ut4 = createType("CREATE TYPE %s (j frozen<" + KEYSPACE + "." + ut3 + ">, k int)");
+        createTable("CREATE TABLE %s (x int PRIMARY KEY, y list<frozen<" + KEYSPACE + "." + ut4 + ">>)");
+        execute("INSERT INTO %s (x, y) VALUES (1, [ { k: 1 } ])");
+
+        String ut5 = createType("CREATE TYPE %s (a int, b int)");
+        String ut6 = createType("CREATE TYPE %s (i int, j frozen<" + KEYSPACE + "." + ut5 + ">)");
+        createTable("CREATE TABLE %s (x int PRIMARY KEY, y set<frozen<" + KEYSPACE + "." + ut6 + ">>)");
+        execute("INSERT INTO %s (x, y) VALUES (1, { { i: 1 } })");
+    }
+
+    @Test
+    public void testFor7684() throws Throwable
+    {
+        String myType = createType("CREATE TYPE %s (x double)");
+        createTable("CREATE TABLE %s (k int, v frozen<" + myType + ">, b boolean static, PRIMARY KEY (k, v))");
+
+        execute("INSERT INTO %s(k, v) VALUES (?, {x:?})", 1, -104.99251);
+        execute("UPDATE %s SET b = ? WHERE k = ?", true, 1);
+
+        assertRows(execute("SELECT v.x FROM %s WHERE k = ? AND v = {x:?}", 1, -104.99251),
+            row(-104.99251)
+        );
+
+        flush();
+
+        assertRows(execute("SELECT v.x FROM %s WHERE k = ? AND v = {x:?}", 1, -104.99251),
+                   row(-104.99251)
+        );
+    }
+
+    @Test
+    public void testCreateInvalidTablesWithUDT() throws Throwable
+    {
+        String myType = createType("CREATE TYPE %s (f int)");
+
+        // Using a UDT without frozen shouldn't work
+        assertInvalidMessage("Non-frozen User-Defined types are not supported, please use frozen<>",
+                             "CREATE TABLE " + KEYSPACE + ".wrong (k int PRIMARY KEY, v " + KEYSPACE + '.' + myType + ")");
+
+        assertInvalidMessage("Statement on keyspace " + KEYSPACE + " cannot refer to a user type in keyspace otherkeyspace;" +
+                             " user types can only be used in the keyspace they are defined in",
+                             "CREATE TABLE " + KEYSPACE + ".wrong (k int PRIMARY KEY, v frozen<otherKeyspace.myType>)");
+
+        assertInvalidMessage("Unknown type " + KEYSPACE + ".unknowntype",
+                             "CREATE TABLE " + KEYSPACE + ".wrong (k int PRIMARY KEY, v frozen<" + KEYSPACE + '.' + "unknownType>)");
+    }
+
+    @Test
+    public void testAlterUDT() throws Throwable
+    {
+        String myType = KEYSPACE + '.' + createType("CREATE TYPE %s (a int)");
+        createTable("CREATE TABLE %s (a int PRIMARY KEY, b frozen<" + myType + ">)");
+        execute("INSERT INTO %s (a, b) VALUES (1, {a: 1})");
+
+        assertRows(execute("SELECT b.a FROM %s"), row(1));
+
+        flush();
+
+        execute("ALTER TYPE " + myType + " ADD b int");
+        execute("INSERT INTO %s (a, b) VALUES (2, {a: 2, b :2})");
+
+        assertRows(execute("SELECT b.a, b.b FROM %s"),
+                   row(1, null),
+                   row(2, 2));
+
+        flush();
+
+        assertRows(execute("SELECT b.a, b.b FROM %s"),
+                   row(1, null),
+                   row(2, 2));
+    }
+
+    @Test
+    public void testAlteringUserTypeNestedWithinMap() throws Throwable
+    {
+        // test frozen and non-frozen collections
+        String[] columnTypePrefixes = {"frozen<map<text, ", "map<text, frozen<"};
+        for (String columnTypePrefix : columnTypePrefixes)
+        {
+            String ut1 = createType("CREATE TYPE %s (a int)");
+            String columnType = columnTypePrefix + KEYSPACE + "." + ut1 + ">>";
+
+            createTable("CREATE TABLE %s (x int PRIMARY KEY, y " + columnType + ")");
+
+            execute("INSERT INTO %s (x, y) VALUES(1, {'firstValue':{a:1}})");
+            assertRows(execute("SELECT * FROM %s"), row(1, map("firstValue", userType(1))));
+            flush();
+
+            execute("ALTER TYPE " + KEYSPACE + "." + ut1 + " ADD b int");
+            execute("INSERT INTO %s (x, y) VALUES(2, {'secondValue':{a:2, b:2}})");
+            execute("INSERT INTO %s (x, y) VALUES(3, {'thirdValue':{a:3}})");
+            execute("INSERT INTO %s (x, y) VALUES(4, {'fourthValue':{b:4}})");
+
+            assertRows(execute("SELECT * FROM %s"),
+                    row(1, map("firstValue", userType(1))),
+                    row(2, map("secondValue", userType(2, 2))),
+                    row(3, map("thirdValue", userType(3, null))),
+                    row(4, map("fourthValue", userType(null, 4))));
+
+            flush();
+
+            assertRows(execute("SELECT * FROM %s"),
+                    row(1, map("firstValue", userType(1))),
+                    row(2, map("secondValue", userType(2, 2))),
+                    row(3, map("thirdValue", userType(3, null))),
+                    row(4, map("fourthValue", userType(null, 4))));
+        }
+    }
+
+    @Test
+    public void testAlteringUserTypeNestedWithinSet() throws Throwable
+    {
+        // test frozen and non-frozen collections
+        String[] columnTypePrefixes = {"frozen<set<", "set<frozen<"};
+        for (String columnTypePrefix : columnTypePrefixes)
+        {
+            String ut1 = createType("CREATE TYPE %s (a int)");
+            String columnType = columnTypePrefix + KEYSPACE + "." + ut1 + ">>";
+
+            createTable("CREATE TABLE %s (x int PRIMARY KEY, y " + columnType + ")");
+
+            execute("INSERT INTO %s (x, y) VALUES(1, {1} )");
+            assertRows(execute("SELECT * FROM %s"), row(1, set(userType(1))));
+            flush();
+
+            execute("ALTER TYPE " + KEYSPACE + "." + ut1 + " ADD b int");
+            execute("INSERT INTO %s (x, y) VALUES(2, {{a:2, b:2}})");
+            execute("INSERT INTO %s (x, y) VALUES(3, {{a:3}})");
+            execute("INSERT INTO %s (x, y) VALUES(4, {{b:4}})");
+
+            assertRows(execute("SELECT * FROM %s"),
+                    row(1, set(userType(1))),
+                    row(2, set(userType(2, 2))),
+                    row(3, set(userType(3, null))),
+                    row(4, set(userType(null, 4))));
+
+            flush();
+
+            assertRows(execute("SELECT * FROM %s"),
+                    row(1, set(userType(1))),
+                    row(2, set(userType(2, 2))),
+                    row(3, set(userType(3, null))),
+                    row(4, set(userType(null, 4))));
+        }
+    }
+
+    @Test
+    public void testAlteringUserTypeNestedWithinList() throws Throwable
+    {
+        // test frozen and non-frozen collections
+        String[] columnTypePrefixes = {"frozen<list<", "list<frozen<"};
+        for (String columnTypePrefix : columnTypePrefixes)
+        {
+            String ut1 = createType("CREATE TYPE %s (a int)");
+            String columnType = columnTypePrefix + KEYSPACE + "." + ut1 + ">>";
+
+            createTable("CREATE TABLE %s (x int PRIMARY KEY, y " + columnType + ")");
+
+            execute("INSERT INTO %s (x, y) VALUES(1, [1] )");
+            assertRows(execute("SELECT * FROM %s"), row(1, list(userType(1))));
+            flush();
+
+            execute("ALTER TYPE " + KEYSPACE + "." + ut1 + " ADD b int");
+            execute("INSERT INTO %s (x, y) VALUES(2, [{a:2, b:2}])");
+            execute("INSERT INTO %s (x, y) VALUES(3, [{a:3}])");
+            execute("INSERT INTO %s (x, y) VALUES(4, [{b:4}])");
+
+            assertRows(execute("SELECT * FROM %s"),
+                    row(1, list(userType(1))),
+                    row(2, list(userType(2, 2))),
+                    row(3, list(userType(3, null))),
+                    row(4, list(userType(null, 4))));
+
+            flush();
+
+            assertRows(execute("SELECT * FROM %s"),
+                    row(1, list(userType(1))),
+                    row(2, list(userType(2, 2))),
+                    row(3, list(userType(3, null))),
+                    row(4, list(userType(null, 4))));
+        }
+    }
+
+    @Test
+    public void testAlteringUserTypeNestedWithinTuple() throws Throwable
+    {
+        String type = createType("CREATE TYPE %s (a int, b int)");
+
+        createTable("CREATE TABLE %s (a int PRIMARY KEY, b frozen<tuple<int, " + KEYSPACE + "." + type + ">>)");
+
+        execute("INSERT INTO %s (a, b) VALUES(1, (1, {a:1, b:1}))");
+        assertRows(execute("SELECT * FROM %s"), row(1, tuple(1, userType(1, 1))));
+        flush();
+
+        execute("ALTER TYPE " + KEYSPACE + "." + type + " ADD c int");
+        execute("INSERT INTO %s (a, b) VALUES(2, (2, {a: 2, b: 2, c: 2}))");
+        execute("INSERT INTO %s (a, b) VALUES(3, (3, {a: 3, b: 3}))");
+        execute("INSERT INTO %s (a, b) VALUES(4, (4, {b:4}))");
+
+        assertRows(execute("SELECT * FROM %s"),
+                   row(1, tuple(1, userType(1, 1))),
+                   row(2, tuple(2, userType(2, 2, 2))),
+                   row(3, tuple(3, userType(3, 3, null))),
+                   row(4, tuple(4, userType(null, 4, null))));
+
+        flush();
+
+        assertRows(execute("SELECT * FROM %s"),
+                   row(1, tuple(1, userType(1, 1))),
+                   row(2, tuple(2, userType(2, 2, 2))),
+                   row(3, tuple(3, userType(3, 3, null))),
+                   row(4, tuple(4, userType(null, 4, null))));
+    }
+
+    @Test
+    public void testAlteringUserTypeNestedWithinNestedTuple() throws Throwable
+    {
+        String type = createType("CREATE TYPE %s (a int, b int)");
+
+        createTable("CREATE TABLE %s (a int PRIMARY KEY, b frozen<tuple<int, tuple<int, " + KEYSPACE + "." + type + ">>>)");
+
+        execute("INSERT INTO %s (a, b) VALUES(1, (1, (1, {a:1, b:1})))");
+        assertRows(execute("SELECT * FROM %s"), row(1, tuple(1, tuple(1, userType(1, 1)))));
+        flush();
+
+        execute("ALTER TYPE " + KEYSPACE + "." + type + " ADD c int");
+        execute("INSERT INTO %s (a, b) VALUES(2, (2, (1, {a: 2, b: 2, c: 2})))");
+        execute("INSERT INTO %s (a, b) VALUES(3, (3, (1, {a: 3, b: 3})))");
+        execute("INSERT INTO %s (a, b) VALUES(4, (4, (1, {b:4})))");
+
+        assertRows(execute("SELECT * FROM %s"),
+                   row(1, tuple(1, tuple(1, userType(1, 1)))),
+                   row(2, tuple(2, tuple(1, userType(2, 2, 2)))),
+                   row(3, tuple(3, tuple(1, userType(3, 3, null)))),
+                   row(4, tuple(4, tuple(1, userType(null, 4, null)))));
+
+        flush();
+
+        assertRows(execute("SELECT * FROM %s"),
+                   row(1, tuple(1, tuple(1, userType(1, 1)))),
+                   row(2, tuple(2, tuple(1, userType(2, 2, 2)))),
+                   row(3, tuple(3, tuple(1, userType(3, 3, null)))),
+                   row(4, tuple(4, tuple(1, userType(null, 4, null)))));
+    }
+
+    @Test
+    public void testAlteringUserTypeNestedWithinUserType() throws Throwable
+    {
+        String type = createType("CREATE TYPE %s (a int, b int)");
+        String otherType = createType("CREATE TYPE %s (x frozen<" + KEYSPACE + "." + type + ">)");
+
+        createTable("CREATE TABLE %s (a int PRIMARY KEY, b frozen<" + KEYSPACE + "." + otherType + ">)");
+
+        execute("INSERT INTO %s (a, b) VALUES(1, {x: {a:1, b:1}})");
+        assertRows(execute("SELECT b.x.a, b.x.b FROM %s"), row(1, 1));
+        flush();
+
+        execute("ALTER TYPE " + KEYSPACE + "." + type + " ADD c int");
+        execute("INSERT INTO %s (a, b) VALUES(2, {x: {a: 2, b: 2, c: 2}})");
+        execute("INSERT INTO %s (a, b) VALUES(3, {x: {a: 3, b: 3}})");
+        execute("INSERT INTO %s (a, b) VALUES(4, {x: {b:4}})");
+
+        assertRows(execute("SELECT b.x.a, b.x.b, b.x.c FROM %s"),
+                   row(1, 1, null),
+                   row(2, 2, 2),
+                   row(3, 3, null),
+                   row(null, 4, null));
+
+        flush();
+
+        assertRows(execute("SELECT b.x.a, b.x.b, b.x.c FROM %s"),
+                   row(1, 1, null),
+                   row(2, 2, 2),
+                   row(3, 3, null),
+                   row(null, 4, null));
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.user_types_test()
+     */
+    @Test
+    public void testUserTypes() throws Throwable
+    {
+        UUID userID_1 = UUID.fromString("550e8400-e29b-41d4-a716-446655440000");
+
+        String addressType = createType("CREATE TYPE %s (street text, city text, zip_code int, phones set<text >)");
+
+        String nameType = createType("CREATE TYPE %s (firstname text, lastname text)");
+
+        createTable("CREATE TABLE %s (id uuid PRIMARY KEY, name frozen < " + nameType + " >, addresses map < text, frozen < " + addressType + " >> )");
+
+        execute("INSERT INTO %s (id, name) VALUES(?, { firstname: 'Paul', lastname: 'smith' } )", userID_1);
+
+        assertRows(execute("SELECT name.firstname FROM %s WHERE id = ?", userID_1), row("Paul"));
+
+        execute("UPDATE %s SET addresses = addresses + { 'home': { street: '...', city:'SF', zip_code:94102, phones:{ } } } WHERE id = ?", userID_1);
+
+        // TODO: deserialize the value here and check it 's right.
+        execute("SELECT addresses FROM %s WHERE id = ? ", userID_1);
+    }
+
+    /**
+     * Test user type test that does a little more nesting,
+     * migrated from cql_tests.py:TestCQL.more_user_types_test()
+     */
+    @Test
+    public void testNestedUserTypes() throws Throwable
+    {
+        String type1 = createType("CREATE TYPE %s ( s set<text>, m map<text, text>, l list<text>)");
+
+        String type2 = createType("CREATE TYPE %s ( s set < frozen < " + type1 + " >>,)");
+
+        createTable("CREATE TABLE %s (id int PRIMARY KEY, val frozen<" + type2 + ">)");
+
+        execute("INSERT INTO %s (id, val) VALUES (0, { s : {{ s : {'foo', 'bar'}, m : { 'foo' : 'bar' }, l : ['foo', 'bar']} }})");
+
+        // TODO: check result once we have an easy way to do it. For now we just check it doesn't crash
+        execute("SELECT * FROM %s");
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.add_field_to_udt_test()
+     */
+    @Test
+    public void testAddFieldToUdt() throws Throwable
+    {
+        String typeName = createType("CREATE TYPE %s (fooint int, fooset set <text>)");
+        createTable("CREATE TABLE %s (key int PRIMARY KEY, data frozen <" + typeName + ">)");
+
+        execute("INSERT INTO %s (key, data) VALUES (1, {fooint: 1, fooset: {'2'}})");
+        execute("ALTER TYPE " + keyspace() + "." + typeName + " ADD foomap map <int,text>");
+        execute("INSERT INTO %s (key, data) VALUES (1, {fooint: 1, fooset: {'2'}, foomap: {3 : 'bar'}})");
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f797bfa4/test/unit/org/apache/cassandra/cql3/validation/miscellaneous/CrcCheckChanceTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/miscellaneous/CrcCheckChanceTest.java b/test/unit/org/apache/cassandra/cql3/validation/miscellaneous/CrcCheckChanceTest.java
new file mode 100644
index 0000000..dcea930
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/validation/miscellaneous/CrcCheckChanceTest.java
@@ -0,0 +1,160 @@
+/*
+ * 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.validation.miscellaneous;
+
+import java.util.List;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+
+import junit.framework.Assert;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.db.compaction.CompactionInterruptedException;
+import org.apache.cassandra.db.compaction.CompactionManager;
+import org.apache.cassandra.utils.FBUtilities;
+
+import org.junit.Test;
+
+
+public class CrcCheckChanceTest extends CQLTester
+{
+    @Test
+    public void testChangingCrcCheckChance() throws Throwable
+    {
+        //Start with crc_check_chance of 99%
+        createTable("CREATE TABLE %s (p text, c text, v text, s text static, PRIMARY KEY (p, c)) WITH compression = {'sstable_compression': 'LZ4Compressor', 'crc_check_chance' : 0.99}");
+
+        execute("CREATE INDEX foo ON %s(v)");
+
+        execute("INSERT INTO %s(p, c, v, s) values (?, ?, ?, ?)", "p1", "k1", "v1", "sv1");
+        execute("INSERT INTO %s(p, c, v) values (?, ?, ?)", "p1", "k2", "v2");
+        execute("INSERT INTO %s(p, s) values (?, ?)", "p2", "sv2");
+
+
+        ColumnFamilyStore cfs = Keyspace.open(CQLTester.KEYSPACE).getColumnFamilyStore(currentTable());
+        ColumnFamilyStore indexCfs = cfs.indexManager.getIndexesBackedByCfs().iterator().next();
+        cfs.forceBlockingFlush();
+
+        Assert.assertEquals(0.99, cfs.metadata.compressionParameters.getCrcCheckChance());
+        Assert.assertEquals(0.99, cfs.getSSTables().iterator().next().getCompressionMetadata().parameters.getCrcCheckChance());
+        Assert.assertEquals(0.99, indexCfs.metadata.compressionParameters.getCrcCheckChance());
+        Assert.assertEquals(0.99, indexCfs.getSSTables().iterator().next().getCompressionMetadata().parameters.getCrcCheckChance());
+
+        //Test for stack overflow
+        cfs.setCrcCheckChance(0.99);
+
+        assertRows(execute("SELECT * FROM %s WHERE p=?", "p1"),
+                row("p1", "k1", "sv1", "v1"),
+                row("p1", "k2", "sv1", "v2")
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE v=?", "v1"),
+                row("p1", "k1", "sv1", "v1")
+        );
+
+
+
+        //Write a few SSTables then Compact
+
+        execute("INSERT INTO %s(p, c, v, s) values (?, ?, ?, ?)", "p1", "k1", "v1", "sv1");
+        execute("INSERT INTO %s(p, c, v) values (?, ?, ?)", "p1", "k2", "v2");
+        execute("INSERT INTO %s(p, s) values (?, ?)", "p2", "sv2");
+
+        cfs.forceBlockingFlush();
+
+
+        execute("INSERT INTO %s(p, c, v, s) values (?, ?, ?, ?)", "p1", "k1", "v1", "sv1");
+        execute("INSERT INTO %s(p, c, v) values (?, ?, ?)", "p1", "k2", "v2");
+        execute("INSERT INTO %s(p, s) values (?, ?)", "p2", "sv2");
+
+        cfs.forceBlockingFlush();
+
+        execute("INSERT INTO %s(p, c, v, s) values (?, ?, ?, ?)", "p1", "k1", "v1", "sv1");
+        execute("INSERT INTO %s(p, c, v) values (?, ?, ?)", "p1", "k2", "v2");
+        execute("INSERT INTO %s(p, s) values (?, ?)", "p2", "sv2");
+
+        cfs.forceBlockingFlush();
+
+        cfs.forceMajorCompaction();
+
+        //Verify when we alter the value the live sstable readers hold the new one
+        alterTable("ALTER TABLE %s WITH compression = {'sstable_compression': 'LZ4Compressor', 'crc_check_chance': 0.01}");
+
+        Assert.assertEquals( 0.01, cfs.metadata.compressionParameters.getCrcCheckChance());
+        Assert.assertEquals( 0.01, cfs.getSSTables().iterator().next().getCompressionMetadata().parameters.getCrcCheckChance());
+        Assert.assertEquals( 0.01, indexCfs.metadata.compressionParameters.getCrcCheckChance());
+        Assert.assertEquals( 0.01, indexCfs.getSSTables().iterator().next().getCompressionMetadata().parameters.getCrcCheckChance());
+
+        assertRows(execute("SELECT * FROM %s WHERE p=?", "p1"),
+                row("p1", "k1", "sv1", "v1"),
+                row("p1", "k2", "sv1", "v2")
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE v=?", "v1"),
+                row("p1", "k1", "sv1", "v1")
+        );
+
+
+        //Verify the call used by JMX still works
+        cfs.setCrcCheckChance(0.03);
+        Assert.assertEquals( 0.03, cfs.metadata.compressionParameters.getCrcCheckChance());
+        Assert.assertEquals( 0.03, cfs.getSSTables().iterator().next().getCompressionMetadata().parameters.getCrcCheckChance());
+        Assert.assertEquals( 0.03, indexCfs.metadata.compressionParameters.getCrcCheckChance());
+        Assert.assertEquals( 0.03, indexCfs.getSSTables().iterator().next().getCompressionMetadata().parameters.getCrcCheckChance());
+
+    }
+
+
+    @Test
+    public void testDropDuringCompaction() throws Throwable
+    {
+        CompactionManager.instance.disableAutoCompaction();
+
+        //Start with crc_check_chance of 99%
+        createTable("CREATE TABLE %s (p text, c text, v text, s text static, PRIMARY KEY (p, c)) WITH compression = {'sstable_compression': 'LZ4Compressor', 'crc_check_chance' : 0.99}");
+
+        ColumnFamilyStore cfs = Keyspace.open(CQLTester.KEYSPACE).getColumnFamilyStore(currentTable());
+
+        //Write a few SSTables then Compact, and drop
+        for (int i = 0; i < 100; i++)
+        {
+            execute("INSERT INTO %s(p, c, v, s) values (?, ?, ?, ?)", "p1", "k1", "v1", "sv1");
+            execute("INSERT INTO %s(p, c, v) values (?, ?, ?)", "p1", "k2", "v2");
+            execute("INSERT INTO %s(p, s) values (?, ?)", "p2", "sv2");
+
+            cfs.forceBlockingFlush();
+        }
+
+        DatabaseDescriptor.setCompactionThroughputMbPerSec(1);
+        List<Future<?>> futures = CompactionManager.instance.submitMaximal(cfs, CompactionManager.GC_ALL); 
+        execute("DROP TABLE %s");
+
+        try
+        {
+            FBUtilities.waitOnFutures(futures);
+        }
+        catch (Throwable t)
+        {
+            if (!(t.getCause() instanceof ExecutionException) || !(t.getCause().getCause() instanceof CompactionInterruptedException))
+                throw t;
+        }
+    }
+}
+

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f797bfa4/test/unit/org/apache/cassandra/cql3/validation/miscellaneous/OverflowTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/miscellaneous/OverflowTest.java b/test/unit/org/apache/cassandra/cql3/validation/miscellaneous/OverflowTest.java
new file mode 100644
index 0000000..5b43599
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/validation/miscellaneous/OverflowTest.java
@@ -0,0 +1,331 @@
+/*
+ * 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.validation.miscellaneous;
+
+import java.math.BigInteger;
+
+import org.junit.Test;
+
+import static junit.framework.Assert.assertFalse;
+import static junit.framework.Assert.assertNull;
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.exceptions.SyntaxException;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+/**
+ * Any tests that do not fit in any other category,
+ * migrated from python dtests, CASSANDRA-9160
+ **/
+public class OverflowTest extends CQLTester
+{
+    /**
+     * Test support for nulls
+     * migrated from cql_tests.py:TestCQL.null_support_test()
+     */
+    @Test
+    public void testNullSupport() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int, c int, v1 int, v2 set<text>, PRIMARY KEY (k, c))");
+
+        execute("INSERT INTO %s (k, c, v1, v2) VALUES (0, 0, null, {'1', '2'})");
+        execute("INSERT INTO %s (k, c, v1) VALUES (0, 1, 1)");
+
+        assertRows(execute("SELECT * FROM %s"),
+                   row(0, 0, null, set("1", "2")),
+                   row(0, 1, 1, null));
+
+        execute("INSERT INTO %s (k, c, v1) VALUES (0, 1, null)");
+        execute("INSERT INTO %s (k, c, v2) VALUES (0, 0, null)");
+
+        assertRows(execute("SELECT * FROM %s"),
+                   row(0, 0, null, null),
+                   row(0, 1, null, null));
+
+        assertInvalid("INSERT INTO %s (k, c, v2) VALUES (0, 2, {1, null})");
+        assertInvalid("SELECT * FROM %s WHERE k = null");
+        assertInvalid("INSERT INTO %s (k, c, v2) VALUES (0, 0, { 'foo', 'bar', null })");
+    }
+
+    /**
+     * Test reserved keywords
+     * migrated from cql_tests.py:TestCQL.reserved_keyword_test()
+     */
+    @Test
+    public void testReservedKeywords() throws Throwable
+    {
+        createTable("CREATE TABLE %s (key text PRIMARY KEY, count counter)");
+
+        String tableName = createTableName();
+        assertInvalidThrow(SyntaxException.class, String.format("CREATE TABLE %s.%s (select text PRIMARY KEY, x int)", keyspace(), tableName));
+    }
+
+    /**
+     * Test identifiers
+     * migrated from cql_tests.py:TestCQL.identifier_test()
+     */
+    @Test
+    public void testIdentifiers() throws Throwable
+    {
+        createTable("CREATE TABLE %s (key_23 int PRIMARY KEY, CoLuMn int)");
+
+        execute("INSERT INTO %s (Key_23, Column) VALUES (0, 0)");
+        execute("INSERT INTO %s (KEY_23, COLUMN) VALUES (0, 0)");
+
+        assertInvalid("INSERT INTO %s (key_23, column, column) VALUES (0, 0, 0)");
+        assertInvalid("INSERT INTO %s (key_23, column, COLUMN) VALUES (0, 0, 0)");
+        assertInvalid("INSERT INTO %s (key_23, key_23, column) VALUES (0, 0, 0)");
+        assertInvalid("INSERT INTO %s (key_23, KEY_23, column) VALUES (0, 0, 0)");
+
+        String tableName = createTableName();
+        assertInvalidThrow(SyntaxException.class, String.format("CREATE TABLE %s.%s (select int PRIMARY KEY, column int)", keyspace(), tableName));
+    }
+
+    /**
+     * Test table options
+     * migrated from cql_tests.py:TestCQL.table_options_test()
+     */
+    @Test
+    public void testTableOptions() throws Throwable
+    {
+        createTable("CREATE TABLE %s ( k int PRIMARY KEY, c int ) WITH "
+                    + "comment = 'My comment' "
+                    + "AND read_repair_chance = 0.5 "
+                    + "AND dclocal_read_repair_chance = 0.5 "
+                    + "AND gc_grace_seconds = 4 "
+                    + "AND bloom_filter_fp_chance = 0.01 "
+                    + "AND compaction = { 'class' : 'LeveledCompactionStrategy', 'sstable_size_in_mb' : 10 } "
+                    + "AND compression = { 'sstable_compression' : '' } "
+                    + "AND caching = 'all' ");
+
+        execute("ALTER TABLE %s WITH "
+                + "comment = 'other comment' "
+                + "AND read_repair_chance = 0.3 "
+                + "AND dclocal_read_repair_chance = 0.3 "
+                + "AND gc_grace_seconds = 100 "
+                + "AND bloom_filter_fp_chance = 0.1 "
+                + "AND compaction = { 'class': 'SizeTieredCompactionStrategy', 'min_sstable_size' : 42 } "
+                + "AND compression = { 'sstable_compression' : 'SnappyCompressor' } "
+                + "AND caching = 'rows_only' ");
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.unescaped_string_test()
+     */
+    @Test
+    public void testUnescapedString() throws Throwable
+    {
+        createTable("CREATE TABLE %s ( k text PRIMARY KEY, c text, )");
+
+        //The \ in this query string is not forwarded to cassandra.
+        //The ' is being escaped in python, but only ' is forwarded
+        //over the wire instead of \'.
+        assertInvalidThrow(SyntaxException.class, "INSERT INTO %s (k, c) VALUES ('foo', 'CQL is cassandra\'s best friend')");
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.boolean_test()
+     */
+    @Test
+    public void testBoolean() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k boolean PRIMARY KEY, b boolean)");
+
+        execute("INSERT INTO %s (k, b) VALUES (true, false)");
+        assertRows(execute("SELECT * FROM %s WHERE k = true"),
+                   row(true, false));
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.float_with_exponent_test()
+     */
+    @Test
+    public void testFloatWithExponent() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, d double, f float)");
+
+        execute("INSERT INTO %s (k, d, f) VALUES (0, 3E+10, 3.4E3)");
+        execute("INSERT INTO %s (k, d, f) VALUES (1, 3.E10, -23.44E-3)");
+        execute("INSERT INTO %s (k, d, f) VALUES (2, 3, -2)");
+    }
+
+    /**
+     * Test regression from #5189,
+     * migrated from cql_tests.py:TestCQL.compact_metadata_test()
+     */
+    @Test
+    public void testCompactMetadata() throws Throwable
+    {
+        createTable("CREATE TABLE %s (id int primary key, i int ) WITH COMPACT STORAGE");
+
+        execute("INSERT INTO %s (id, i) VALUES (1, 2)");
+        assertRows(execute("SELECT * FROM %s"),
+                   row(1, 2));
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.conversion_functions_test()
+     */
+    @Test
+    public void testConversionFunctions() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, i varint, b blob)");
+
+        execute("INSERT INTO %s (k, i, b) VALUES (0, blobAsVarint(bigintAsBlob(3)), textAsBlob('foobar'))");
+        assertRows(execute("SELECT i, blobAsText(b) FROM %s WHERE k = 0"),
+                   row(BigInteger.valueOf(3), "foobar"));
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.empty_blob_test()
+     */
+    @Test
+    public void testEmptyBlob() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, b blob)");
+        execute("INSERT INTO %s (k, b) VALUES (0, 0x)");
+
+        assertRows(execute("SELECT * FROM %s"),
+                   row(0, ByteBufferUtil.bytes("")));
+    }
+
+    private Object[][] fill() throws Throwable
+    {
+        for (int i = 0; i < 2; i++)
+            for (int j = 0; j < 2; j++)
+                execute("INSERT INTO %s (k1, k2, v) VALUES (?, ?, ?)", i, j, i + j);
+
+        return getRows(execute("SELECT * FROM %s"));
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.empty_in_test()
+     */
+    @Test
+    public void testEmpty() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k1 int, k2 int, v int, PRIMARY KEY (k1, k2))");
+
+        // Inserts a few rows to make sure we don 't actually query something
+        Object[][] rows = fill();
+
+        // Test empty IN() in SELECT
+        assertEmpty(execute("SELECT v FROM %s WHERE k1 IN ()"));
+        assertEmpty(execute("SELECT v FROM %s WHERE k1 = 0 AND k2 IN ()"));
+
+        // Test empty IN() in DELETE
+        execute("DELETE FROM %s WHERE k1 IN ()");
+        assertArrayEquals(rows, getRows(execute("SELECT * FROM %s")));
+
+        // Test empty IN() in UPDATE
+        execute("UPDATE %s SET v = 3 WHERE k1 IN () AND k2 = 2");
+        assertArrayEquals(rows, getRows(execute("SELECT * FROM %s")));
+
+        // Same test, but for compact
+        createTable("CREATE TABLE %s (k1 int, k2 int, v int, PRIMARY KEY (k1, k2)) WITH COMPACT STORAGE");
+
+        rows = fill();
+
+        assertEmpty(execute("SELECT v FROM %s WHERE k1 IN ()"));
+        assertEmpty(execute("SELECT v FROM %s WHERE k1 = 0 AND k2 IN ()"));
+
+        // Test empty IN() in DELETE
+        execute("DELETE FROM %s WHERE k1 IN ()");
+        assertArrayEquals(rows, getRows(execute("SELECT * FROM %s")));
+
+        // Test empty IN() in UPDATE
+        execute("UPDATE %s SET v = 3 WHERE k1 IN () AND k2 = 2");
+        assertArrayEquals(rows, getRows(execute("SELECT * FROM %s")));
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.function_with_null_test()
+     */
+    @Test
+    public void testFunctionWithNull() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, t timeuuid,)");
+
+        execute("INSERT INTO %s (k) VALUES (0)");
+        Object[][] rows = getRows(execute("SELECT dateOf(t) FROM %s WHERE k=0"));
+        assertNull(rows[0][0]);
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.column_name_validation_test()
+     */
+    @Test
+    public void testColumnNameValidation() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k text, c int, v timeuuid, PRIMARY KEY (k, c))");
+
+        assertInvalid("INSERT INTO %s (k, c) VALUES ('', 0)");
+
+        // Insert a value that don't fit 'int'
+        assertInvalid("INSERT INTO %s (k, c) VALUES (0, 10000000000)");
+
+        // Insert a non-version 1 uuid
+        assertInvalid("INSERT INTO %s (k, c, v) VALUES (0, 0, 550e8400-e29b-41d4-a716-446655440000)");
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.nan_infinity_test()
+     */
+    @Test
+    public void testNanInfinityValues() throws Throwable
+    {
+        createTable("CREATE TABLE %s (f float PRIMARY KEY)");
+
+        execute("INSERT INTO %s (f) VALUES (NaN)");
+        execute("INSERT INTO %s (f) VALUES (-NaN)");
+        execute("INSERT INTO %s (f) VALUES (Infinity)");
+        execute("INSERT INTO %s (f) VALUES (-Infinity)");
+
+        Object[][] selected = getRows(execute("SELECT * FROM %s"));
+
+        // selected should be[[nan],[inf],[-inf]],
+        // but assert element - wise because NaN!=NaN
+        assertEquals(3, selected.length);
+        assertEquals(1, selected[0].length);
+        assertTrue(Float.isNaN((Float) selected[0][0]));
+
+        assertTrue(Float.isInfinite((Float) selected[1][0])); //inf
+        assertTrue(((Float) selected[1][0]) > 0);
+
+        assertTrue(Float.isInfinite((Float) selected[2][0])); //-inf
+        assertTrue(((Float) selected[2][0]) < 0);
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.blobAs_functions_test()
+     */
+    @Test
+    public void testBlobAsFunction() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, v int)");
+
+        //  A blob that is not 4 bytes should be rejected
+        assertInvalid("INSERT INTO %s (k, v) VALUES (0, blobAsInt(0x01))");
+
+        execute("INSERT INTO %s (k, v) VALUES (0, blobAsInt(0x00000001))");
+        assertRows(execute("select v from %s where k=0"), row(1));
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f797bfa4/test/unit/org/apache/cassandra/cql3/validation/miscellaneous/SSTableMetadataTrackingTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/miscellaneous/SSTableMetadataTrackingTest.java b/test/unit/org/apache/cassandra/cql3/validation/miscellaneous/SSTableMetadataTrackingTest.java
new file mode 100644
index 0000000..2a2ca7b
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/validation/miscellaneous/SSTableMetadataTrackingTest.java
@@ -0,0 +1,161 @@
+/*
+ * 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.validation.miscellaneous;
+
+import org.junit.Test;
+
+import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.io.sstable.metadata.StatsMetadata;
+import static org.junit.Assert.assertEquals;
+
+public class SSTableMetadataTrackingTest extends CQLTester
+{
+    @Test
+    public void baseCheck() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int, b int, c text, PRIMARY KEY (a, b))");
+        ColumnFamilyStore cfs = Keyspace.open(keyspace()).getColumnFamilyStore(currentTable());
+        execute("INSERT INTO %s (a,b,c) VALUES (1,1,'1') using timestamp 9999");
+        cfs.forceBlockingFlush();
+        StatsMetadata metadata = cfs.getSSTables().iterator().next().getSSTableMetadata();
+        assertEquals(9999, metadata.minTimestamp);
+        assertEquals(Integer.MAX_VALUE, metadata.maxLocalDeletionTime);
+        cfs.forceMajorCompaction();
+        metadata = cfs.getSSTables().iterator().next().getSSTableMetadata();
+        assertEquals(9999, metadata.minTimestamp);
+        assertEquals(Integer.MAX_VALUE, metadata.maxLocalDeletionTime);
+    }
+
+    @Test
+    public void testMinMaxtimestampRange() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int, b int, c text, PRIMARY KEY (a, b))");
+        ColumnFamilyStore cfs = Keyspace.open(keyspace()).getColumnFamilyStore(currentTable());
+        execute("INSERT INTO %s (a,b,c) VALUES (1,1,'1') using timestamp 10000");
+        execute("DELETE FROM %s USING TIMESTAMP 9999 WHERE a = 1 and b = 1");
+        cfs.forceBlockingFlush();
+        StatsMetadata metadata = cfs.getSSTables().iterator().next().getSSTableMetadata();
+        assertEquals(9999, metadata.minTimestamp);
+        assertEquals(10000, metadata.maxTimestamp);
+        assertEquals(Integer.MAX_VALUE, metadata.maxLocalDeletionTime, 5);
+        cfs.forceMajorCompaction();
+        metadata = cfs.getSSTables().iterator().next().getSSTableMetadata();
+        assertEquals(9999, metadata.minTimestamp);
+        assertEquals(10000, metadata.maxTimestamp);
+        assertEquals(Integer.MAX_VALUE, metadata.maxLocalDeletionTime, 5);
+    }
+
+    @Test
+    public void testMinMaxtimestampRow() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int, b int, c text, PRIMARY KEY (a, b))");
+        ColumnFamilyStore cfs = Keyspace.open(keyspace()).getColumnFamilyStore(currentTable());
+        execute("INSERT INTO %s (a,b,c) VALUES (1,1,'1') using timestamp 10000");
+        execute("DELETE FROM %s USING TIMESTAMP 9999 WHERE a = 1");
+        cfs.forceBlockingFlush();
+        StatsMetadata metadata = cfs.getSSTables().iterator().next().getSSTableMetadata();
+        assertEquals(9999, metadata.minTimestamp);
+        assertEquals(10000, metadata.maxTimestamp);
+        assertEquals(Integer.MAX_VALUE, metadata.maxLocalDeletionTime, 5);
+        cfs.forceMajorCompaction();
+        metadata = cfs.getSSTables().iterator().next().getSSTableMetadata();
+        assertEquals(9999, metadata.minTimestamp);
+        assertEquals(10000, metadata.maxTimestamp);
+        assertEquals(Integer.MAX_VALUE, metadata.maxLocalDeletionTime, 5);
+    }
+
+
+    @Test
+    public void testTrackMetadata_rangeTombstone() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int, b int, c text, PRIMARY KEY (a, b)) WITH gc_grace_seconds = 10000");
+        ColumnFamilyStore cfs = Keyspace.open(keyspace()).getColumnFamilyStore(currentTable());
+        execute("DELETE FROM %s USING TIMESTAMP 9999 WHERE a = 1 and b = 1");
+        cfs.forceBlockingFlush();
+        assertEquals(1, cfs.getSSTables().size());
+        StatsMetadata metadata = cfs.getSSTables().iterator().next().getSSTableMetadata();
+        assertEquals(9999, metadata.minTimestamp);
+        assertEquals(9999, metadata.maxTimestamp);
+        assertEquals(System.currentTimeMillis()/1000, metadata.maxLocalDeletionTime, 5);
+        cfs.forceMajorCompaction();
+        StatsMetadata metadata2 = cfs.getSSTables().iterator().next().getSSTableMetadata();
+        assertEquals(metadata.maxLocalDeletionTime, metadata2.maxLocalDeletionTime);
+        assertEquals(metadata.minTimestamp, metadata2.minTimestamp);
+        assertEquals(metadata.maxTimestamp, metadata2.maxTimestamp);
+    }
+
+    @Test
+    public void testTrackMetadata_rowTombstone() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int, b int, c text, PRIMARY KEY (a, b))");
+        ColumnFamilyStore cfs = Keyspace.open(keyspace()).getColumnFamilyStore(currentTable());
+        execute("DELETE FROM %s USING TIMESTAMP 9999 WHERE a = 1");
+
+        cfs.forceBlockingFlush();
+        assertEquals(1, cfs.getSSTables().size());
+        StatsMetadata metadata = cfs.getSSTables().iterator().next().getSSTableMetadata();
+        assertEquals(9999, metadata.minTimestamp);
+        assertEquals(9999, metadata.maxTimestamp);
+        assertEquals(System.currentTimeMillis()/1000, metadata.maxLocalDeletionTime, 5);
+        cfs.forceMajorCompaction();
+        StatsMetadata metadata2 = cfs.getSSTables().iterator().next().getSSTableMetadata();
+        assertEquals(metadata.maxLocalDeletionTime, metadata2.maxLocalDeletionTime);
+        assertEquals(metadata.minTimestamp, metadata2.minTimestamp);
+        assertEquals(metadata.maxTimestamp, metadata2.maxTimestamp);
+    }
+
+    @Test
+    public void testTrackMetadata_rowMarker() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int, PRIMARY KEY (a))");
+        ColumnFamilyStore cfs = Keyspace.open(keyspace()).getColumnFamilyStore(currentTable());
+        execute("INSERT INTO %s (a) VALUES (1) USING TIMESTAMP 9999");
+
+        cfs.forceBlockingFlush();
+        assertEquals(1, cfs.getSSTables().size());
+        StatsMetadata metadata = cfs.getSSTables().iterator().next().getSSTableMetadata();
+        assertEquals(9999, metadata.minTimestamp);
+        assertEquals(9999, metadata.maxTimestamp);
+        assertEquals(Integer.MAX_VALUE, metadata.maxLocalDeletionTime);
+        cfs.forceMajorCompaction();
+        StatsMetadata metadata2 = cfs.getSSTables().iterator().next().getSSTableMetadata();
+        assertEquals(metadata.maxLocalDeletionTime, metadata2.maxLocalDeletionTime);
+        assertEquals(metadata.minTimestamp, metadata2.minTimestamp);
+        assertEquals(metadata.maxTimestamp, metadata2.maxTimestamp);
+    }
+    @Test
+    public void testTrackMetadata_rowMarkerDelete() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int, PRIMARY KEY (a))");
+        ColumnFamilyStore cfs = Keyspace.open(keyspace()).getColumnFamilyStore(currentTable());
+        execute("DELETE FROM %s USING TIMESTAMP 9999 WHERE a=1");
+        cfs.forceBlockingFlush();
+        assertEquals(1, cfs.getSSTables().size());
+        StatsMetadata metadata = cfs.getSSTables().iterator().next().getSSTableMetadata();
+        assertEquals(9999, metadata.minTimestamp);
+        assertEquals(9999, metadata.maxTimestamp);
+        assertEquals(System.currentTimeMillis()/1000, metadata.maxLocalDeletionTime, 5);
+        cfs.forceMajorCompaction();
+        StatsMetadata metadata2 = cfs.getSSTables().iterator().next().getSSTableMetadata();
+        assertEquals(metadata.maxLocalDeletionTime, metadata2.maxLocalDeletionTime);
+        assertEquals(metadata.minTimestamp, metadata2.minTimestamp);
+        assertEquals(metadata.maxTimestamp, metadata2.maxTimestamp);
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f797bfa4/test/unit/org/apache/cassandra/cql3/validation/miscellaneous/TombstonesTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/miscellaneous/TombstonesTest.java b/test/unit/org/apache/cassandra/cql3/validation/miscellaneous/TombstonesTest.java
new file mode 100644
index 0000000..857eb40
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/validation/miscellaneous/TombstonesTest.java
@@ -0,0 +1,167 @@
+/*
+ * 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.validation.miscellaneous;
+
+import java.util.concurrent.TimeUnit;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.db.filter.TombstoneOverwhelmingException;
+
+import static junit.framework.Assert.assertTrue;
+import static junit.framework.Assert.fail;
+
+/**
+ * Test that TombstoneOverwhelmingException gets thrown when it should be and doesn't when it shouldn't be.
+ */
+public class TombstonesTest extends CQLTester
+{
+    static final int ORIGINAL_THRESHOLD = DatabaseDescriptor.getTombstoneFailureThreshold();
+    static final int THRESHOLD = 100;
+
+    @BeforeClass
+    public static void setUp() throws Throwable
+    {
+        DatabaseDescriptor.setTombstoneFailureThreshold(THRESHOLD);
+    }
+
+    @AfterClass
+    public static void tearDown()
+    {
+        DatabaseDescriptor.setTombstoneFailureThreshold(ORIGINAL_THRESHOLD);
+    }
+
+    @Test
+    public void testBelowThresholdSelect() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a text, b text, c text, PRIMARY KEY (a, b));");
+
+        // insert exactly the amount of tombstones that shouldn't trigger an exception
+        for (int i = 0; i < THRESHOLD; i++)
+            execute("INSERT INTO %s (a, b, c) VALUES ('key', 'column" + i + "', null);");
+
+        try
+        {
+            execute("SELECT * FROM %s WHERE a = 'key';");
+        }
+        catch (Throwable e)
+        {
+            fail("SELECT with tombstones below the threshold should not have failed, but has: " + e);
+        }
+    }
+
+    @Test
+    public void testBeyondThresholdSelect() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a text, b text, c text, PRIMARY KEY (a, b));");
+
+        // insert exactly the amount of tombstones that *SHOULD* trigger an exception
+        for (int i = 0; i < THRESHOLD + 1; i++)
+            execute("INSERT INTO %s (a, b, c) VALUES ('key', 'column" + i + "', null);");
+
+        try
+        {
+            execute("SELECT * FROM %s WHERE a = 'key';");
+            fail("SELECT with tombstones beyond the threshold should have failed, but hasn't");
+        }
+        catch (Throwable e)
+        {
+            assertTrue(e instanceof TombstoneOverwhelmingException);
+        }
+    }
+
+    @Test
+    public void testAllShadowedSelect() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a text, b text, c text, PRIMARY KEY (a, b));");
+
+        // insert exactly the amount of tombstones that *SHOULD* normally trigger an exception
+        for (int i = 0; i < THRESHOLD + 1; i++)
+            execute("INSERT INTO %s (a, b, c) VALUES ('key', 'column" + i + "', null);");
+
+        // delete all with a partition level tombstone
+        execute("DELETE FROM %s WHERE a = 'key'");
+
+        try
+        {
+            execute("SELECT * FROM %s WHERE a = 'key';");
+        }
+        catch (Throwable e)
+        {
+            fail("SELECT with tombstones shadowed by a partition tombstone should not have failed, but has: " + e);
+        }
+    }
+
+    @Test
+    public void testLiveShadowedCellsSelect() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a text, b text, c text, PRIMARY KEY (a, b));");
+
+        for (int i = 0; i < THRESHOLD + 1; i++)
+            execute("INSERT INTO %s (a, b, c) VALUES ('key', 'column" + i + "', 'column');");
+
+        // delete all with a partition level tombstone
+        execute("DELETE FROM %s WHERE a = 'key'");
+
+        try
+        {
+            execute("SELECT * FROM %s WHERE a = 'key';");
+        }
+        catch (Throwable e)
+        {
+            fail("SELECT with regular cells shadowed by a partition tombstone should not have failed, but has: " + e);
+        }
+    }
+
+    @Test
+    public void testExpiredTombstones() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a text, b text, c text, PRIMARY KEY (a, b)) WITH gc_grace_seconds = 1;");
+
+        for (int i = 0; i < THRESHOLD + 1; i++)
+            execute("INSERT INTO %s (a, b, c) VALUES ('key', 'column" + i + "', null);");
+
+        // not yet past gc grace - must throw a TOE
+        try
+        {
+            execute("SELECT * FROM %s WHERE a = 'key';");
+            fail("SELECT with tombstones beyond the threshold should have failed, but hasn't");
+        }
+        catch (Throwable e)
+        {
+            assertTrue(e instanceof TombstoneOverwhelmingException);
+        }
+
+        // sleep past gc grace
+        TimeUnit.SECONDS.sleep(2);
+
+        // past gc grace - must not throw a TOE now
+        try
+        {
+            execute("SELECT * FROM %s WHERE a = 'key';");
+        }
+        catch (Throwable e)
+        {
+            fail("SELECT with expired tombstones beyond the threshold should not have failed, but has: " + e);
+        }
+    }
+}


[05/32] cassandra git commit: Migrate CQL tests from dtest to unit tests

Posted by jm...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/f797bfa4/test/unit/org/apache/cassandra/cql3/validation/entities/FrozenCollectionsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/FrozenCollectionsTest.java b/test/unit/org/apache/cassandra/cql3/validation/entities/FrozenCollectionsTest.java
new file mode 100644
index 0000000..beed560
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/FrozenCollectionsTest.java
@@ -0,0 +1,1111 @@
+/*
+ * 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.validation.entities;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.db.marshal.*;
+import org.apache.cassandra.dht.ByteOrderedPartitioner;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.exceptions.SyntaxException;
+import org.apache.commons.lang3.StringUtils;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+
+public class FrozenCollectionsTest extends CQLTester
+{
+    @BeforeClass
+    public static void setUpClass()
+    {
+        DatabaseDescriptor.setPartitioner(new ByteOrderedPartitioner());
+    }
+
+    @Test
+    public void testPartitionKeyUsage() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k frozen<set<int>> PRIMARY KEY, v int)");
+
+        execute("INSERT INTO %s (k, v) VALUES (?, ?)", set(), 1);
+        execute("INSERT INTO %s (k, v) VALUES (?, ?)", set(1, 2, 3), 1);
+        execute("INSERT INTO %s (k, v) VALUES (?, ?)", set(4, 5, 6), 0);
+        execute("INSERT INTO %s (k, v) VALUES (?, ?)", set(7, 8, 9), 0);
+
+        // overwrite with an update
+        execute("UPDATE %s SET v=? WHERE k=?", 0, set());
+        execute("UPDATE %s SET v=? WHERE k=?", 0, set(1, 2, 3));
+
+        assertRows(execute("SELECT * FROM %s"),
+            row(set(), 0),
+            row(set(1, 2, 3), 0),
+            row(set(4, 5, 6), 0),
+            row(set(7, 8, 9), 0)
+        );
+
+        assertRows(execute("SELECT k FROM %s"),
+            row(set()),
+            row(set(1, 2, 3)),
+            row(set(4, 5, 6)),
+            row(set(7, 8, 9))
+        );
+
+        assertRows(execute("SELECT * FROM %s LIMIT 2"),
+                row(set(), 0),
+                row(set(1, 2, 3), 0)
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE k=?", set(4, 5, 6)),
+            row(set(4, 5, 6), 0)
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE k=?", set()),
+                row(set(), 0)
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE k IN ?", list(set(4, 5, 6), set())),
+                row(set(4, 5, 6), 0),
+                row(set(), 0)
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE token(k) >= token(?)", set(4, 5, 6)),
+                row(set(4, 5, 6), 0),
+                row(set(7, 8, 9), 0)
+        );
+
+        assertInvalid("INSERT INTO %s (k, v) VALUES (null, 0)");
+
+        execute("DELETE FROM %s WHERE k=?", set());
+        execute("DELETE FROM %s WHERE k=?", set(4, 5, 6));
+        assertRows(execute("SELECT * FROM %s"),
+            row(set(1, 2, 3), 0),
+            row(set(7, 8, 9), 0)
+        );
+    }
+
+    @Test
+    public void testNestedPartitionKeyUsage() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k frozen<map<set<int>, list<int>>> PRIMARY KEY, v int)");
+
+        execute("INSERT INTO %s (k, v) VALUES (?, ?)", map(), 1);
+        execute("INSERT INTO %s (k, v) VALUES (?, ?)", map(set(), list(1, 2, 3)), 0);
+        execute("INSERT INTO %s (k, v) VALUES (?, ?)", map(set(1, 2, 3), list(1, 2, 3)), 1);
+        execute("INSERT INTO %s (k, v) VALUES (?, ?)", map(set(4, 5, 6), list(1, 2, 3)), 0);
+        execute("INSERT INTO %s (k, v) VALUES (?, ?)", map(set(7, 8, 9), list(1, 2, 3)), 0);
+
+        // overwrite with an update
+        execute("UPDATE %s SET v=? WHERE k=?", 0, map());
+        execute("UPDATE %s SET v=? WHERE k=?", 0, map(set(1, 2, 3), list(1, 2, 3)));
+
+        assertRows(execute("SELECT * FROM %s"),
+            row(map(), 0),
+            row(map(set(), list(1, 2, 3)), 0),
+            row(map(set(1, 2, 3), list(1, 2, 3)), 0),
+            row(map(set(4, 5, 6), list(1, 2, 3)), 0),
+            row(map(set(7, 8, 9), list(1, 2, 3)), 0)
+        );
+
+        assertRows(execute("SELECT k FROM %s"),
+            row(map()),
+            row(map(set(), list(1, 2, 3))),
+            row(map(set(1, 2, 3), list(1, 2, 3))),
+            row(map(set(4, 5, 6), list(1, 2, 3))),
+            row(map(set(7, 8, 9), list(1, 2, 3)))
+        );
+
+        assertRows(execute("SELECT * FROM %s LIMIT 3"),
+            row(map(), 0),
+            row(map(set(), list(1, 2, 3)), 0),
+            row(map(set(1, 2, 3), list(1, 2, 3)), 0)
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE k=?", map(set(4, 5, 6), list(1, 2, 3))),
+            row(map(set(4, 5, 6), list(1, 2, 3)), 0)
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE k=?", map()),
+                row(map(), 0)
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE k=?", map(set(), list(1, 2, 3))),
+                row(map(set(), list(1, 2, 3)), 0)
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE k IN ?", list(map(set(4, 5, 6), list(1, 2, 3)), map(), map(set(), list(1, 2, 3)))),
+            row(map(set(4, 5, 6), list(1, 2, 3)), 0),
+            row(map(), 0),
+            row(map(set(), list(1, 2, 3)), 0)
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE token(k) >= token(?)", map(set(4, 5, 6), list(1, 2, 3))),
+            row(map(set(4, 5, 6), list(1, 2, 3)), 0),
+            row(map(set(7, 8, 9), list(1, 2, 3)), 0)
+        );
+
+        execute("DELETE FROM %s WHERE k=?", map());
+        execute("DELETE FROM %s WHERE k=?", map(set(), list(1, 2, 3)));
+        execute("DELETE FROM %s WHERE k=?", map(set(4, 5, 6), list(1, 2, 3)));
+        assertRows(execute("SELECT * FROM %s"),
+            row(map(set(1, 2, 3), list(1, 2, 3)), 0),
+            row(map(set(7, 8, 9), list(1, 2, 3)), 0)
+        );
+
+    }
+
+    @Test
+    public void testClusteringKeyUsage() throws Throwable
+    {
+        for (String option : Arrays.asList("", " WITH COMPACT STORAGE"))
+        {
+            createTable("CREATE TABLE %s (a int, b frozen<set<int>>, c int, PRIMARY KEY (a, b))" + option);
+
+            execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, set(), 1);
+            execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, set(1, 2, 3), 1);
+            execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, set(4, 5, 6), 0);
+            execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, set(7, 8, 9), 0);
+
+            // overwrite with an update
+            execute("UPDATE %s SET c=? WHERE a=? AND b=?", 0, 0, set());
+            execute("UPDATE %s SET c=? WHERE a=? AND b=?", 0, 0, set(1, 2, 3));
+
+            assertRows(execute("SELECT * FROM %s"),
+                row(0, set(), 0),
+                row(0, set(1, 2, 3), 0),
+                row(0, set(4, 5, 6), 0),
+                row(0, set(7, 8, 9), 0)
+            );
+
+            assertRows(execute("SELECT b FROM %s"),
+                row(set()),
+                row(set(1, 2, 3)),
+                row(set(4, 5, 6)),
+                row(set(7, 8, 9))
+            );
+
+            assertRows(execute("SELECT * FROM %s LIMIT 2"),
+                row(0, set(), 0),
+                row(0, set(1, 2, 3), 0)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a=? AND b=?", 0, set(4, 5, 6)),
+                row(0, set(4, 5, 6), 0)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a=? AND b=?", 0, set()),
+                row(0, set(), 0)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a=? AND b IN ?", 0, list(set(4, 5, 6), set())),
+                row(0, set(), 0),
+                row(0, set(4, 5, 6), 0)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a=? AND b > ?", 0, set(4, 5, 6)),
+                row(0, set(7, 8, 9), 0)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a=? AND b >= ?", 0, set(4, 5, 6)),
+                row(0, set(4, 5, 6), 0),
+                row(0, set(7, 8, 9), 0)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a=? AND b < ?", 0, set(4, 5, 6)),
+                row(0, set(), 0),
+                row(0, set(1, 2, 3), 0)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a=? AND b <= ?", 0, set(4, 5, 6)),
+                row(0, set(), 0),
+                row(0, set(1, 2, 3), 0),
+                row(0, set(4, 5, 6), 0)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a=? AND b > ? AND b <= ?", 0, set(1, 2, 3), set(4, 5, 6)),
+                row(0, set(4, 5, 6), 0)
+            );
+
+            execute("DELETE FROM %s WHERE a=? AND b=?", 0, set());
+            execute("DELETE FROM %s WHERE a=? AND b=?", 0, set(4, 5, 6));
+            assertRows(execute("SELECT * FROM %s"),
+                row(0, set(1, 2, 3), 0),
+                row(0, set(7, 8, 9), 0)
+            );
+        }
+    }
+
+    @Test
+    public void testNestedClusteringKeyUsage() throws Throwable
+    {
+        for (String option : Arrays.asList("", " WITH COMPACT STORAGE"))
+        {
+            createTable("CREATE TABLE %s (a int, b frozen<map<set<int>, list<int>>>, c frozen<set<int>>, d int, PRIMARY KEY (a, b, c))" + option);
+
+            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, map(), set(), 0);
+            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, map(set(), list(1, 2, 3)), set(), 0);
+            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, map(set(1, 2, 3), list(1, 2, 3)), set(1, 2, 3), 0);
+            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, map(set(4, 5, 6), list(1, 2, 3)), set(1, 2, 3), 0);
+            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, map(set(7, 8, 9), list(1, 2, 3)), set(1, 2, 3), 0);
+
+            assertRows(execute("SELECT * FROM %s"),
+                row(0, map(), set(), 0),
+                row(0, map(set(), list(1, 2, 3)), set(), 0),
+                row(0, map(set(1, 2, 3), list(1, 2, 3)), set(1, 2, 3), 0),
+                row(0, map(set(4, 5, 6), list(1, 2, 3)), set(1, 2, 3), 0),
+                row(0, map(set(7, 8, 9), list(1, 2, 3)), set(1, 2, 3), 0)
+            );
+
+            assertRows(execute("SELECT b FROM %s"),
+                row(map()),
+                row(map(set(), list(1, 2, 3))),
+                row(map(set(1, 2, 3), list(1, 2, 3))),
+                row(map(set(4, 5, 6), list(1, 2, 3))),
+                row(map(set(7, 8, 9), list(1, 2, 3)))
+            );
+
+            assertRows(execute("SELECT c FROM %s"),
+                row(set()),
+                row(set()),
+                row(set(1, 2, 3)),
+                row(set(1, 2, 3)),
+                row(set(1, 2, 3))
+            );
+
+            assertRows(execute("SELECT * FROM %s LIMIT 3"),
+                row(0, map(), set(), 0),
+                row(0, map(set(), list(1, 2, 3)), set(), 0),
+                row(0, map(set(1, 2, 3), list(1, 2, 3)), set(1, 2, 3), 0)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a=0 ORDER BY b DESC LIMIT 4"),
+                row(0, map(set(7, 8, 9), list(1, 2, 3)), set(1, 2, 3), 0),
+                row(0, map(set(4, 5, 6), list(1, 2, 3)), set(1, 2, 3), 0),
+                row(0, map(set(1, 2, 3), list(1, 2, 3)), set(1, 2, 3), 0),
+                row(0, map(set(), list(1, 2, 3)), set(), 0)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a=? AND b=?", 0, map()),
+                row(0, map(), set(), 0)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a=? AND b=?", 0, map(set(), list(1, 2, 3))),
+                row(0, map(set(), list(1, 2, 3)), set(), 0)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a=? AND b=?", 0, map(set(1, 2, 3), list(1, 2, 3))),
+                row(0, map(set(1, 2, 3), list(1, 2, 3)), set(1, 2, 3), 0)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a=? AND b=? AND c=?", 0, map(set(), list(1, 2, 3)), set()),
+                    row(0, map(set(), list(1, 2, 3)), set(), 0)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a=? AND (b, c) IN ?", 0, list(tuple(map(set(4, 5, 6), list(1, 2, 3)), set(1, 2, 3)),
+                                                                                     tuple(map(), set()))),
+                row(0, map(), set(), 0),
+                row(0, map(set(4, 5, 6), list(1, 2, 3)), set(1, 2, 3), 0)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a=? AND b > ?", 0, map(set(4, 5, 6), list(1, 2, 3))),
+                row(0, map(set(7, 8, 9), list(1, 2, 3)), set(1, 2, 3), 0)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a=? AND b >= ?", 0, map(set(4, 5, 6), list(1, 2, 3))),
+                row(0, map(set(4, 5, 6), list(1, 2, 3)), set(1, 2, 3), 0),
+                row(0, map(set(7, 8, 9), list(1, 2, 3)), set(1, 2, 3), 0)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a=? AND b < ?", 0, map(set(4, 5, 6), list(1, 2, 3))),
+                row(0, map(), set(), 0),
+                row(0, map(set(), list(1, 2, 3)), set(), 0),
+                row(0, map(set(1, 2, 3), list(1, 2, 3)), set(1, 2, 3), 0)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a=? AND b <= ?", 0, map(set(4, 5, 6), list(1, 2, 3))),
+                row(0, map(), set(), 0),
+                row(0, map(set(), list(1, 2, 3)), set(), 0),
+                row(0, map(set(1, 2, 3), list(1, 2, 3)), set(1, 2, 3), 0),
+                row(0, map(set(4, 5, 6), list(1, 2, 3)), set(1, 2, 3), 0)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a=? AND b > ? AND b <= ?", 0, map(set(1, 2, 3), list(1, 2, 3)), map(set(4, 5, 6), list(1, 2, 3))),
+                row(0, map(set(4, 5, 6), list(1, 2, 3)), set(1, 2, 3), 0)
+            );
+
+            execute("DELETE FROM %s WHERE a=? AND b=? AND c=?", 0, map(), set());
+            assertEmpty(execute("SELECT * FROM %s WHERE a=? AND b=? AND c=?", 0, map(), set()));
+
+            execute("DELETE FROM %s WHERE a=? AND b=? AND c=?", 0, map(set(), list(1, 2, 3)), set());
+            assertEmpty(execute("SELECT * FROM %s WHERE a=? AND b=? AND c=?", 0, map(set(), list(1, 2, 3)), set()));
+
+            execute("DELETE FROM %s WHERE a=? AND b=? AND c=?", 0, map(set(4, 5, 6), list(1, 2, 3)), set(1, 2, 3));
+            assertEmpty(execute("SELECT * FROM %s WHERE a=? AND b=? AND c=?", 0, map(set(4, 5, 6), list(1, 2, 3)), set(1, 2, 3)));
+
+            assertRows(execute("SELECT * FROM %s"),
+                    row(0, map(set(1, 2, 3), list(1, 2, 3)), set(1, 2, 3), 0),
+                    row(0, map(set(7, 8, 9), list(1, 2, 3)), set(1, 2, 3), 0)
+            );
+        }
+    }
+
+    @Test
+    public void testNormalColumnUsage() throws Throwable
+    {
+        for (String option : Arrays.asList("", " WITH COMPACT STORAGE"))
+        {
+            createTable("CREATE TABLE %s (a int PRIMARY KEY, b frozen<map<set<int>, list<int>>>, c frozen<set<int>>)" + option);
+
+            execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, map(), set());
+            execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 1, map(set(), list(99999, 999999, 99999)), set());
+            execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 2, map(set(1, 2, 3), list(1, 2, 3)), set(1, 2, 3));
+            execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 3, map(set(4, 5, 6), list(1, 2, 3)), set(1, 2, 3));
+            execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 4, map(set(7, 8, 9), list(1, 2, 3)), set(1, 2, 3));
+
+            // overwrite with update
+            execute ("UPDATE %s SET b=? WHERE a=?", map(set(), list(1, 2, 3)), 1);
+
+            assertRows(execute("SELECT * FROM %s"),
+                row(0, map(), set()),
+                row(1, map(set(), list(1, 2, 3)), set()),
+                row(2, map(set(1, 2, 3), list(1, 2, 3)), set(1, 2, 3)),
+                row(3, map(set(4, 5, 6), list(1, 2, 3)), set(1, 2, 3)),
+                row(4, map(set(7, 8, 9), list(1, 2, 3)), set(1, 2, 3))
+            );
+
+            assertRows(execute("SELECT b FROM %s"),
+                row(map()),
+                row(map(set(), list(1, 2, 3))),
+                row(map(set(1, 2, 3), list(1, 2, 3))),
+                row(map(set(4, 5, 6), list(1, 2, 3))),
+                row(map(set(7, 8, 9), list(1, 2, 3)))
+            );
+
+            assertRows(execute("SELECT c FROM %s"),
+                row(set()),
+                row(set()),
+                row(set(1, 2, 3)),
+                row(set(1, 2, 3)),
+                row(set(1, 2, 3))
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a=?", 3),
+                row(3, map(set(4, 5, 6), list(1, 2, 3)), set(1, 2, 3))
+            );
+
+            execute("UPDATE %s SET b=? WHERE a=?", null, 1);
+            assertRows(execute("SELECT * FROM %s WHERE a=?", 1),
+                row(1, null, set())
+            );
+
+            execute("UPDATE %s SET b=? WHERE a=?", map(), 1);
+            assertRows(execute("SELECT * FROM %s WHERE a=?", 1),
+                row(1, map(), set())
+            );
+
+            execute("UPDATE %s SET c=? WHERE a=?", null, 2);
+            assertRows(execute("SELECT * FROM %s WHERE a=?", 2),
+                row(2, map(set(1, 2, 3), list(1, 2, 3)), null)
+            );
+
+            execute("UPDATE %s SET c=? WHERE a=?", set(), 2);
+            assertRows(execute("SELECT * FROM %s WHERE a=?", 2),
+                    row(2, map(set(1, 2, 3), list(1, 2, 3)), set())
+            );
+
+            execute("DELETE b FROM %s WHERE a=?", 3);
+            assertRows(execute("SELECT * FROM %s WHERE a=?", 3),
+                row(3, null, set(1, 2, 3))
+            );
+
+            execute("DELETE c FROM %s WHERE a=?", 4);
+            assertRows(execute("SELECT * FROM %s WHERE a=?", 4),
+                row(4, map(set(7, 8, 9), list(1, 2, 3)), null)
+            );
+        }
+    }
+
+    @Test
+    public void testStaticColumnUsage() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int, b int, c frozen<map<set<int>, list<int>>> static, d int, PRIMARY KEY (a, b))");
+        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, map(), 0);
+        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, map(), 0);
+        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 1, 0, map(set(), list(1, 2, 3)), 0);
+        execute("INSERT INTO %s (a, b, d) VALUES (?, ?, ?)", 1, 1, 0);
+        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 2, 0, map(set(1, 2, 3), list(1, 2, 3)), 0);
+
+        assertRows(execute("SELECT * FROM %s"),
+            row(0, 0, map(), 0),
+            row(0, 1, map(), 0),
+            row(1, 0, map(set(), list(1, 2, 3)), 0),
+            row(1, 1, map(set(), list(1, 2, 3)), 0),
+            row(2, 0, map(set(1, 2, 3), list(1, 2, 3)), 0)
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE a=? AND b=?", 0, 1),
+            row(0, 1, map(), 0)
+        );
+
+        execute("DELETE c FROM %s WHERE a=?", 0);
+        assertRows(execute("SELECT * FROM %s"),
+                row(0, 0, null, 0),
+                row(0, 1, null, 0),
+                row(1, 0, map(set(), list(1, 2, 3)), 0),
+                row(1, 1, map(set(), list(1, 2, 3)), 0),
+                row(2, 0, map(set(1, 2, 3), list(1, 2, 3)), 0)
+        );
+
+        execute("DELETE FROM %s WHERE a=?", 0);
+        assertRows(execute("SELECT * FROM %s"),
+                row(1, 0, map(set(), list(1, 2, 3)), 0),
+                row(1, 1, map(set(), list(1, 2, 3)), 0),
+                row(2, 0, map(set(1, 2, 3), list(1, 2, 3)), 0)
+        );
+
+        execute("UPDATE %s SET c=? WHERE a=?", map(set(1, 2, 3), list(1, 2, 3)), 1);
+        assertRows(execute("SELECT * FROM %s"),
+                row(1, 0, map(set(1, 2, 3), list(1, 2, 3)), 0),
+                row(1, 1, map(set(1, 2, 3), list(1, 2, 3)), 0),
+                row(2, 0, map(set(1, 2, 3), list(1, 2, 3)), 0)
+        );
+    }
+
+    private void assertInvalidCreateWithMessage(String createTableStatement, String errorMessage) throws Throwable
+    {
+         try
+        {
+            createTableMayThrow(createTableStatement);
+            Assert.fail("Expected CREATE TABLE statement to error: " + createTableStatement);
+        }
+        catch (InvalidRequestException | ConfigurationException | SyntaxException ex)
+        {
+            Assert.assertTrue("Expected error message to contain '" + errorMessage + "', but got '" + ex.getMessage() + "'",
+                    ex.getMessage().contains(errorMessage));
+        }
+    }
+
+    private void assertInvalidAlterWithMessage(String createTableStatement, String errorMessage) throws Throwable
+    {
+        try
+        {
+            alterTableMayThrow(createTableStatement);
+            Assert.fail("Expected CREATE TABLE statement to error: " + createTableStatement);
+        }
+        catch (InvalidRequestException | ConfigurationException ex)
+        {
+            Assert.assertTrue("Expected error message to contain '" + errorMessage + "', but got '" + ex.getMessage() + "'",
+                    ex.getMessage().contains(errorMessage));
+        }
+    }
+
+    @Test
+    public void testInvalidOperations() throws Throwable
+    {
+        // lists
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, l frozen<list<int>>)");
+        assertInvalid("UPDATE %s SET l[?]=? WHERE k=?", 0, 0, 0);
+        assertInvalid("UPDATE %s SET l = ? + l WHERE k=?", list(0), 0);
+        assertInvalid("UPDATE %s SET l = l + ? WHERE k=?", list(4), 0);
+        assertInvalid("UPDATE %s SET l = l - ? WHERE k=?", list(3), 0);
+        assertInvalid("DELETE l[?] FROM %s WHERE k=?", 0, 0);
+
+        // sets
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, s frozen<set<int>>)");
+        assertInvalid("UPDATE %s SET s = s + ? WHERE k=?", set(0), 0);
+        assertInvalid("UPDATE %s SET s = s - ? WHERE k=?", set(3), 0);
+
+        // maps
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, m frozen<map<int, int>>)");
+        assertInvalid("UPDATE %s SET m[?]=? WHERE k=?", 0, 0, 0);
+        assertInvalid("UPDATE %s SET m = m + ? WHERE k=?", map(4, 4), 0);
+        assertInvalid("DELETE m[?] FROM %s WHERE k=?", 0, 0);
+
+        assertInvalidCreateWithMessage("CREATE TABLE %s (k int PRIMARY KEY, t set<set<int>>)",
+                "Non-frozen collections are not allowed inside collections");
+
+        assertInvalidCreateWithMessage("CREATE TABLE %s (k int PRIMARY KEY, t frozen<set<counter>>)",
+                                       "Counters are not allowed inside collections");
+
+        assertInvalidCreateWithMessage("CREATE TABLE %s (k int PRIMARY KEY, t frozen<text>)",
+                "frozen<> is only allowed on collections, tuples, and user-defined types");
+    }
+
+    @Test
+    public void testAltering() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int, b frozen<list<int>>, c frozen<list<int>>, PRIMARY KEY (a, b))");
+
+        alterTable("ALTER TABLE %s ALTER c TYPE frozen<list<blob>>");
+
+        assertInvalidAlterWithMessage("ALTER TABLE %s ALTER b TYPE frozen<list<blob>>",
+                                      "types are not order-compatible");
+
+        assertInvalidAlterWithMessage("ALTER TABLE %s ALTER b TYPE list<int>",
+                                      "types are not order-compatible");
+
+        assertInvalidAlterWithMessage("ALTER TABLE %s ALTER c TYPE list<blob>",
+                                      "types are incompatible");
+
+        alterTable("ALTER TABLE %s DROP c");
+        alterTable("ALTER TABLE %s ADD c frozen<set<int>>");
+        assertInvalidAlterWithMessage("ALTER TABLE %s ALTER c TYPE frozen<set<blob>>",
+                                      "types are incompatible");
+
+        alterTable("ALTER TABLE %s DROP c");
+        alterTable("ALTER TABLE %s ADD c frozen<map<int, int>>");
+        assertInvalidAlterWithMessage("ALTER TABLE %s ALTER c TYPE frozen<map<blob, int>>",
+                                      "types are incompatible");
+        alterTable("ALTER TABLE %s ALTER c TYPE frozen<map<int, blob>>");
+    }
+
+    private void assertInvalidIndexCreationWithMessage(String statement, String errorMessage) throws Throwable
+    {
+        try
+        {
+            createIndexMayThrow(statement);
+            Assert.fail("Expected index creation to fail: " + statement);
+        }
+        catch (InvalidRequestException ex)
+        {
+            Assert.assertTrue("Expected error message to contain '" + errorMessage + "', but got '" + ex.getMessage() + "'",
+                              ex.getMessage().contains(errorMessage));
+        }
+    }
+
+    @Test
+    public void testSecondaryIndex() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a frozen<map<int, text>> PRIMARY KEY, b frozen<map<int, text>>)");
+
+        // for now, we don't support indexing values or keys of collections in the primary key
+        assertInvalidIndexCreationWithMessage("CREATE INDEX ON %s (full(a))", "Cannot create secondary index on partition key column");
+        assertInvalidIndexCreationWithMessage("CREATE INDEX ON %s (keys(a))", "Cannot create index on keys of frozen<map> column");
+        assertInvalidIndexCreationWithMessage("CREATE INDEX ON %s (keys(b))", "Cannot create index on keys of frozen<map> column");
+
+        createTable("CREATE TABLE %s (a int, b frozen<list<int>>, c frozen<set<int>>, d frozen<map<int, text>>, PRIMARY KEY (a, b))");
+
+        createIndex("CREATE INDEX ON %s (full(b))");
+        createIndex("CREATE INDEX ON %s (full(c))");
+        createIndex("CREATE INDEX ON %s (full(d))");
+
+        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, list(1, 2, 3), set(1, 2, 3), map(1, "a"));
+        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, list(4, 5, 6), set(1, 2, 3), map(1, "a"));
+        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 1, list(1, 2, 3), set(4, 5, 6), map(2, "b"));
+        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 1, list(4, 5, 6), set(4, 5, 6), map(2, "b"));
+
+        // CONTAINS KEY doesn't work on non-maps
+        assertInvalidMessage("Cannot use CONTAINS KEY on non-map column",
+                             "SELECT * FROM %s WHERE b CONTAINS KEY ?", 1);
+
+        assertInvalidMessage("Cannot use CONTAINS KEY on non-map column",
+                             "SELECT * FROM %s WHERE b CONTAINS KEY ? ALLOW FILTERING", 1);
+
+        assertInvalidMessage("Cannot use CONTAINS KEY on non-map column",
+                             "SELECT * FROM %s WHERE c CONTAINS KEY ?", 1);
+
+        // normal indexes on frozen collections don't support CONTAINS or CONTAINS KEY
+        assertInvalidMessage("Cannot restrict column \"b\" by a CONTAINS relation without a secondary index",
+                             "SELECT * FROM %s WHERE b CONTAINS ?", 1);
+
+        assertInvalidMessage("Cannot restrict column \"b\" by a CONTAINS relation without a secondary index",
+                             "SELECT * FROM %s WHERE b CONTAINS ? ALLOW FILTERING", 1);
+
+        assertInvalidMessage("No secondary indexes on the restricted columns support the provided operator",
+                             "SELECT * FROM %s WHERE d CONTAINS KEY ?", 1);
+
+        assertInvalidMessage("No secondary indexes on the restricted columns support the provided operator",
+                             "SELECT * FROM %s WHERE d CONTAINS KEY ? ALLOW FILTERING", 1);
+
+        assertInvalidMessage("Cannot restrict column \"b\" by a CONTAINS relation without a secondary index",
+                             "SELECT * FROM %s WHERE b CONTAINS ? AND d CONTAINS KEY ? ALLOW FILTERING", 1, 1);
+
+        // index lookup on b
+        assertRows(execute("SELECT * FROM %s WHERE b=?", list(1, 2, 3)),
+            row(0, list(1, 2, 3), set(1, 2, 3), map(1, "a")),
+            row(1, list(1, 2, 3), set(4, 5, 6), map(2, "b"))
+        );
+
+        assertEmpty(execute("SELECT * FROM %s WHERE b=?", list(-1)));
+
+        assertInvalidMessage("ALLOW FILTERING", "SELECT * FROM %s WHERE b=? AND c=?", list(1, 2, 3), set(4, 5, 6));
+        assertRows(execute("SELECT * FROM %s WHERE b=? AND c=? ALLOW FILTERING", list(1, 2, 3), set(4, 5, 6)),
+            row(1, list(1, 2, 3), set(4, 5, 6), map(2, "b"))
+        );
+
+        assertInvalidMessage("ALLOW FILTERING", "SELECT * FROM %s WHERE b=? AND c CONTAINS ?", list(1, 2, 3), 5);
+        assertRows(execute("SELECT * FROM %s WHERE b=? AND c CONTAINS ? ALLOW FILTERING", list(1, 2, 3), 5),
+            row(1, list(1, 2, 3), set(4, 5, 6), map(2, "b"))
+        );
+
+        assertInvalidMessage("ALLOW FILTERING", "SELECT * FROM %s WHERE b=? AND d=?", list(1, 2, 3), map(1, "a"));
+        assertRows(execute("SELECT * FROM %s WHERE b=? AND d=? ALLOW FILTERING", list(1, 2, 3), map(1, "a")),
+            row(0, list(1, 2, 3), set(1, 2, 3), map(1, "a"))
+        );
+
+        assertInvalidMessage("ALLOW FILTERING", "SELECT * FROM %s WHERE b=? AND d CONTAINS ?", list(1, 2, 3), "a");
+        assertRows(execute("SELECT * FROM %s WHERE b=? AND d CONTAINS ? ALLOW FILTERING", list(1, 2, 3), "a"),
+            row(0, list(1, 2, 3), set(1, 2, 3), map(1, "a"))
+        );
+
+        assertInvalidMessage("ALLOW FILTERING", "SELECT * FROM %s WHERE b=? AND d CONTAINS KEY ?", list(1, 2, 3), 1);
+        assertRows(execute("SELECT * FROM %s WHERE b=? AND d CONTAINS KEY ? ALLOW FILTERING", list(1, 2, 3), 1),
+            row(0, list(1, 2, 3), set(1, 2, 3), map(1, "a"))
+        );
+
+        // index lookup on c
+        assertRows(execute("SELECT * FROM %s WHERE c=?", set(1, 2, 3)),
+            row(0, list(1, 2, 3), set(1, 2, 3), map(1, "a")),
+            row(0, list(4, 5, 6), set(1, 2, 3), map(1, "a"))
+        );
+
+        // ordering of c should not matter
+        assertRows(execute("SELECT * FROM %s WHERE c=?", set(2, 1, 3)),
+                row(0, list(1, 2, 3), set(1, 2, 3), map(1, "a")),
+                row(0, list(4, 5, 6), set(1, 2, 3), map(1, "a"))
+        );
+
+        assertEmpty(execute("SELECT * FROM %s WHERE c=?", set(-1)));
+
+        assertInvalidMessage("ALLOW FILTERING", "SELECT * FROM %s WHERE c=? AND b=?", set(1, 2, 3), list(1, 2, 3));
+        assertRows(execute("SELECT * FROM %s WHERE c=? AND b=? ALLOW FILTERING", set(1, 2, 3), list(1, 2, 3)),
+            row(0, list(1, 2, 3), set(1, 2, 3), map(1, "a"))
+        );
+
+        assertInvalidMessage("ALLOW FILTERING", "SELECT * FROM %s WHERE c=? AND b CONTAINS ?", set(1, 2, 3), 1);
+        assertRows(execute("SELECT * FROM %s WHERE c=? AND b CONTAINS ? ALLOW FILTERING", set(1, 2, 3), 1),
+            row(0, list(1, 2, 3), set(1, 2, 3), map(1, "a"))
+        );
+
+        assertInvalidMessage("ALLOW FILTERING", "SELECT * FROM %s WHERE c=? AND d = ?", set(1, 2, 3), map(1, "a"));
+        assertRows(execute("SELECT * FROM %s WHERE c=? AND d = ? ALLOW FILTERING", set(1, 2, 3), map(1, "a")),
+            row(0, list(1, 2, 3), set(1, 2, 3), map(1, "a")),
+            row(0, list(4, 5, 6), set(1, 2, 3), map(1, "a"))
+        );
+
+        assertInvalidMessage("ALLOW FILTERING", "SELECT * FROM %s WHERE c=? AND d CONTAINS ?", set(1, 2, 3), "a");
+        assertRows(execute("SELECT * FROM %s WHERE c=? AND d CONTAINS ? ALLOW FILTERING", set(1, 2, 3), "a"),
+            row(0, list(1, 2, 3), set(1, 2, 3), map(1, "a")),
+            row(0, list(4, 5, 6), set(1, 2, 3), map(1, "a"))
+        );
+
+        assertInvalidMessage("ALLOW FILTERING", "SELECT * FROM %s WHERE c=? AND d CONTAINS KEY ?", set(1, 2, 3), 1);
+        assertRows(execute("SELECT * FROM %s WHERE c=? AND d CONTAINS KEY ? ALLOW FILTERING", set(1, 2, 3), 1),
+            row(0, list(1, 2, 3), set(1, 2, 3), map(1, "a")),
+            row(0, list(4, 5, 6), set(1, 2, 3), map(1, "a"))
+        );
+
+        // index lookup on d
+        assertRows(execute("SELECT * FROM %s WHERE d=?", map(1, "a")),
+            row(0, list(1, 2, 3), set(1, 2, 3), map(1, "a")),
+            row(0, list(4, 5, 6), set(1, 2, 3), map(1, "a"))
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE d=?", map(2, "b")),
+            row(1, list(1, 2, 3), set(4, 5, 6), map(2, "b")),
+            row(1, list(4, 5, 6), set(4, 5, 6), map(2, "b"))
+        );
+
+        assertEmpty(execute("SELECT * FROM %s WHERE d=?", map(3, "c")));
+
+        assertInvalidMessage("ALLOW FILTERING", "SELECT * FROM %s WHERE d=? AND c=?", map(1, "a"), set(1, 2, 3));
+        assertRows(execute("SELECT * FROM %s WHERE d=? AND b=? ALLOW FILTERING", map(1, "a"), list(1, 2, 3)),
+            row(0, list(1, 2, 3), set(1, 2, 3), map(1, "a"))
+        );
+
+        assertInvalidMessage("ALLOW FILTERING", "SELECT * FROM %s WHERE d=? AND b CONTAINS ?", map(1, "a"), 3);
+        assertRows(execute("SELECT * FROM %s WHERE d=? AND b CONTAINS ? ALLOW FILTERING", map(1, "a"), 3),
+            row(0, list(1, 2, 3), set(1, 2, 3), map(1, "a"))
+        );
+
+        assertInvalidMessage("ALLOW FILTERING", "SELECT * FROM %s WHERE d=? AND b=? AND c=?", map(1, "a"), list(1, 2, 3), set(1, 2, 3));
+        assertRows(execute("SELECT * FROM %s WHERE d=? AND b=? AND c=? ALLOW FILTERING", map(1, "a"), list(1, 2, 3), set(1, 2, 3)),
+            row(0, list(1, 2, 3), set(1, 2, 3), map(1, "a"))
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE d=? AND b CONTAINS ? AND c CONTAINS ? ALLOW FILTERING", map(1, "a"), 2, 2),
+            row(0, list(1, 2, 3), set(1, 2, 3), map(1, "a"))
+        );
+
+        execute("DELETE d FROM %s WHERE a=? AND b=?", 0, list(1, 2, 3));
+        assertRows(execute("SELECT * FROM %s WHERE d=?", map(1, "a")),
+            row(0, list(4, 5, 6), set(1, 2, 3), map(1, "a"))
+        );
+    }
+
+    /** Test for CASSANDRA-8302 */
+    @Test
+    public void testClusteringColumnFiltering() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int, b frozen<map<int, int>>, c int, d int, PRIMARY KEY (a, b, c))");
+        createIndex("CREATE INDEX c_index ON %s (c)");
+        createIndex("CREATE INDEX d_index ON %s (d)");
+
+        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, map(0, 0, 1, 1), 0, 0);
+        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, map(1, 1, 2, 2), 0, 0);
+        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 1, map(0, 0, 1, 1), 0, 0);
+        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 1, map(1, 1, 2, 2), 0, 0);
+
+        assertRows(execute("SELECT * FROM %s WHERE d=? AND b CONTAINS ? ALLOW FILTERING", 0, 0),
+                row(0, map(0, 0, 1, 1), 0, 0),
+                row(1, map(0, 0, 1, 1), 0, 0)
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE d=? AND b CONTAINS KEY ? ALLOW FILTERING", 0, 0),
+                row(0, map(0, 0, 1, 1), 0, 0),
+                row(1, map(0, 0, 1, 1), 0, 0)
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE a=? AND d=? AND b CONTAINS ? ALLOW FILTERING", 0, 0, 0),
+                row(0, map(0, 0, 1, 1), 0, 0)
+        );
+        assertRows(execute("SELECT * FROM %s WHERE a=? AND d=? AND b CONTAINS KEY ? ALLOW FILTERING", 0, 0, 0),
+                row(0, map(0, 0, 1, 1), 0, 0)
+        );
+
+        dropIndex("DROP INDEX %s.d_index");
+
+        assertRows(execute("SELECT * FROM %s WHERE c=? AND b CONTAINS ? ALLOW FILTERING", 0, 0),
+                row(0, map(0, 0, 1, 1), 0, 0),
+                row(1, map(0, 0, 1, 1), 0, 0)
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE c=? AND b CONTAINS KEY ? ALLOW FILTERING", 0, 0),
+                row(0, map(0, 0, 1, 1), 0, 0),
+                row(1, map(0, 0, 1, 1), 0, 0)
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE a=? AND c=? AND b CONTAINS ? ALLOW FILTERING", 0, 0, 0),
+                row(0, map(0, 0, 1, 1), 0, 0)
+        );
+        assertRows(execute("SELECT * FROM %s WHERE a=? AND c=? AND b CONTAINS KEY ? ALLOW FILTERING", 0, 0, 0),
+                row(0, map(0, 0, 1, 1), 0, 0)
+        );
+    }
+
+    @Test
+    public void testFrozenListInMap() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int primary key, m map<frozen<list<int>>, int>)");
+
+        execute("INSERT INTO %s (k, m) VALUES (1, {[1, 2, 3] : 1})");
+        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
+                row(1, map(list(1, 2, 3), 1)));
+
+        execute("UPDATE %s SET m[[1, 2, 3]]=2 WHERE k=1");
+        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
+                row(1, map(list(1, 2, 3), 2)));
+
+        execute("UPDATE %s SET m = m + ? WHERE k=1", map(list(4, 5, 6), 3));
+        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
+                row(1,
+                    map(list(1, 2, 3), 2,
+                        list(4, 5, 6), 3)));
+
+        execute("DELETE m[[1, 2, 3]] FROM %s WHERE k = 1");
+        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
+                row(1, map(list(4, 5, 6), 3)));
+    }
+
+    @Test
+    public void testFrozenListInSet() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int primary key, s set<frozen<list<int>>>)");
+
+        execute("INSERT INTO %s (k, s) VALUES (1, {[1, 2, 3]})");
+        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
+                row(1, set(list(1, 2, 3)))
+        );
+
+        execute("UPDATE %s SET s = s + ? WHERE k=1", set(list(4, 5, 6)));
+        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
+                row(1, set(list(1, 2, 3), list(4, 5, 6)))
+        );
+
+        execute("UPDATE %s SET s = s - ? WHERE k=1", set(list(4, 5, 6)));
+        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
+                row(1, set(list(1, 2, 3)))
+        );
+
+        execute("DELETE s[[1, 2, 3]] FROM %s WHERE k = 1");
+        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
+                row(1, null)
+        );
+    }
+
+    @Test
+    public void testFrozenListInList() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int primary key, l list<frozen<list<int>>>)");
+
+        execute("INSERT INTO %s (k, l) VALUES (1, [[1, 2, 3]])");
+        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
+                row(1, list(list(1, 2, 3)))
+        );
+
+        execute("UPDATE %s SET l[?]=? WHERE k=1", 0, list(4, 5, 6));
+        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
+                row(1, list(list(4, 5, 6)))
+        );
+
+        execute("UPDATE %s SET l = ? + l WHERE k=1", list(list(1, 2, 3)));
+        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
+                row(1, list(list(1, 2, 3), list(4, 5, 6)))
+        );
+
+        execute("UPDATE %s SET l = l + ? WHERE k=1", list(list(7, 8, 9)));
+        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
+                row(1, list(list(1, 2, 3), list(4, 5, 6), list(7, 8, 9)))
+        );
+
+        execute("UPDATE %s SET l = l - ? WHERE k=1", list(list(4, 5, 6)));
+        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
+                row(1, list(list(1, 2, 3), list(7, 8, 9)))
+        );
+
+        execute("DELETE l[0] FROM %s WHERE k = 1");
+        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
+                row(1, list(list(7, 8, 9)))
+        );
+    }
+
+    @Test
+    public void testFrozenMapInMap() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int primary key, m map<frozen<map<int, int>>, int>)");
+
+        execute("INSERT INTO %s (k, m) VALUES (1, {{1 : 1, 2 : 2} : 1})");
+        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
+                row(1, map(map(1, 1, 2, 2), 1)));
+
+        execute("UPDATE %s SET m[?]=2 WHERE k=1", map(1, 1, 2, 2));
+        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
+                row(1, map(map(1, 1, 2, 2), 2)));
+
+        execute("UPDATE %s SET m = m + ? WHERE k=1", map(map(3, 3, 4, 4), 3));
+        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
+                row(1,
+                    map(map(1, 1, 2, 2), 2,
+                        map(3, 3, 4, 4), 3)));
+
+        execute("DELETE m[?] FROM %s WHERE k = 1", map(1, 1, 2, 2));
+        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
+                row(1, map(map(3, 3, 4, 4), 3)));
+    }
+
+    @Test
+    public void testFrozenMapInSet() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int primary key, s set<frozen<map<int, int>>>)");
+
+        execute("INSERT INTO %s (k, s) VALUES (1, {{1 : 1, 2 : 2}})");
+
+        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
+                row(1, set(map(1, 1, 2, 2)))
+        );
+
+        execute("UPDATE %s SET s = s + ? WHERE k=1", set(map(3, 3, 4, 4)));
+        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
+                row(1, set(map(1, 1, 2, 2), map(3, 3, 4, 4)))
+        );
+
+        execute("UPDATE %s SET s = s - ? WHERE k=1", set(map(3, 3, 4, 4)));
+        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
+                row(1, set(map(1, 1, 2, 2)))
+        );
+
+        execute("DELETE s[?] FROM %s WHERE k = 1", map(1, 1, 2, 2));
+        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
+                row(1, null)
+        );
+    }
+
+    @Test
+    public void testFrozenMapInList() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int primary key, l list<frozen<map<int, int>>>)");
+
+        execute("INSERT INTO %s (k, l) VALUES (1, [{1 : 1, 2 : 2}])");
+        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
+                row(1, list(map(1, 1, 2, 2)))
+        );
+
+        execute("UPDATE %s SET l[?]=? WHERE k=1", 0, map(3, 3, 4, 4));
+        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
+                row(1, list(map(3, 3, 4, 4)))
+        );
+
+        execute("UPDATE %s SET l = ? + l WHERE k=1", list(map(1, 1, 2, 2)));
+        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
+                row(1, list(map(1, 1, 2, 2), map(3, 3, 4, 4)))
+        );
+
+        execute("UPDATE %s SET l = l + ? WHERE k=1", list(map(5, 5, 6, 6)));
+        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
+                row(1, list(map(1, 1, 2, 2), map(3, 3, 4, 4), map(5, 5, 6, 6)))
+        );
+
+        execute("UPDATE %s SET l = l - ? WHERE k=1", list(map(3, 3, 4, 4)));
+        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
+                row(1, list(map(1, 1, 2, 2), map(5, 5, 6, 6)))
+        );
+
+        execute("DELETE l[0] FROM %s WHERE k = 1");
+        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
+                row(1, list(map(5, 5, 6, 6)))
+        );
+    }
+
+    @Test
+    public void testFrozenSetInMap() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int primary key, m map<frozen<set<int>>, int>)");
+
+        execute("INSERT INTO %s (k, m) VALUES (1, {{1, 2, 3} : 1})");
+        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
+                row(1, map(set(1, 2, 3), 1)));
+
+        execute("UPDATE %s SET m[?]=2 WHERE k=1", set(1, 2, 3));
+        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
+                row(1, map(set(1, 2, 3), 2)));
+
+        execute("UPDATE %s SET m = m + ? WHERE k=1", map(set(4, 5, 6), 3));
+        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
+                row(1,
+                    map(set(1, 2, 3), 2,
+                        set(4, 5, 6), 3)));
+
+        execute("DELETE m[?] FROM %s WHERE k = 1", set(1, 2, 3));
+        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
+                row(1, map(set(4, 5, 6), 3)));
+    }
+
+    @Test
+    public void testFrozenSetInSet() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int primary key, s set<frozen<set<int>>>)");
+
+        execute("INSERT INTO %s (k, s) VALUES (1, {{1, 2, 3}})");
+
+        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
+                row(1, set(set(1, 2, 3)))
+        );
+
+        execute("UPDATE %s SET s = s + ? WHERE k=1", set(set(4, 5, 6)));
+        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
+                row(1, set(set(1, 2, 3), set(4, 5, 6)))
+        );
+
+        execute("UPDATE %s SET s = s - ? WHERE k=1", set(set(4, 5, 6)));
+        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
+                row(1, set(set(1, 2, 3)))
+        );
+
+        execute("DELETE s[?] FROM %s WHERE k = 1", set(1, 2, 3));
+        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
+                row(1, null)
+        );
+    }
+
+    @Test
+    public void testFrozenSetInList() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int primary key, l list<frozen<set<int>>>)");
+
+        execute("INSERT INTO %s (k, l) VALUES (1, [{1, 2, 3}])");
+        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
+                row(1, list(set(1, 2, 3)))
+        );
+
+        execute("UPDATE %s SET l[?]=? WHERE k=1", 0, set(4, 5, 6));
+        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
+                row(1, list(set(4, 5, 6)))
+        );
+
+        execute("UPDATE %s SET l = ? + l WHERE k=1", list(set(1, 2, 3)));
+        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
+                row(1, list(set(1, 2, 3), set(4, 5, 6)))
+        );
+
+        execute("UPDATE %s SET l = l + ? WHERE k=1", list(set(7, 8, 9)));
+        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
+                row(1, list(set(1, 2, 3), set(4, 5, 6), set(7, 8, 9)))
+        );
+
+        execute("UPDATE %s SET l = l - ? WHERE k=1", list(set(4, 5, 6)));
+        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
+                row(1, list(set(1, 2, 3), set(7, 8, 9)))
+        );
+
+        execute("DELETE l[0] FROM %s WHERE k = 1");
+        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
+                row(1, list(set(7, 8, 9)))
+        );
+    }
+
+    @Test
+    public void testUserDefinedTypes() throws Throwable
+    {
+        String myType = createType("CREATE TYPE %s (a set<int>, b tuple<list<int>>)");
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, v frozen<" + myType + ">)");
+        execute("INSERT INTO %s (k, v) VALUES (?, {a: ?, b: ?})", 0, set(1, 2, 3), tuple(list(1, 2, 3)));
+        assertRows(execute("SELECT v.a, v.b FROM %s WHERE k=?", 0),
+            row(set(1, 2, 3), tuple(list(1, 2, 3)))
+        );
+    }
+
+    private static String clean(String classname)
+    {
+        return StringUtils.remove(classname, "org.apache.cassandra.db.marshal.");
+    }
+
+    @Test
+    public void testToString()
+    {
+        // set<frozen<list<int>>>
+        SetType t = SetType.getInstance(ListType.getInstance(Int32Type.instance, false), true);
+        assertEquals("SetType(FrozenType(ListType(Int32Type)))", clean(t.toString()));
+        assertEquals("SetType(ListType(Int32Type))", clean(t.toString(true)));
+
+        // frozen<set<list<int>>>
+        t = SetType.getInstance(ListType.getInstance(Int32Type.instance, false), false);
+        assertEquals("FrozenType(SetType(ListType(Int32Type)))", clean(t.toString()));
+        assertEquals("SetType(ListType(Int32Type))", clean(t.toString(true)));
+
+        // map<frozen<list<int>>, int>
+        MapType m = MapType.getInstance(ListType.getInstance(Int32Type.instance, false), Int32Type.instance, true);
+        assertEquals("MapType(FrozenType(ListType(Int32Type)),Int32Type)", clean(m.toString()));
+        assertEquals("MapType(ListType(Int32Type),Int32Type)", clean(m.toString(true)));
+
+        // frozen<map<list<int>, int>>
+        m = MapType.getInstance(ListType.getInstance(Int32Type.instance, false), Int32Type.instance, false);
+        assertEquals("FrozenType(MapType(ListType(Int32Type),Int32Type))", clean(m.toString()));
+        assertEquals("MapType(ListType(Int32Type),Int32Type)", clean(m.toString(true)));
+
+        // tuple<set<int>>
+        List<AbstractType<?>> types = new ArrayList<>();
+        types.add(SetType.getInstance(Int32Type.instance, true));
+        TupleType tuple = new TupleType(types);
+        assertEquals("TupleType(SetType(Int32Type))", clean(tuple.toString()));
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f797bfa4/test/unit/org/apache/cassandra/cql3/validation/entities/SecondaryIndexTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/SecondaryIndexTest.java b/test/unit/org/apache/cassandra/cql3/validation/entities/SecondaryIndexTest.java
new file mode 100644
index 0000000..a433d06
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/SecondaryIndexTest.java
@@ -0,0 +1,644 @@
+/*
+ * 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.validation.entities;
+
+import java.nio.ByteBuffer;
+import java.util.HashMap;
+import java.util.Locale;
+import java.util.Map;
+import java.util.UUID;
+
+import org.apache.commons.lang.StringUtils;
+
+import org.junit.Test;
+
+import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.exceptions.SyntaxException;
+import org.apache.cassandra.utils.FBUtilities;
+
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+public class SecondaryIndexTest extends CQLTester
+{
+    private static final int TOO_BIG = 1024 * 65;
+
+    @Test
+    public void testCreateAndDropIndex() throws Throwable
+    {
+        testCreateAndDropIndex("test", false);
+        testCreateAndDropIndex("test2", true);
+    }
+
+    @Test
+    public void testCreateAndDropIndexWithQuotedIdentifier() throws Throwable
+    {
+        testCreateAndDropIndex("\"quoted_ident\"", false);
+        testCreateAndDropIndex("\"quoted_ident2\"", true);
+    }
+
+    @Test
+    public void testCreateAndDropIndexWithCamelCaseIdentifier() throws Throwable
+    {
+        testCreateAndDropIndex("CamelCase", false);
+        testCreateAndDropIndex("CamelCase2", true);
+    }
+
+    /**
+     * Test creating and dropping an index with the specified name.
+     *
+     * @param indexName the index name
+     * @param addKeyspaceOnDrop add the keyspace name in the drop statement
+     * @throws Throwable if an error occurs
+     */
+    private void testCreateAndDropIndex(String indexName, boolean addKeyspaceOnDrop) throws Throwable
+    {
+        execute("USE system");
+        assertInvalidMessage("Index '" + removeQuotes(indexName.toLowerCase(Locale.US)) + "' could not be found", "DROP INDEX " + indexName + ";");
+
+        createTable("CREATE TABLE %s (a int primary key, b int);");
+        createIndex("CREATE INDEX " + indexName + " ON %s(b);");
+        createIndex("CREATE INDEX IF NOT EXISTS " + indexName + " ON %s(b);");
+
+        assertInvalidMessage("Index already exists", "CREATE INDEX " + indexName + " ON %s(b)");
+
+        execute("INSERT INTO %s (a, b) values (?, ?);", 0, 0);
+        execute("INSERT INTO %s (a, b) values (?, ?);", 1, 1);
+        execute("INSERT INTO %s (a, b) values (?, ?);", 2, 2);
+        execute("INSERT INTO %s (a, b) values (?, ?);", 3, 1);
+
+        assertRows(execute("SELECT * FROM %s where b = ?", 1), row(1, 1), row(3, 1));
+        assertInvalidMessage("Index '" + removeQuotes(indexName.toLowerCase(Locale.US)) + "' could not be found in any of the tables of keyspace 'system'", "DROP INDEX " + indexName);
+
+        if (addKeyspaceOnDrop)
+        {
+            dropIndex("DROP INDEX " + KEYSPACE + "." + indexName);
+        }
+        else
+        {
+            execute("USE " + KEYSPACE);
+            dropIndex("DROP INDEX " + indexName);
+        }
+
+        assertInvalidMessage("No secondary indexes on the restricted columns support the provided operators",
+                             "SELECT * FROM %s where b = ?", 1);
+        dropIndex("DROP INDEX IF EXISTS " + indexName);
+        assertInvalidMessage("Index '" + removeQuotes(indexName.toLowerCase(Locale.US)) + "' could not be found", "DROP INDEX " + indexName);
+    }
+
+    /**
+     * Removes the quotes from the specified index name.
+     *
+     * @param indexName the index name from which the quotes must be removed.
+     * @return the unquoted index name.
+     */
+    private static String removeQuotes(String indexName)
+    {
+        return StringUtils.remove(indexName, '\"');
+    }
+
+    /**
+     * Check that you can query for an indexed column even with a key EQ clause,
+     * migrated from cql_tests.py:TestCQL.static_cf_test()
+     */
+    @Test
+    public void testSelectWithEQ() throws Throwable
+    {
+        createTable("CREATE TABLE %s (userid uuid PRIMARY KEY, firstname text, lastname text, age int)");
+        createIndex("CREATE INDEX byAge ON %s(age)");
+
+        UUID id1 = UUID.fromString("550e8400-e29b-41d4-a716-446655440000");
+        UUID id2 = UUID.fromString("f47ac10b-58cc-4372-a567-0e02b2c3d479");
+
+        execute("INSERT INTO %s (userid, firstname, lastname, age) VALUES (?, 'Frodo', 'Baggins', 32)", id1);
+        execute("UPDATE %s SET firstname = 'Samwise', lastname = 'Gamgee', age = 33 WHERE userid = ?", id2);
+
+        assertEmpty(execute("SELECT firstname FROM %s WHERE userid = ? AND age = 33", id1));
+
+        assertRows(execute("SELECT firstname FROM %s WHERE userid = ? AND age = 33", id2),
+                   row("Samwise"));
+    }
+
+    /**
+     * Check CREATE INDEX without name and validate the index can be dropped,
+     * migrated from cql_tests.py:TestCQL.nameless_index_test()
+     */
+    @Test
+    public void testNamelessIndex() throws Throwable
+    {
+        createTable(" CREATE TABLE %s (id text PRIMARY KEY, birth_year int)");
+
+        createIndex("CREATE INDEX on %s (birth_year)");
+
+        execute("INSERT INTO %s (id, birth_year) VALUES ('Tom', 42)");
+        execute("INSERT INTO %s (id, birth_year) VALUES ('Paul', 24)");
+        execute("INSERT INTO %s (id, birth_year) VALUES ('Bob', 42)");
+
+        assertRows(execute("SELECT id FROM %s WHERE birth_year = 42"),
+                   row("Tom"),
+                   row("Bob"));
+
+        execute("DROP INDEX %s_birth_year_idx");
+
+        assertInvalid("SELECT id FROM users WHERE birth_year = 42");
+    }
+
+    /**
+     * Test range queries with 2ndary indexes (#4257),
+     * migrated from cql_tests.py:TestCQL.range_query_2ndary_test()
+     */
+    @Test
+    public void testRangeQuery() throws Throwable
+    {
+        createTable("CREATE TABLE %s (id int primary key, row int, setid int)");
+        createIndex("CREATE INDEX indextest_setid_idx ON %s (setid)");
+
+        execute("INSERT INTO %s (id, row, setid) VALUES (?, ?, ?)", 0, 0, 0);
+        execute("INSERT INTO %s (id, row, setid) VALUES (?, ?, ?)", 1, 1, 0);
+        execute("INSERT INTO %s (id, row, setid) VALUES (?, ?, ?)", 2, 2, 0);
+        execute("INSERT INTO %s (id, row, setid) VALUES (?, ?, ?)", 3, 3, 0);
+
+        assertInvalid("SELECT * FROM %s WHERE setid = 0 AND row < 1");
+
+        assertRows(execute("SELECT * FROM %s WHERE setid = 0 AND row < 1 ALLOW FILTERING"),
+                   row(0, 0, 0));
+    }
+
+    /**
+     * Check for unknown compression parameters options (#4266),
+     * migrated from cql_tests.py:TestCQL.compression_option_validation_test()
+     */
+    @Test
+    public void testUnknownCompressionOptions() throws Throwable
+    {
+        String tableName = createTableName();
+        assertInvalidThrow(SyntaxException.class, String.format(
+                                                               "CREATE TABLE %s (key varchar PRIMARY KEY, password varchar, gender varchar) WITH compression_parameters:sstable_compressor = 'DeflateCompressor'", tableName));
+
+
+        assertInvalidThrow(ConfigurationException.class, String.format(
+                                                                      "CREATE TABLE %s (key varchar PRIMARY KEY, password varchar, gender varchar) WITH compression = { 'sstable_compressor': 'DeflateCompressor' }", tableName));
+    }
+
+    /**
+     * Check one can use arbitrary name for datacenter when creating keyspace (#4278),
+     * migrated from cql_tests.py:TestCQL.keyspace_creation_options_test()
+     */
+    @Test
+    public void testDataCenterName() throws Throwable
+    {
+       execute("CREATE KEYSPACE Foo WITH replication = { 'class' : 'NetworkTopologyStrategy', 'us-east' : 1, 'us-west' : 1 };");
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.indexes_composite_test()
+     */
+    @Test
+    public void testIndexOnComposite() throws Throwable
+    {
+        String tableName = createTable("CREATE TABLE %s (blog_id int, timestamp int, author text, content text, PRIMARY KEY (blog_id, timestamp))");
+
+        execute("INSERT INTO %s (blog_id, timestamp, author, content) VALUES (?, ?, ?, ?)", 0, 0, "bob", "1st post");
+        execute("INSERT INTO %s (blog_id, timestamp, author, content) VALUES (?, ?, ?, ?)", 0, 1, "tom", "2nd post");
+        execute("INSERT INTO %s (blog_id, timestamp, author, content) VALUES (?, ?, ?, ?)", 0, 2, "bob", "3rd post");
+        execute("INSERT INTO %s (blog_id, timestamp, author, content) VALUES (?, ?, ?, ?)", 0, 3, "tom", "4th post");
+        execute("INSERT INTO %s (blog_id, timestamp, author, content) VALUES (?, ?, ?, ?)", 1, 0, "bob", "5th post");
+
+        createIndex("CREATE INDEX authoridx ON %s (author)");
+
+        assertTrue(waitForIndex(keyspace(), tableName, "authoridx"));
+
+        assertRows(execute("SELECT blog_id, timestamp FROM %s WHERE author = 'bob'"),
+                   row(1, 0),
+                   row(0, 0),
+                   row(0, 2));
+
+        execute("INSERT INTO %s (blog_id, timestamp, author, content) VALUES (?, ?, ?, ?)", 1, 1, "tom", "6th post");
+        execute("INSERT INTO %s (blog_id, timestamp, author, content) VALUES (?, ?, ?, ?)", 1, 2, "tom", "7th post");
+        execute("INSERT INTO %s (blog_id, timestamp, author, content) VALUES (?, ?, ?, ?)", 1, 3, "bob", "8th post");
+
+        assertRows(execute("SELECT blog_id, timestamp FROM %s WHERE author = 'bob'"),
+                   row(1, 0),
+                   row(1, 3),
+                   row(0, 0),
+                   row(0, 2));
+
+        execute("DELETE FROM %s WHERE blog_id = 0 AND timestamp = 2");
+
+        assertRows(execute("SELECT blog_id, timestamp FROM %s WHERE author = 'bob'"),
+                   row(1, 0),
+                   row(1, 3),
+                   row(0, 0));
+    }
+
+    /**
+     * Test for the validation bug of #4709,
+     * migrated from cql_tests.py:TestCQL.refuse_in_with_indexes_test()
+     */
+    @Test
+    public void testInvalidIndexSelect() throws Throwable
+    {
+        createTable("create table %s (pk varchar primary key, col1 varchar, col2 varchar)");
+        createIndex("create index on %s (col1)");
+        createIndex("create index on %s (col2)");
+
+        execute("insert into %s (pk, col1, col2) values ('pk1','foo1','bar1')");
+        execute("insert into %s (pk, col1, col2) values ('pk1a','foo1','bar1')");
+        execute("insert into %s (pk, col1, col2) values ('pk1b','foo1','bar1')");
+        execute("insert into %s (pk, col1, col2) values ('pk1c','foo1','bar1')");
+        execute("insert into %s (pk, col1, col2) values ('pk2','foo2','bar2')");
+        execute("insert into %s (pk, col1, col2) values ('pk3','foo3','bar3')");
+        assertInvalid("select * from %s where col2 in ('bar1', 'bar2')");
+
+        //Migrated from cql_tests.py:TestCQL.bug_6050_test()
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, a int, b int)");
+
+        createIndex("CREATE INDEX ON %s (a)");
+        assertInvalid("SELECT * FROM %s WHERE a = 3 AND b IN (1, 3)");
+
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.edge_2i_on_complex_pk_test()
+     */
+    @Test
+    public void testIndexesOnComplexPrimaryKey() throws Throwable
+    {
+        createTable("CREATE TABLE %s (pk0 int, pk1 int, ck0 int, ck1 int, ck2 int, value int, PRIMARY KEY ((pk0, pk1), ck0, ck1, ck2))");
+
+        execute("CREATE INDEX ON %s (pk0)");
+        execute("CREATE INDEX ON %s (ck0)");
+        execute("CREATE INDEX ON %s (ck1)");
+        execute("CREATE INDEX ON %s (ck2)");
+
+        execute("INSERT INTO %s (pk0, pk1, ck0, ck1, ck2, value) VALUES (0, 1, 2, 3, 4, 5)");
+        execute("INSERT INTO %s (pk0, pk1, ck0, ck1, ck2, value) VALUES (1, 2, 3, 4, 5, 0)");
+        execute("INSERT INTO %s (pk0, pk1, ck0, ck1, ck2, value) VALUES (2, 3, 4, 5, 0, 1)");
+        execute("INSERT INTO %s (pk0, pk1, ck0, ck1, ck2, value) VALUES (3, 4, 5, 0, 1, 2)");
+        execute("INSERT INTO %s (pk0, pk1, ck0, ck1, ck2, value) VALUES (4, 5, 0, 1, 2, 3)");
+        execute("INSERT INTO %s (pk0, pk1, ck0, ck1, ck2, value) VALUES (5, 0, 1, 2, 3, 4)");
+
+        assertRows(execute("SELECT value FROM %s WHERE pk0 = 2"),
+                   row(1));
+
+        assertRows(execute("SELECT value FROM %s WHERE ck0 = 0"),
+                   row(3));
+
+        assertRows(execute("SELECT value FROM %s WHERE pk0 = 3 AND pk1 = 4 AND ck1 = 0"),
+                   row(2));
+
+        assertRows(execute("SELECT value FROM %s WHERE pk0 = 5 AND pk1 = 0 AND ck0 = 1 AND ck2 = 3 ALLOW FILTERING"),
+                   row(4));
+    }
+
+    /**
+     * Test for CASSANDRA-5240,
+     * migrated from cql_tests.py:TestCQL.bug_5240_test()
+     */
+    @Test
+    public void testIndexOnCompoundRowKey() throws Throwable
+    {
+        createTable("CREATE TABLE %s (interval text, seq int, id int, severity int, PRIMARY KEY ((interval, seq), id) ) WITH CLUSTERING ORDER BY (id DESC)");
+
+        execute("CREATE INDEX ON %s (severity)");
+
+        execute("insert into %s (interval, seq, id , severity) values('t',1, 1, 1)");
+        execute("insert into %s (interval, seq, id , severity) values('t',1, 2, 1)");
+        execute("insert into %s (interval, seq, id , severity) values('t',1, 3, 2)");
+        execute("insert into %s (interval, seq, id , severity) values('t',1, 4, 3)");
+        execute("insert into %s (interval, seq, id , severity) values('t',2, 1, 3)");
+        execute("insert into %s (interval, seq, id , severity) values('t',2, 2, 3)");
+        execute("insert into %s (interval, seq, id , severity) values('t',2, 3, 1)");
+        execute("insert into %s (interval, seq, id , severity) values('t',2, 4, 2)");
+
+        assertRows(execute("select * from %s where severity = 3 and interval = 't' and seq =1"),
+                   row("t", 1, 4, 3));
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.secondary_index_counters()
+     */
+    @Test
+    public void testIndexOnCountersInvalid() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, c counter)");
+        assertInvalid("CREATE INDEX ON test(c)");
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.collection_indexing_test()
+     */
+    @Test
+    public void testIndexOnCollections() throws Throwable
+    {
+        createTable(" CREATE TABLE %s ( k int, v int, l list<int>, s set<text>, m map<text, int>, PRIMARY KEY (k, v))");
+
+        createIndex("CREATE INDEX ON %s (l)");
+        createIndex("CREATE INDEX ON %s (s)");
+        createIndex("CREATE INDEX ON %s (m)");
+
+        execute("INSERT INTO %s (k, v, l, s, m) VALUES (0, 0, [1, 2],    {'a'},      {'a' : 1})");
+        execute("INSERT INTO %s (k, v, l, s, m) VALUES (0, 1, [3, 4],    {'b', 'c'}, {'a' : 1, 'b' : 2})");
+        execute("INSERT INTO %s (k, v, l, s, m) VALUES (0, 2, [1],       {'a', 'c'}, {'c' : 3})");
+        execute("INSERT INTO %s (k, v, l, s, m) VALUES (1, 0, [1, 2, 4], {},         {'b' : 1})");
+        execute("INSERT INTO %s (k, v, l, s, m) VALUES (1, 1, [4, 5],    {'d'},      {'a' : 1, 'b' : 3})");
+
+        // lists
+        assertRows(execute("SELECT k, v FROM %s WHERE l CONTAINS 1"), row(1, 0), row(0, 0), row(0, 2));
+        assertRows(execute("SELECT k, v FROM %s WHERE k = 0 AND l CONTAINS 1"), row(0, 0), row(0, 2));
+        assertRows(execute("SELECT k, v FROM %s WHERE l CONTAINS 2"), row(1, 0), row(0, 0));
+        assertEmpty(execute("SELECT k, v FROM %s WHERE l CONTAINS 6"));
+
+        // sets
+        assertRows(execute("SELECT k, v FROM %s WHERE s CONTAINS 'a'"), row(0, 0), row(0, 2));
+        assertRows(execute("SELECT k, v FROM %s WHERE k = 0 AND s CONTAINS 'a'"), row(0, 0), row(0, 2));
+        assertRows(execute("SELECT k, v FROM %s WHERE s CONTAINS 'd'"), row(1, 1));
+        assertEmpty(execute("SELECT k, v FROM %s  WHERE s CONTAINS 'e'"));
+
+        // maps
+        assertRows(execute("SELECT k, v FROM %s WHERE m CONTAINS 1"), row(1, 0), row(1, 1), row(0, 0), row(0, 1));
+        assertRows(execute("SELECT k, v FROM %s WHERE k = 0 AND m CONTAINS 1"), row(0, 0), row(0, 1));
+        assertRows(execute("SELECT k, v FROM %s WHERE m CONTAINS 2"), row(0, 1));
+        assertEmpty(execute("SELECT k, v FROM %s  WHERE m CONTAINS 4"));
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.map_keys_indexing()
+     */
+    @Test
+    public void testIndexOnMapKeys() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int, v int, m map<text, int>, PRIMARY KEY (k, v))");
+
+        createIndex("CREATE INDEX ON %s(keys(m))");
+
+        execute("INSERT INTO %s (k, v, m) VALUES (0, 0, {'a' : 1})");
+        execute("INSERT INTO %s (k, v, m) VALUES (0, 1, {'a' : 1, 'b' : 2})");
+        execute("INSERT INTO %s (k, v, m) VALUES (0, 2, {'c' : 3})");
+        execute("INSERT INTO %s (k, v, m) VALUES (1, 0, {'b' : 1})");
+        execute("INSERT INTO %s (k, v, m) VALUES (1, 1, {'a' : 1, 'b' : 3})");
+
+        // maps
+        assertRows(execute("SELECT k, v FROM %s WHERE m CONTAINS KEY 'a'"), row(1, 1), row(0, 0), row(0, 1));
+        assertRows(execute("SELECT k, v FROM %s WHERE k = 0 AND m CONTAINS KEY 'a'"), row(0, 0), row(0, 1));
+        assertRows(execute("SELECT k, v FROM %s WHERE m CONTAINS KEY 'c'"), row(0, 2));
+        assertEmpty(execute("SELECT k, v FROM %s  WHERE m CONTAINS KEY 'd'"));
+
+        // we're not allowed to create a value index if we already have a key one
+        assertInvalid("CREATE INDEX ON %s(m)");
+    }
+
+    /**
+     * Test for #6950 bug,
+     * migrated from cql_tests.py:TestCQL.key_index_with_reverse_clustering()
+     */
+    @Test
+    public void testIndexOnKeyWithReverseClustering() throws Throwable
+    {
+        createTable(" CREATE TABLE %s (k1 int, k2 int, v int, PRIMARY KEY ((k1, k2), v) ) WITH CLUSTERING ORDER BY (v DESC)");
+
+        createIndex("CREATE INDEX ON %s (k2)");
+
+        execute("INSERT INTO %s (k1, k2, v) VALUES (0, 0, 1)");
+        execute("INSERT INTO %s (k1, k2, v) VALUES (0, 1, 2)");
+        execute("INSERT INTO %s (k1, k2, v) VALUES (0, 0, 3)");
+        execute("INSERT INTO %s (k1, k2, v) VALUES (1, 0, 4)");
+        execute("INSERT INTO %s (k1, k2, v) VALUES (1, 1, 5)");
+        execute("INSERT INTO %s (k1, k2, v) VALUES (2, 0, 7)");
+        execute("INSERT INTO %s (k1, k2, v) VALUES (2, 1, 8)");
+        execute("INSERT INTO %s (k1, k2, v) VALUES (3, 0, 1)");
+
+        assertRows(execute("SELECT * FROM %s WHERE k2 = 0 AND v >= 2 ALLOW FILTERING"),
+                   row(2, 0, 7),
+                   row(0, 0, 3),
+                   row(1, 0, 4));
+    }
+
+    /**
+     * Test for CASSANDRA-6612,
+     * migrated from cql_tests.py:TestCQL.bug_6612_test()
+     */
+    @Test
+    public void testSelectCountOnIndexedColumn() throws Throwable
+    {
+        createTable("CREATE TABLE %s (username text, session_id text, app_name text, account text, last_access timestamp, created_on timestamp, PRIMARY KEY (username, session_id, app_name, account))");
+
+        createIndex("create index ON %s (app_name)");
+        createIndex("create index ON %s (last_access)");
+
+        assertRows(execute("select count(*) from %s where app_name='foo' and account='bar' and last_access > 4 allow filtering"), row(0L));
+
+        execute("insert into %s (username, session_id, app_name, account, last_access, created_on) values ('toto', 'foo', 'foo', 'bar', 12, 13)");
+
+        assertRows(execute("select count(*) from %s where app_name='foo' and account='bar' and last_access > 4 allow filtering"), row(1L));
+    }
+
+    /**
+     * Test for CASSANDRA-5732, Can not query secondary index
+     * migrated from cql_tests.py:TestCQL.bug_5732_test(),
+     * which was executing with a row cache size of 100 MB
+     * and restarting the node, here we just cleanup the cache.
+     */
+    @Test
+    public void testCanQuerySecondaryIndex() throws Throwable
+    {
+        String tableName = createTable("CREATE TABLE %s (k int PRIMARY KEY, v int,)");
+
+        execute("ALTER TABLE %s WITH CACHING='ALL'");
+        execute("INSERT INTO %s (k,v) VALUES (0,0)");
+        execute("INSERT INTO %s (k,v) VALUES (1,1)");
+
+        createIndex("CREATE INDEX testindex on %s (v)");
+        assertTrue(waitForIndex(keyspace(), tableName, "testindex"));
+
+        assertRows(execute("SELECT k FROM %s WHERE v = 0"), row(0));
+        cleanupCache();
+        assertRows(execute("SELECT k FROM %s WHERE v = 0"), row(0));
+    }
+
+    // CASSANDRA-8280/8081
+    // reject updates with indexed values where value > 64k
+    @Test
+    public void testIndexOnCompositeValueOver64k() throws Throwable
+    {
+        createTable("CREATE TABLE %s(a int, b int, c blob, PRIMARY KEY (a))");
+        createIndex("CREATE INDEX ON %s(c)");
+        failInsert("INSERT INTO %s (a, b, c) VALUES (0, 0, ?)", ByteBuffer.allocate(TOO_BIG));
+    }
+
+    @Test
+    public void testIndexOnClusteringColumnInsertPartitionKeyAndClusteringsOver64k() throws Throwable
+    {
+        createTable("CREATE TABLE %s(a blob, b blob, c blob, d int, PRIMARY KEY (a, b, c))");
+        createIndex("CREATE INDEX ON %s(b)");
+
+        // CompositeIndexOnClusteringKey creates index entries composed of the
+        // PK plus all of the non-indexed clustering columns from the primary row
+        // so we should reject where len(a) + len(c) > 65560 as this will form the
+        // total clustering in the index table
+        ByteBuffer a = ByteBuffer.allocate(100);
+        ByteBuffer b = ByteBuffer.allocate(10);
+        ByteBuffer c = ByteBuffer.allocate(FBUtilities.MAX_UNSIGNED_SHORT - 99);
+
+        failInsert("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, 0)", a, b, c);
+    }
+
+    @Test
+    public void testCompactTableWithValueOver64k() throws Throwable
+    {
+        createTable("CREATE TABLE %s(a int, b blob, PRIMARY KEY (a)) WITH COMPACT STORAGE");
+        createIndex("CREATE INDEX ON %s(b)");
+        failInsert("INSERT INTO %s (a, b) VALUES (0, ?)", ByteBuffer.allocate(TOO_BIG));
+    }
+
+    @Test
+    public void testIndexOnCollectionValueInsertPartitionKeyAndCollectionKeyOver64k() throws Throwable
+    {
+        createTable("CREATE TABLE %s(a blob , b map<blob, int>, PRIMARY KEY (a))");
+        createIndex("CREATE INDEX ON %s(b)");
+
+        // A collection key > 64k by itself will be rejected from
+        // the primary table.
+        // To test index validation we need to ensure that
+        // len(b) < 64k, but len(a) + len(b) > 64k as that will
+        // form the clustering in the index table
+        ByteBuffer a = ByteBuffer.allocate(100);
+        ByteBuffer b = ByteBuffer.allocate(FBUtilities.MAX_UNSIGNED_SHORT - 100);
+
+        failInsert("UPDATE %s SET b[?] = 0 WHERE a = ?", b, a);
+    }
+
+    @Test
+    public void testIndexOnCollectionKeyInsertPartitionKeyAndClusteringOver64k() throws Throwable
+    {
+        createTable("CREATE TABLE %s(a blob, b blob, c map<blob, int>, PRIMARY KEY (a, b))");
+        createIndex("CREATE INDEX ON %s(KEYS(c))");
+
+        // Basically the same as the case with non-collection clustering
+        // CompositeIndexOnCollectionKeyy creates index entries composed of the
+        // PK plus all of the clustering columns from the primary row, except the
+        // collection element - which becomes the partition key in the index table
+        ByteBuffer a = ByteBuffer.allocate(100);
+        ByteBuffer b = ByteBuffer.allocate(FBUtilities.MAX_UNSIGNED_SHORT - 100);
+        ByteBuffer c = ByteBuffer.allocate(10);
+
+        failInsert("UPDATE %s SET c[?] = 0 WHERE a = ? and b = ?", c, a, b);
+    }
+
+    @Test
+    public void testIndexOnPartitionKeyInsertValueOver64k() throws Throwable
+    {
+        createTable("CREATE TABLE %s(a int, b int, c blob, PRIMARY KEY ((a, b)))");
+        createIndex("CREATE INDEX ON %s(a)");
+        succeedInsert("INSERT INTO %s (a, b, c) VALUES (0, 0, ?)", ByteBuffer.allocate(TOO_BIG));
+    }
+
+    @Test
+    public void testIndexOnClusteringColumnInsertValueOver64k() throws Throwable
+    {
+        createTable("CREATE TABLE %s(a int, b int, c blob, PRIMARY KEY (a, b))");
+        createIndex("CREATE INDEX ON %s(b)");
+        succeedInsert("INSERT INTO %s (a, b, c) VALUES (0, 0, ?)", ByteBuffer.allocate(TOO_BIG));
+    }
+
+    @Test
+    public void testIndexOnFullCollectionEntryInsertCollectionValueOver64k() throws Throwable
+    {
+        createTable("CREATE TABLE %s(a int, b frozen<map<int, blob>>, PRIMARY KEY (a))");
+        createIndex("CREATE INDEX ON %s(full(b))");
+        Map<Integer, ByteBuffer> map = new HashMap();
+        map.put(0, ByteBuffer.allocate(1024 * 65));
+        failInsert("INSERT INTO %s (a, b) VALUES (0, ?)", map);
+    }
+
+    public void failInsert(String insertCQL, Object...args) throws Throwable
+    {
+        try
+        {
+            execute(insertCQL, args);
+            fail("Expected statement to fail validation");
+        }
+        catch (Exception e)
+        {
+            // as expected
+        }
+    }
+
+    public void succeedInsert(String insertCQL, Object...args) throws Throwable
+    {
+        execute(insertCQL, args);
+        flush();
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.clustering_indexing_test()
+     */
+    @Test
+    public void testIndexesOnClustering() throws Throwable
+    {
+        createTable("CREATE TABLE %s ( id1 int, id2 int, author text, time bigint, v1 text, v2 text, PRIMARY KEY ((id1, id2), author, time))");
+
+        createIndex("CREATE INDEX ON %s (time)");
+        execute("CREATE INDEX ON %s (id2)");
+
+        execute("INSERT INTO %s (id1, id2, author, time, v1, v2) VALUES(0, 0, 'bob', 0, 'A', 'A')");
+        execute("INSERT INTO %s (id1, id2, author, time, v1, v2) VALUES(0, 0, 'bob', 1, 'B', 'B')");
+        execute("INSERT INTO %s (id1, id2, author, time, v1, v2) VALUES(0, 1, 'bob', 2, 'C', 'C')");
+        execute("INSERT INTO %s (id1, id2, author, time, v1, v2) VALUES(0, 0, 'tom', 0, 'D', 'D')");
+        execute("INSERT INTO %s (id1, id2, author, time, v1, v2) VALUES(0, 1, 'tom', 1, 'E', 'E')");
+
+        assertRows(execute("SELECT v1 FROM %s WHERE time = 1"),
+                   row("B"), row("E"));
+
+        assertRows(execute("SELECT v1 FROM %s WHERE id2 = 1"),
+                   row("C"), row("E"));
+
+        assertRows(execute("SELECT v1 FROM %s WHERE id1 = 0 AND id2 = 0 AND author = 'bob' AND time = 0"),
+                   row("A"));
+
+        // Test for CASSANDRA-8206
+        execute("UPDATE %s SET v2 = null WHERE id1 = 0 AND id2 = 0 AND author = 'bob' AND time = 1");
+
+        assertRows(execute("SELECT v1 FROM %s WHERE id2 = 0"),
+                   row("A"), row("B"), row("D"));
+
+        assertRows(execute("SELECT v1 FROM %s WHERE time = 1"),
+                   row("B"), row("E"));
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.invalid_clustering_indexing_test()
+     */
+    @Test
+    public void testIndexesOnClusteringInvalid() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int, b int, c int, d int, PRIMARY KEY ((a, b))) WITH COMPACT STORAGE");
+        assertInvalid("CREATE INDEX ON %s (a)");
+        assertInvalid("CREATE INDEX ON %s (b)");
+
+        createTable("CREATE TABLE %s (a int, b int, c int, PRIMARY KEY (a, b)) WITH COMPACT STORAGE");
+        assertInvalid("CREATE INDEX ON %s (a)");
+        assertInvalid("CREATE INDEX ON %s (b)");
+        assertInvalid("CREATE INDEX ON %s (c)");
+
+        createTable("CREATE TABLE %s (a int, b int, c int static , PRIMARY KEY (a, b))");
+        assertInvalid("CREATE INDEX ON %s (c)");
+    }
+
+}


[26/32] cassandra git commit: 2.2 commit for CASSANDRA-9160

Posted by jm...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/01115f72/test/unit/org/apache/cassandra/cql3/MultiColumnRelationTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/MultiColumnRelationTest.java b/test/unit/org/apache/cassandra/cql3/MultiColumnRelationTest.java
deleted file mode 100644
index b380b1e..0000000
--- a/test/unit/org/apache/cassandra/cql3/MultiColumnRelationTest.java
+++ /dev/null
@@ -1,936 +0,0 @@
-/*
- * 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;
-
-import org.junit.Test;
-
-public class MultiColumnRelationTest extends CQLTester
-{
-    @Test
-    public void testSingleClusteringInvalidQueries() throws Throwable
-    {
-        for (String compactOption : new String[] { "", " WITH COMPACT STORAGE" })
-        {
-            createTable("CREATE TABLE %s (a int, b int, c int, PRIMARY KEY (a, b))" + compactOption);
-
-            assertInvalidSyntax("SELECT * FROM %s WHERE () = (?, ?)", 1, 2);
-            assertInvalidMessage("b cannot be restricted by more than one relation if it includes an Equal",
-                                 "SELECT * FROM %s WHERE a = 0 AND (b) = (?) AND (b) > (?)", 0, 0);
-            assertInvalidMessage("More than one restriction was found for the start bound on b",
-                                 "SELECT * FROM %s WHERE a = 0 AND (b) > (?) AND (b) > (?)", 0, 1);
-            assertInvalidMessage("Multi-column relations can only be applied to clustering columns but was applied to: a",
-                                 "SELECT * FROM %s WHERE (a, b) = (?, ?)", 0, 0);
-        }
-    }
-
-    @Test
-    public void testMultiClusteringInvalidQueries() throws Throwable
-    {
-        for (String compactOption : new String[] { "", " WITH COMPACT STORAGE" })
-        {
-            createTable("CREATE TABLE %s (a int, b int, c int, d int, PRIMARY KEY (a, b, c, d))" + compactOption);
-
-            assertInvalidSyntax("SELECT * FROM %s WHERE a = 0 AND (b, c) > ()");
-            assertInvalidMessage("Expected 2 elements in value tuple, but got 3: (?, ?, ?)",
-                                 "SELECT * FROM %s WHERE a = 0 AND (b, c) > (?, ?, ?)", 1, 2, 3);
-            assertInvalidMessage("Invalid null value in condition for column c",
-                                 "SELECT * FROM %s WHERE a = 0 AND (b, c) > (?, ?)", 1, null);
-
-            // Wrong order of columns
-            assertInvalidMessage("Clustering columns must appear in the PRIMARY KEY order in multi-column relations: (d, c, b) = (?, ?, ?)",
-                                 "SELECT * FROM %s WHERE a = 0 AND (d, c, b) = (?, ?, ?)", 0, 0, 0);
-            assertInvalidMessage("Clustering columns must appear in the PRIMARY KEY order in multi-column relations: (d, c, b) > (?, ?, ?)",
-                                 "SELECT * FROM %s WHERE a = 0 AND (d, c, b) > (?, ?, ?)", 0, 0, 0);
-
-            // Wrong number of values
-            assertInvalidMessage("Expected 3 elements in value tuple, but got 2: (?, ?)",
-                                 "SELECT * FROM %s WHERE a=0 AND (b, c, d) IN ((?, ?))", 0, 1);
-            assertInvalidMessage("Expected 3 elements in value tuple, but got 5: (?, ?, ?, ?, ?)",
-                                 "SELECT * FROM %s WHERE a=0 AND (b, c, d) IN ((?, ?, ?, ?, ?))", 0, 1, 2, 3, 4);
-
-            // Missing first clustering column
-            assertInvalidMessage("PRIMARY KEY column \"c\" cannot be restricted as preceding column \"b\" is not restricted",
-                                 "SELECT * FROM %s WHERE a = 0 AND (c, d) = (?, ?)", 0, 0);
-            assertInvalidMessage("PRIMARY KEY column \"c\" cannot be restricted as preceding column \"b\" is not restricted",
-                                 "SELECT * FROM %s WHERE a = 0 AND (c, d) > (?, ?)", 0, 0);
-
-            // Nulls
-            assertInvalidMessage("Invalid null value in condition for columns: [b, c, d]",
-                                 "SELECT * FROM %s WHERE a = 0 AND (b, c, d) IN ((?, ?, ?))", 1, 2, null);
-
-            // Wrong type for 'd'
-            assertInvalid("SELECT * FROM %s WHERE a = 0 AND (b, c, d) = (?, ?, ?)", 1, 2, "foobar");
-            assertInvalid("SELECT * FROM %s WHERE a = 0 AND b = (?, ?, ?)", 1, 2, 3);
-
-            // Mix single and tuple inequalities
-            assertInvalidMessage("Mixing single column relations and multi column relations on clustering columns is not allowed",
-                                 "SELECT * FROM %s WHERE a = 0 AND (b, c, d) > (?, ?, ?) AND b < ?", 0, 1, 0, 1);
-            assertInvalidMessage("Mixing single column relations and multi column relations on clustering columns is not allowed",
-                                 "SELECT * FROM %s WHERE a = 0 AND (b, c, d) > (?, ?, ?) AND c < ?", 0, 1, 0, 1);
-            assertInvalidMessage("Mixing single column relations and multi column relations on clustering columns is not allowed",
-                                 "SELECT * FROM %s WHERE a = 0 AND b > ? AND (b, c, d) < (?, ?, ?)", 1, 1, 1, 0);
-            assertInvalidMessage("Mixing single column relations and multi column relations on clustering columns is not allowed",
-                                 "SELECT * FROM %s WHERE a = 0 AND c > ? AND (b, c, d) < (?, ?, ?)", 1, 1, 1, 0);
-
-            assertInvalidMessage("Multi-column relations can only be applied to clustering columns but was applied to: a",
-                                 "SELECT * FROM %s WHERE (a, b, c, d) IN ((?, ?, ?, ?))", 0, 1, 2, 3);
-            assertInvalidMessage("PRIMARY KEY column \"c\" cannot be restricted as preceding column \"b\" is not restricted",
-                                 "SELECT * FROM %s WHERE (c, d) IN ((?, ?))", 0, 1);
-
-            assertInvalidMessage("Clustering column \"c\" cannot be restricted (preceding column \"b\" is restricted by a non-EQ relation)",
-                                 "SELECT * FROM %s WHERE a = ? AND b > ?  AND (c, d) IN ((?, ?))", 0, 0, 0, 0);
-
-            assertInvalidMessage("Clustering column \"c\" cannot be restricted (preceding column \"b\" is restricted by a non-EQ relation)",
-                                 "SELECT * FROM %s WHERE a = ? AND b > ?  AND (c, d) > (?, ?)", 0, 0, 0, 0);
-            assertInvalidMessage("PRIMARY KEY column \"c\" cannot be restricted (preceding column \"b\" is restricted by a non-EQ relation)",
-                                 "SELECT * FROM %s WHERE a = ? AND (c, d) > (?, ?) AND b > ?  ", 0, 0, 0, 0);
-            assertInvalidMessage("Column \"c\" cannot be restricted by two tuple-notation inequalities not starting with the same column",
-                                 "SELECT * FROM %s WHERE a = ? AND (b, c) > (?, ?) AND (b) < (?) AND (c) < (?)", 0, 0, 0, 0, 0);
-            assertInvalidMessage("Column \"c\" cannot be restricted by two tuple-notation inequalities not starting with the same column",
-                                 "SELECT * FROM %s WHERE a = ? AND (c) < (?) AND (b, c) > (?, ?) AND (b) < (?)", 0, 0, 0, 0, 0);
-            assertInvalidMessage("Clustering column \"c\" cannot be restricted (preceding column \"b\" is restricted by a non-EQ relation)",
-                                 "SELECT * FROM %s WHERE a = ? AND (b) < (?) AND (c) < (?) AND (b, c) > (?, ?)", 0, 0, 0, 0, 0);
-
-            assertInvalidMessage("Column \"c\" cannot be restricted by two tuple-notation inequalities not starting with the same column",
-                                 "SELECT * FROM %s WHERE a = ? AND (b, c) > (?, ?) AND (c) < (?)", 0, 0, 0, 0);
-        }
-    }
-
-    @Test
-    public void testMultiAndSingleColumnRelationMix() throws Throwable
-    {
-        for (String compactOption : new String[]{"", " WITH COMPACT STORAGE"})
-        {
-            createTable("CREATE TABLE %s (a int, b int, c int, d int, PRIMARY KEY (a, b, c, d))" + compactOption);
-
-            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 0, 0);
-            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 1, 0);
-            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 1, 1);
-
-            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 0, 0);
-            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 1, 0);
-            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 1, 1);
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and b = ? and (c, d) = (?, ?)", 0, 1, 0, 0),
-                       row(0, 1, 0, 0));
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and b IN (?, ?) and (c, d) = (?, ?)", 0, 0, 1, 0, 0),
-                       row(0, 0, 0, 0),
-                       row(0, 1, 0, 0));
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and b = ? and (c) IN ((?))", 0, 1, 0),
-                       row(0, 1, 0, 0));
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and b IN (?, ?) and (c) IN ((?))", 0, 0, 1, 0),
-                       row(0, 0, 0, 0),
-                       row(0, 1, 0, 0));
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and b = ? and (c) IN ((?), (?))", 0, 1, 0, 1),
-                       row(0, 1, 0, 0),
-                       row(0, 1, 1, 0),
-                       row(0, 1, 1, 1));
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and b = ? and (c, d) IN ((?, ?))", 0, 1, 0, 0),
-                       row(0, 1, 0, 0));
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and b = ? and (c, d) IN ((?, ?), (?, ?))", 0, 1, 0, 0, 1, 1),
-                       row(0, 1, 0, 0),
-                       row(0, 1, 1, 1));
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and b IN (?, ?) and (c, d) IN ((?, ?), (?, ?))", 0, 0, 1, 0, 0, 1, 1),
-                       row(0, 0, 0, 0),
-                       row(0, 0, 1, 1),
-                       row(0, 1, 0, 0),
-                       row(0, 1, 1, 1));
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and b = ? and (c, d) > (?, ?)", 0, 1, 0, 0),
-                       row(0, 1, 1, 0),
-                       row(0, 1, 1, 1));
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and b IN (?, ?) and (c, d) > (?, ?)", 0, 0, 1, 0, 0),
-                       row(0, 0, 1, 0),
-                       row(0, 0, 1, 1),
-                       row(0, 1, 1, 0),
-                       row(0, 1, 1, 1));
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and b = ? and (c, d) > (?, ?) and (c) <= (?) ", 0, 1, 0, 0, 1),
-                       row(0, 1, 1, 0),
-                       row(0, 1, 1, 1));
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and b = ? and (c, d) >= (?, ?) and (c, d) < (?, ?)", 0, 1, 0, 0, 1, 1),
-                       row(0, 1, 0, 0),
-                       row(0, 1, 1, 0));
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b, c) = (?, ?) and d = ?", 0, 0, 1, 0),
-                       row(0, 0, 1, 0));
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b, c) IN ((?, ?), (?, ?)) and d = ?", 0, 0, 1, 0, 0, 0),
-                       row(0, 0, 0, 0),
-                       row(0, 0, 1, 0));
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and b = ? and (c) = (?) and d = ?", 0, 0, 1, 0),
-                       row(0, 0, 1, 0));
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b, c) = (?, ?) and d IN (?, ?)", 0, 0, 1, 0, 2),
-                       row(0, 0, 1, 0));
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and b = ? and (c) = (?) and d IN (?, ?)", 0, 0, 1, 0, 2),
-                       row(0, 0, 1, 0));
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b, c) = (?, ?) and d >= ?", 0, 0, 1, 0),
-                       row(0, 0, 1, 0),
-                       row(0, 0, 1, 1));
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and d < 1 and (b, c) = (?, ?) and d >= ?", 0, 0, 1, 0),
-                       row(0, 0, 1, 0));
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and d < 1 and (b, c) IN ((?, ?), (?, ?)) and d >= ?", 0, 0, 1, 0, 0, 0),
-                       row(0, 0, 0, 0),
-                       row(0, 0, 1, 0));
-        }
-    }
-
-    @Test
-    public void testSeveralMultiColumnRelation() throws Throwable
-    {
-        for (String compactOption : new String[]{"", " WITH COMPACT STORAGE"})
-        {
-            createTable("CREATE TABLE %s (a int, b int, c int, d int, PRIMARY KEY (a, b, c, d))" + compactOption);
-
-            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 0, 0);
-            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 1, 0);
-            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 1, 1);
-
-            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 0, 0);
-            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 1, 0);
-            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 1, 1);
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b) = (?) and (c, d) = (?, ?)", 0, 1, 0, 0),
-                       row(0, 1, 0, 0));
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b) IN ((?), (?)) and (c, d) = (?, ?)", 0, 0, 1, 0, 0),
-                       row(0, 0, 0, 0),
-                       row(0, 1, 0, 0));
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b) = (?) and (c) IN ((?))", 0, 1, 0),
-                       row(0, 1, 0, 0));
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b) IN ((?),(?)) and (c) IN ((?))", 0, 0, 1, 0),
-                       row(0, 0, 0, 0),
-                       row(0, 1, 0, 0));
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b) = (?) and (c) IN ((?), (?))", 0, 1, 0, 1),
-                       row(0, 1, 0, 0),
-                       row(0, 1, 1, 0),
-                       row(0, 1, 1, 1));
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b) = (?) and (c, d) IN ((?, ?))", 0, 1, 0, 0),
-                       row(0, 1, 0, 0));
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b) = (?) and (c, d) IN ((?, ?), (?, ?))", 0, 1, 0, 0, 1, 1),
-                       row(0, 1, 0, 0),
-                       row(0, 1, 1, 1));
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b) IN ((?), (?)) and (c, d) IN ((?, ?), (?, ?))", 0, 0, 1, 0, 0, 1, 1),
-                       row(0, 0, 0, 0),
-                       row(0, 0, 1, 1),
-                       row(0, 1, 0, 0),
-                       row(0, 1, 1, 1));
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b) = (?) and (c, d) > (?, ?)", 0, 1, 0, 0),
-                       row(0, 1, 1, 0),
-                       row(0, 1, 1, 1));
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b) IN ((?),(?)) and (c, d) > (?, ?)", 0, 0, 1, 0, 0),
-                       row(0, 0, 1, 0),
-                       row(0, 0, 1, 1),
-                       row(0, 1, 1, 0),
-                       row(0, 1, 1, 1));
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b) = (?) and (c, d) > (?, ?) and (c) <= (?) ", 0, 1, 0, 0, 1),
-                       row(0, 1, 1, 0),
-                       row(0, 1, 1, 1));
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b) = (?) and (c, d) >= (?, ?) and (c, d) < (?, ?)", 0, 1, 0, 0, 1, 1),
-                       row(0, 1, 0, 0),
-                       row(0, 1, 1, 0));
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b, c) = (?, ?) and d = ?", 0, 0, 1, 0),
-                       row(0, 0, 1, 0));
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b, c) IN ((?, ?), (?, ?)) and d = ?", 0, 0, 1, 0, 0, 0),
-                       row(0, 0, 0, 0),
-                       row(0, 0, 1, 0));
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and (d) < (1) and (b, c) = (?, ?) and (d) >= (?)", 0, 0, 1, 0),
-                       row(0, 0, 1, 0));
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and (d) < (1) and (b, c) IN ((?, ?), (?, ?)) and (d) >= (?)", 0, 0, 1, 0, 0, 0),
-                       row(0, 0, 0, 0),
-                       row(0, 0, 1, 0));
-        }
-    }
-
-    @Test
-    public void testSinglePartitionInvalidQueries() throws Throwable
-    {
-        for (String compactOption : new String[]{"", " WITH COMPACT STORAGE"})
-        {
-            createTable("CREATE TABLE %s (a int PRIMARY KEY, b int)" + compactOption);
-            assertInvalidMessage("Multi-column relations can only be applied to clustering columns but was applied to: a",
-                                 "SELECT * FROM %s WHERE (a) > (?)", 0);
-            assertInvalidMessage("Multi-column relations can only be applied to clustering columns but was applied to: a",
-                                 "SELECT * FROM %s WHERE (a) = (?)", 0);
-            assertInvalidMessage("Multi-column relations can only be applied to clustering columns but was applied to: b",
-                                 "SELECT * FROM %s WHERE (b) = (?)", 0);
-        }
-    }
-
-    @Test
-    public void testSingleClustering() throws Throwable
-    {
-        for (String compactOption : new String[]{"", " WITH COMPACT STORAGE"})
-        {
-            createTable("CREATE TABLE %s (a int, b int, c int, PRIMARY KEY (a, b))" + compactOption);
-
-            execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, 0, 0);
-            execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, 1, 0);
-            execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, 2, 0);
-
-            // Equalities
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) = (?)", 0, 1),
-                    row(0, 1, 0)
-            );
-
-            // Same but check the whole tuple can be prepared
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) = ?", 0, tuple(1)),
-                    row(0, 1, 0)
-            );
-
-            assertEmpty(execute("SELECT * FROM %s WHERE a = ? AND (b) = (?)", 0, 3));
-
-            // Inequalities
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) > (?)", 0, 0),
-                    row(0, 1, 0),
-                    row(0, 2, 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) >= (?)", 0, 1),
-                    row(0, 1, 0),
-                    row(0, 2, 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) < (?)", 0, 2),
-                    row(0, 0, 0),
-                    row(0, 1, 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) <= (?)", 0, 1),
-                    row(0, 0, 0),
-                    row(0, 1, 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) > (?) AND (b) < (?)", 0, 0, 2),
-                    row(0, 1, 0)
-            );
-        }
-    }
-
-    @Test
-    public void testNonEqualsRelation() throws Throwable
-    {
-        for (String compactOption : new String[]{"", " WITH COMPACT STORAGE"})
-        {
-            createTable("CREATE TABLE %s (a int PRIMARY KEY, b int)" + compactOption);
-            assertInvalidMessage("Unsupported \"!=\" relation: (b) != (0)",
-                    "SELECT * FROM %s WHERE a = 0 AND (b) != (0)");
-        }
-    }
-
-    @Test
-    public void testMultipleClustering() throws Throwable
-    {
-        for (String compactOption : new String[]{"", " WITH COMPACT STORAGE"})
-        {
-            createTable("CREATE TABLE %s (a int, b int, c int, d int, PRIMARY KEY (a, b, c, d))" + compactOption);
-
-            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 0, 0);
-            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 1, 0);
-            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 1, 1);
-
-            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 0, 0);
-            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 1, 0);
-            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 1, 1);
-
-            // Empty query
-            assertEmpty(execute("SELECT * FROM %s WHERE a = 0 AND (b, c, d) IN ()"));
-
-            // Equalities
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) = (?)", 0, 1),
-                    row(0, 1, 0, 0),
-                    row(0, 1, 1, 0),
-                    row(0, 1, 1, 1)
-            );
-
-            // Same with whole tuple prepared
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) = ?", 0, tuple(1)),
-                    row(0, 1, 0, 0),
-                    row(0, 1, 1, 0),
-                    row(0, 1, 1, 1)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c) = (?, ?)", 0, 1, 1),
-                    row(0, 1, 1, 0),
-                    row(0, 1, 1, 1)
-            );
-
-            // Same with whole tuple prepared
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c) = ?", 0, tuple(1, 1)),
-                    row(0, 1, 1, 0),
-                    row(0, 1, 1, 1)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) = (?, ?, ?)", 0, 1, 1, 1),
-                    row(0, 1, 1, 1)
-            );
-
-            // Same with whole tuple prepared
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) = ?", 0, tuple(1, 1, 1)),
-                    row(0, 1, 1, 1)
-            );
-
-            // Inequalities
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) > (?)", 0, 0),
-                    row(0, 1, 0, 0),
-                    row(0, 1, 1, 0),
-                    row(0, 1, 1, 1)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) >= (?)", 0, 0),
-                    row(0, 0, 0, 0),
-                    row(0, 0, 1, 0),
-                    row(0, 0, 1, 1),
-                    row(0, 1, 0, 0),
-                    row(0, 1, 1, 0),
-                    row(0, 1, 1, 1)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c) > (?, ?)", 0, 1, 0),
-                    row(0, 1, 1, 0),
-                    row(0, 1, 1, 1)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c) >= (?, ?)", 0, 1, 0),
-                    row(0, 1, 0, 0),
-                    row(0, 1, 1, 0),
-                    row(0, 1, 1, 1)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) > (?, ?, ?)", 0, 1, 1, 0),
-                    row(0, 1, 1, 1)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) >= (?, ?, ?)", 0, 1, 1, 0),
-                    row(0, 1, 1, 0),
-                    row(0, 1, 1, 1)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) < (?)", 0, 1),
-                    row(0, 0, 0, 0),
-                    row(0, 0, 1, 0),
-                    row(0, 0, 1, 1)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) <= (?)", 0, 1),
-                    row(0, 0, 0, 0),
-                    row(0, 0, 1, 0),
-                    row(0, 0, 1, 1),
-                    row(0, 1, 0, 0),
-                    row(0, 1, 1, 0),
-                    row(0, 1, 1, 1)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c) < (?, ?)", 0, 0, 1),
-                    row(0, 0, 0, 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c) <= (?, ?)", 0, 0, 1),
-                    row(0, 0, 0, 0),
-                    row(0, 0, 1, 0),
-                    row(0, 0, 1, 1)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) < (?, ?, ?)", 0, 0, 1, 1),
-                    row(0, 0, 0, 0),
-                    row(0, 0, 1, 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) <= (?, ?, ?)", 0, 0, 1, 1),
-                    row(0, 0, 0, 0),
-                    row(0, 0, 1, 0),
-                    row(0, 0, 1, 1)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) > (?, ?, ?) AND (b) < (?)", 0, 0, 1, 0, 1),
-                    row(0, 0, 1, 1)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) > (?, ?, ?) AND (b, c) < (?, ?)", 0, 0, 1, 1, 1, 1),
-                    row(0, 1, 0, 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) > (?, ?, ?) AND (b, c, d) < (?, ?, ?)", 0, 0, 1, 1, 1, 1, 0),
-                    row(0, 1, 0, 0)
-            );
-
-            // Same with whole tuple prepared
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) > ? AND (b, c, d) < ?", 0, tuple(0, 1, 1), tuple(1, 1, 0)),
-                    row(0, 1, 0, 0)
-            );
-
-            // reversed
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) > (?) ORDER BY b DESC, c DESC, d DESC", 0, 0),
-                    row(0, 1, 1, 1),
-                    row(0, 1, 1, 0),
-                    row(0, 1, 0, 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) >= (?) ORDER BY b DESC, c DESC, d DESC", 0, 0),
-                    row(0, 1, 1, 1),
-                    row(0, 1, 1, 0),
-                    row(0, 1, 0, 0),
-                    row(0, 0, 1, 1),
-                    row(0, 0, 1, 0),
-                    row(0, 0, 0, 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c) > (?, ?) ORDER BY b DESC, c DESC, d DESC", 0, 1, 0),
-                    row(0, 1, 1, 1),
-                    row(0, 1, 1, 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c) >= (?, ?) ORDER BY b DESC, c DESC, d DESC", 0, 1, 0),
-                    row(0, 1, 1, 1),
-                    row(0, 1, 1, 0),
-                    row(0, 1, 0, 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) > (?, ?, ?) ORDER BY b DESC, c DESC, d DESC", 0, 1, 1, 0),
-                    row(0, 1, 1, 1)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) >= (?, ?, ?) ORDER BY b DESC, c DESC, d DESC", 0, 1, 1, 0),
-                    row(0, 1, 1, 1),
-                    row(0, 1, 1, 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) < (?) ORDER BY b DESC, c DESC, d DESC", 0, 1),
-                    row(0, 0, 1, 1),
-                    row(0, 0, 1, 0),
-                    row(0, 0, 0, 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) <= (?) ORDER BY b DESC, c DESC, d DESC", 0, 1),
-                    row(0, 1, 1, 1),
-                    row(0, 1, 1, 0),
-                    row(0, 1, 0, 0),
-                    row(0, 0, 1, 1),
-                    row(0, 0, 1, 0),
-                    row(0, 0, 0, 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c) < (?, ?) ORDER BY b DESC, c DESC, d DESC", 0, 0, 1),
-                    row(0, 0, 0, 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c) <= (?, ?) ORDER BY b DESC, c DESC, d DESC", 0, 0, 1),
-                    row(0, 0, 1, 1),
-                    row(0, 0, 1, 0),
-                    row(0, 0, 0, 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) < (?, ?, ?) ORDER BY b DESC, c DESC, d DESC", 0, 0, 1, 1),
-                    row(0, 0, 1, 0),
-                    row(0, 0, 0, 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) <= (?, ?, ?) ORDER BY b DESC, c DESC, d DESC", 0, 0, 1, 1),
-                    row(0, 0, 1, 1),
-                    row(0, 0, 1, 0),
-                    row(0, 0, 0, 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) > (?, ?, ?) AND (b) < (?) ORDER BY b DESC, c DESC, d DESC", 0, 0, 1, 0, 1),
-                    row(0, 0, 1, 1)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) > (?, ?, ?) AND (b, c) < (?, ?) ORDER BY b DESC, c DESC, d DESC", 0, 0, 1, 1, 1, 1),
-                    row(0, 1, 0, 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) > (?, ?, ?) AND (b, c, d) < (?, ?, ?) ORDER BY b DESC, c DESC, d DESC", 0, 0, 1, 1, 1, 1, 0),
-                    row(0, 1, 0, 0)
-            );
-
-            // IN
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) IN ((?, ?, ?), (?, ?, ?))", 0, 0, 1, 0, 0, 1, 1),
-                    row(0, 0, 1, 0),
-                    row(0, 0, 1, 1)
-            );
-
-            // same query but with whole tuple prepared
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) IN (?, ?)", 0, tuple(0, 1, 0), tuple(0, 1, 1)),
-                    row(0, 0, 1, 0),
-                    row(0, 0, 1, 1)
-            );
-
-            // same query but with whole IN list prepared
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) IN ?", 0, list(tuple(0, 1, 0), tuple(0, 1, 1))),
-                    row(0, 0, 1, 0),
-                    row(0, 0, 1, 1)
-            );
-
-            // same query, but reversed order for the IN values
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) IN (?, ?)", 0, tuple(0, 1, 1), tuple(0, 1, 0)),
-                    row(0, 0, 1, 0),
-                    row(0, 0, 1, 1)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b, c) IN ((?, ?))", 0, 0, 1),
-                    row(0, 0, 1, 0),
-                    row(0, 0, 1, 1)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b) IN ((?))", 0, 0),
-                    row(0, 0, 0, 0),
-                    row(0, 0, 1, 0),
-                    row(0, 0, 1, 1)
-            );
-
-            assertEmpty(execute("SELECT * FROM %s WHERE a = ? and (b) IN ()", 0));
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c) IN ((?, ?)) ORDER BY b DESC, c DESC, d DESC", 0, 0, 1),
-                    row(0, 0, 1, 1),
-                    row(0, 0, 1, 0)
-            );
-
-            assertEmpty(execute("SELECT * FROM %s WHERE a = ? AND (b, c) IN () ORDER BY b DESC, c DESC, d DESC", 0));
-
-            // IN on both partition key and clustering key
-            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 1, 0, 0, 0);
-            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 1, 0, 1, 0);
-            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 1, 0, 1, 1);
-
-            assertRows(execute("SELECT * FROM %s WHERE a IN (?, ?) AND (b, c, d) IN (?, ?)", 0, 1, tuple(0, 1, 0), tuple(0, 1, 1)),
-                    row(0, 0, 1, 0),
-                    row(0, 0, 1, 1),
-                    row(1, 0, 1, 0),
-                    row(1, 0, 1, 1)
-            );
-
-            // same but with whole IN lists prepared
-            assertRows(execute("SELECT * FROM %s WHERE a IN ? AND (b, c, d) IN ?", list(0, 1), list(tuple(0, 1, 0), tuple(0, 1, 1))),
-                    row(0, 0, 1, 0),
-                    row(0, 0, 1, 1),
-                    row(1, 0, 1, 0),
-                    row(1, 0, 1, 1)
-            );
-
-            // same query, but reversed order for the IN values
-            assertRows(execute("SELECT * FROM %s WHERE a IN (?, ?) AND (b, c, d) IN (?, ?)", 1, 0, tuple(0, 1, 1), tuple(0, 1, 0)),
-                    row(0, 0, 1, 0),
-                    row(0, 0, 1, 1),
-                    row(1, 0, 1, 0),
-                    row(1, 0, 1, 1)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a IN (?, ?) and (b, c) IN ((?, ?))", 0, 1, 0, 1),
-                    row(0, 0, 1, 0),
-                    row(0, 0, 1, 1),
-                    row(1, 0, 1, 0),
-                    row(1, 0, 1, 1)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a IN (?, ?) and (b) IN ((?))", 0, 1, 0),
-                    row(0, 0, 0, 0),
-                    row(0, 0, 1, 0),
-                    row(0, 0, 1, 1),
-                    row(1, 0, 0, 0),
-                    row(1, 0, 1, 0),
-                    row(1, 0, 1, 1)
-            );
-        }
-    }
-
-    @Test
-    public void testMultipleClusteringReversedComponents() throws Throwable
-    {
-        for (String compactOption : new String[]{"", " COMPACT STORAGE AND"})
-        {
-            createTable("CREATE TABLE %s (a int, b int, c int, d int, PRIMARY KEY (a, b, c, d)) WITH" + compactOption + " CLUSTERING ORDER BY (b DESC, c ASC, d DESC)");
-
-            // b and d are reversed in the clustering order
-            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 0, 0);
-            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 1, 1);
-            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 1, 0);
-
-            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 0, 0);
-            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 1, 1);
-            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 1, 0);
-
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) > (?)", 0, 0),
-                    row(0, 1, 0, 0),
-                    row(0, 1, 1, 1),
-                    row(0, 1, 1, 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) >= (?)", 0, 0),
-                    row(0, 1, 0, 0),
-                    row(0, 1, 1, 1),
-                    row(0, 1, 1, 0),
-                    row(0, 0, 0, 0),
-                    row(0, 0, 1, 1),
-                    row(0, 0, 1, 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) < (?)", 0, 1),
-                    row(0, 0, 0, 0),
-                    row(0, 0, 1, 1),
-                    row(0, 0, 1, 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) <= (?)", 0, 1),
-                    row(0, 1, 0, 0),
-                    row(0, 1, 1, 1),
-                    row(0, 1, 1, 0),
-                    row(0, 0, 0, 0),
-                    row(0, 0, 1, 1),
-                    row(0, 0, 1, 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a=? AND (b, c, d) IN ((?, ?, ?), (?, ?, ?))", 0, 1, 1, 1, 0, 1, 1),
-                    row(0, 1, 1, 1),
-                    row(0, 0, 1, 1)
-            );
-
-            // same query, but reversed order for the IN values
-            assertRows(execute("SELECT * FROM %s WHERE a=? AND (b, c, d) IN ((?, ?, ?), (?, ?, ?))", 0, 0, 1, 1, 1, 1, 1),
-                    row(0, 1, 1, 1),
-                    row(0, 0, 1, 1)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) IN (?, ?, ?, ?, ?, ?)",
-                            0, tuple(1, 0, 0), tuple(1, 1, 1), tuple(1, 1, 0), tuple(0, 0, 0), tuple(0, 1, 1), tuple(0, 1, 0)),
-                    row(0, 1, 0, 0),
-                    row(0, 1, 1, 1),
-                    row(0, 1, 1, 0),
-                    row(0, 0, 0, 0),
-                    row(0, 0, 1, 1),
-                    row(0, 0, 1, 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c) IN (?)", 0, tuple(0, 1)),
-                    row(0, 0, 1, 1),
-                    row(0, 0, 1, 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c) IN (?)", 0, tuple(0, 0)),
-                    row(0, 0, 0, 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) IN ((?))", 0, 0),
-                    row(0, 0, 0, 0),
-                    row(0, 0, 1, 1),
-                    row(0, 0, 1, 0)
-            );
-
-            // preserve pre-6875 behavior (even though the query result is technically incorrect)
-            assertEmpty(execute("SELECT * FROM %s WHERE a = ? AND (b, c) > (?, ?)", 0, 1, 0));
-        }
-    }
-
-    @Test
-    public void testMultipleClusteringWithIndex() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a int, b int, c int, d int, e int, PRIMARY KEY (a, b, c, d))");
-        createIndex("CREATE INDEX ON %s (b)");
-        createIndex("CREATE INDEX ON %s (e)");
-        execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 0, 0, 0, 0);
-        execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 0, 1, 0, 1);
-        execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 0, 1, 1, 2);
-        execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 1, 0, 0, 0);
-        execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 1, 1, 0, 1);
-        execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 1, 1, 1, 2);
-        execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 2, 0, 0, 0);
-        assertRows(execute("SELECT * FROM %s WHERE (b) = (?)", 1),
-                   row(0, 1, 0, 0, 0),
-                   row(0, 1, 1, 0, 1),
-                   row(0, 1, 1, 1, 2));
-
-        assertInvalidMessage("Cannot execute this query as it might involve data filtering",
-                             "SELECT * FROM %s WHERE (b, c) = (?, ?)", 1, 1);
-        assertRows(execute("SELECT * FROM %s WHERE (b, c) = (?, ?) ALLOW FILTERING", 1, 1),
-                   row(0, 1, 1, 0, 1),
-                   row(0, 1, 1, 1, 2));
-
-        assertInvalidMessage("Cannot execute this query as it might involve data filtering",
-                             "SELECT * FROM %s WHERE (b, c) = (?, ?) AND e = ?", 1, 1, 2);
-        assertRows(execute("SELECT * FROM %s WHERE (b, c) = (?, ?) AND e = ? ALLOW FILTERING", 1, 1, 2),
-                   row(0, 1, 1, 1, 2));
-
-        assertInvalidMessage("Cannot execute this query as it might involve data filtering",
-                             "SELECT * FROM %s WHERE (b) IN ((?)) AND e = ?", 1, 2);
-        assertRows(execute("SELECT * FROM %s WHERE (b) IN ((?)) AND e = ? ALLOW FILTERING", 1, 2),
-                   row(0, 1, 1, 1, 2));
-
-        assertInvalidMessage("Cannot execute this query as it might involve data filtering",
-                             "SELECT * FROM %s WHERE (b) IN ((?), (?)) AND e = ?", 0, 1, 2);
-        assertRows(execute("SELECT * FROM %s WHERE (b) IN ((?), (?)) AND e = ? ALLOW FILTERING", 0, 1, 2),
-                   row(0, 0, 1, 1, 2),
-                   row(0, 1, 1, 1, 2));
-
-        assertInvalidMessage("Cannot execute this query as it might involve data filtering",
-                             "SELECT * FROM %s WHERE (b, c) IN ((?, ?)) AND e = ?", 0, 1, 2);
-        assertRows(execute("SELECT * FROM %s WHERE (b, c) IN ((?, ?)) AND e = ? ALLOW FILTERING", 0, 1, 2),
-                   row(0, 0, 1, 1, 2));
-
-        assertInvalidMessage("Cannot execute this query as it might involve data filtering",
-                             "SELECT * FROM %s WHERE (b, c) IN ((?, ?), (?, ?)) AND e = ?", 0, 1, 1, 1, 2);
-        assertRows(execute("SELECT * FROM %s WHERE (b, c) IN ((?, ?), (?, ?)) AND e = ? ALLOW FILTERING", 0, 1, 1, 1, 2),
-                   row(0, 0, 1, 1, 2),
-                   row(0, 1, 1, 1, 2));
-
-        assertInvalidMessage("Cannot execute this query as it might involve data filtering",
-                             "SELECT * FROM %s WHERE (b) >= (?) AND e = ?", 1, 2);
-        assertRows(execute("SELECT * FROM %s WHERE (b) >= (?) AND e = ? ALLOW FILTERING", 1, 2),
-                   row(0, 1, 1, 1, 2));
-
-        assertInvalidMessage("Cannot execute this query as it might involve data filtering",
-                             "SELECT * FROM %s WHERE (b, c) >= (?, ?) AND e = ?", 1, 1, 2);
-        assertRows(execute("SELECT * FROM %s WHERE (b, c) >= (?, ?) AND e = ? ALLOW FILTERING", 1, 1, 2),
-                   row(0, 1, 1, 1, 2));
-    }
-
-    @Test
-    public void testMultiplePartitionKeyAndMultiClusteringWithIndex() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a int, b int, c int, d int, e int, f int, PRIMARY KEY ((a, b), c, d, e))");
-        createIndex("CREATE INDEX ON %s (c)");
-        createIndex("CREATE INDEX ON %s (f)");
-
-        execute("INSERT INTO %s (a, b, c, d, e, f) VALUES (?, ?, ?, ?, ?, ?)", 0, 0, 0, 0, 0, 0);
-        execute("INSERT INTO %s (a, b, c, d, e, f) VALUES (?, ?, ?, ?, ?, ?)", 0, 0, 0, 1, 0, 1);
-        execute("INSERT INTO %s (a, b, c, d, e, f) VALUES (?, ?, ?, ?, ?, ?)", 0, 0, 0, 1, 1, 2);
-
-        execute("INSERT INTO %s (a, b, c, d, e, f) VALUES (?, ?, ?, ?, ?, ?)", 0, 0, 1, 0, 0, 3);
-        execute("INSERT INTO %s (a, b, c, d, e, f) VALUES (?, ?, ?, ?, ?, ?)", 0, 0, 1, 1, 0, 4);
-        execute("INSERT INTO %s (a, b, c, d, e, f) VALUES (?, ?, ?, ?, ?, ?)", 0, 0, 1, 1, 1, 5);
-
-        execute("INSERT INTO %s (a, b, c, d, e, f) VALUES (?, ?, ?, ?, ?, ?)", 0, 0, 2, 0, 0, 5);
-
-        assertInvalidMessage("Cannot execute this query as it might involve data filtering",
-                             "SELECT * FROM %s WHERE a = ? AND (c) = (?)");
-        assertRows(execute("SELECT * FROM %s WHERE a = ? AND (c) = (?) ALLOW FILTERING", 0, 1),
-                   row(0, 0, 1, 0, 0, 3),
-                   row(0, 0, 1, 1, 0, 4),
-                   row(0, 0, 1, 1, 1, 5));
-
-        assertInvalidMessage("Cannot execute this query as it might involve data filtering",
-                             "SELECT * FROM %s WHERE a = ? AND (c, d) = (?, ?)", 0, 1, 1);
-        assertRows(execute("SELECT * FROM %s WHERE a = ? AND (c, d) = (?, ?) ALLOW FILTERING", 0, 1, 1),
-                   row(0, 0, 1, 1, 0, 4),
-                   row(0, 0, 1, 1, 1, 5));
-
-        assertInvalidMessage("Partition key parts: b must be restricted as other parts are",
-                             "SELECT * FROM %s WHERE a = ? AND (c, d) IN ((?, ?)) ALLOW FILTERING", 0, 1, 1);
-
-        assertInvalidMessage("Partition key parts: b must be restricted as other parts are",
-                             "SELECT * FROM %s WHERE a = ? AND (c, d) >= (?, ?) ALLOW FILTERING", 0, 1, 1);
-
-        assertInvalidMessage("Cannot execute this query as it might involve data filtering",
-                             "SELECT * FROM %s WHERE a = ? AND (c) IN ((?)) AND f = ?", 0, 1, 5);
-        assertRows(execute("SELECT * FROM %s WHERE a = ? AND (c) IN ((?)) AND f = ? ALLOW FILTERING", 0, 1, 5),
-                   row(0, 0, 1, 1, 1, 5));
-
-        assertInvalidMessage("Cannot execute this query as it might involve data filtering",
-                             "SELECT * FROM %s WHERE a = ? AND (c) IN ((?), (?)) AND f = ?", 0, 1, 2, 5);
-        assertRows(execute("SELECT * FROM %s WHERE a = ? AND (c) IN ((?), (?)) AND f = ? ALLOW FILTERING", 0, 1, 2, 5),
-                   row(0, 0, 1, 1, 1, 5),
-                   row(0, 0, 2, 0, 0, 5));
-
-        assertInvalidMessage("Cannot execute this query as it might involve data filtering",
-                             "SELECT * FROM %s WHERE a = ? AND (c, d) IN ((?, ?)) AND f = ?", 0, 1, 0, 3);
-        assertRows(execute("SELECT * FROM %s WHERE a = ? AND (c, d) IN ((?, ?)) AND f = ? ALLOW FILTERING", 0, 1, 0, 3),
-                   row(0, 0, 1, 0, 0, 3));
-
-        assertInvalidMessage("Cannot execute this query as it might involve data filtering",
-                             "SELECT * FROM %s WHERE a = ? AND (c) >= (?) AND f = ?", 0, 1, 5);
-        assertRows(execute("SELECT * FROM %s WHERE a = ? AND (c) >= (?) AND f = ? ALLOW FILTERING", 0, 1, 5),
-                   row(0, 0, 1, 1, 1, 5),
-                   row(0, 0, 2, 0, 0, 5));
-
-        assertInvalidMessage("Cannot execute this query as it might involve data filtering",
-                             "SELECT * FROM %s WHERE a = ? AND (c, d) >= (?, ?) AND f = ?", 0, 1, 1, 5);
-        assertRows(execute("SELECT * FROM %s WHERE a = ? AND (c, d) >= (?, ?) AND f = ? ALLOW FILTERING", 0, 1, 1, 5),
-                   row(0, 0, 1, 1, 1, 5),
-                   row(0, 0, 2, 0, 0, 5));
-    }
-
-    @Test
-    public void testINWithDuplicateValue() throws Throwable
-    {
-        for (String compactOption : new String[] { "", " WITH COMPACT STORAGE" })
-        {
-            createTable("CREATE TABLE %s (k1 int, k2 int, v int, PRIMARY KEY (k1, k2))" + compactOption);
-            execute("INSERT INTO %s (k1,  k2, v) VALUES (?, ?, ?)", 1, 1, 1);
-
-            assertRows(execute("SELECT * FROM %s WHERE k1 IN (?, ?) AND (k2) IN ((?), (?))", 1, 1, 1, 2),
-                       row(1, 1, 1));
-            assertRows(execute("SELECT * FROM %s WHERE k1 = ? AND (k2) IN ((?), (?))", 1, 1, 1),
-                       row(1, 1, 1));
-        }
-    }
-
-    @Test
-    public void testWithUnsetValues() throws Throwable
-    {
-        createTable("CREATE TABLE %s (k int, i int, j int, s text, PRIMARY KEY(k,i,j))");
-        createIndex("CREATE INDEX s_index ON %s (s)");
-
-        assertInvalidMessage("Invalid unset value for tuple field number 0",
-                             "SELECT * from %s WHERE (i, j) = (?,?) ALLOW FILTERING", unset(), 1);
-        assertInvalidMessage("Invalid unset value for tuple field number 0",
-                             "SELECT * from %s WHERE (i, j) IN ((?,?)) ALLOW FILTERING", unset(), 1);
-        assertInvalidMessage("Invalid unset value for tuple field number 1",
-                             "SELECT * from %s WHERE (i, j) > (1,?) ALLOW FILTERING", unset());
-        assertInvalidMessage("Invalid unset value for tuple (i,j)",
-                             "SELECT * from %s WHERE (i, j) = ? ALLOW FILTERING", unset());
-        assertInvalidMessage("Invalid unset value for tuple (j)",
-                             "SELECT * from %s WHERE i = ? AND (j) > ? ALLOW FILTERING", 1, unset());
-        assertInvalidMessage("Invalid unset value for tuple (i,j)",
-                             "SELECT * from %s WHERE (i, j) IN (?, ?) ALLOW FILTERING", unset(), tuple(1, 1));
-        assertInvalidMessage("Invalid unset value for in(i,j)",
-                             "SELECT * from %s WHERE (i, j) IN ? ALLOW FILTERING", unset());
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/01115f72/test/unit/org/apache/cassandra/cql3/PgStringTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/PgStringTest.java b/test/unit/org/apache/cassandra/cql3/PgStringTest.java
deleted file mode 100644
index 0a9d702..0000000
--- a/test/unit/org/apache/cassandra/cql3/PgStringTest.java
+++ /dev/null
@@ -1,76 +0,0 @@
-/*
- * 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;
-
-import org.junit.Test;
-
-import org.apache.cassandra.exceptions.SyntaxException;
-
-public class PgStringTest extends CQLTester
-{
-    @Test
-    public void testPgSyleFunction() throws Throwable
-    {
-        execute("create or replace function "+KEYSPACE+".pgfun1 ( input double ) called on null input returns text language java\n" +
-                "AS $$return \"foobar\";$$");
-    }
-
-    @Test
-    public void testPgSyleInsert() throws Throwable
-    {
-        createTable("CREATE TABLE %s (key ascii primary key, val text)");
-
-        // some non-terminated pg-strings
-        assertInvalidSyntax("INSERT INTO %s (key, val) VALUES ($ $key_empty$$, $$'' value for empty$$)");
-        assertInvalidSyntax("INSERT INTO %s (key, val) VALUES ($$key_empty$$, $$'' value for empty$ $)");
-        assertInvalidSyntax("INSERT INTO %s (key, val) VALUES ($$key_empty$ $, $$'' value for empty$$)");
-
-        // different pg-style markers for multiple strings
-        execute("INSERT INTO %s (key, val) VALUES ($$prim$ $ $key$$, $$some '' arbitrary value$$)");
-        // same empty pg-style marker for multiple strings
-        execute("INSERT INTO %s (key, val) VALUES ($$key_empty$$, $$'' value for empty$$)");
-        // stange but valid pg-style
-        execute("INSERT INTO %s (key, val) VALUES ($$$foo$_$foo$$, $$$'' value for empty$$)");
-        // these are conventional quoted strings
-        execute("INSERT INTO %s (key, val) VALUES ('$txt$key$$$$txt$', '$txt$'' other value$txt$')");
-
-        assertRows(execute("SELECT key, val FROM %s WHERE key='prim$ $ $key'"),
-                   row("prim$ $ $key", "some '' arbitrary value")
-        );
-        assertRows(execute("SELECT key, val FROM %s WHERE key='key_empty'"),
-                   row("key_empty", "'' value for empty")
-        );
-        assertRows(execute("SELECT key, val FROM %s WHERE key='$foo$_$foo'"),
-                   row("$foo$_$foo", "$'' value for empty")
-        );
-        assertRows(execute("SELECT key, val FROM %s WHERE key='$txt$key$$$$txt$'"),
-                   row("$txt$key$$$$txt$", "$txt$' other value$txt$")
-        );
-
-        // invalid syntax
-        assertInvalidSyntax("INSERT INTO %s (key, val) VALUES ($ascii$prim$$$key$invterm$, $txt$some '' arbitrary value$txt$)");
-    }
-
-    @Test(expected = SyntaxException.class)
-    public void testMarkerPgFail() throws Throwable
-    {
-        // must throw SyntaxException - not StringIndexOutOfBoundsException or similar
-        execute("create function "+KEYSPACE+".pgfun1 ( input double ) called on null input returns bigint language java\n" +
-                "AS $javasrc$return 0L;$javasrc$;");
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/01115f72/test/unit/org/apache/cassandra/cql3/RangeDeletionTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/RangeDeletionTest.java b/test/unit/org/apache/cassandra/cql3/RangeDeletionTest.java
deleted file mode 100644
index b31d0c2..0000000
--- a/test/unit/org/apache/cassandra/cql3/RangeDeletionTest.java
+++ /dev/null
@@ -1,35 +0,0 @@
-/*
- * 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;
-
-import org.junit.Test;
-
-public class RangeDeletionTest extends CQLTester
-{
-    @Test
-    public void testCassandra8558() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a int, b int, c int, d int, PRIMARY KEY (a, b, c))");
-
-        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 1, 1, 1, 1);
-        flush();
-        execute("DELETE FROM %s WHERE a=? AND b=?", 1, 1);
-        flush();
-        assertEmpty(execute("SELECT * FROM %s WHERE a=? AND b=? AND c=?", 1, 1, 1));
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/01115f72/test/unit/org/apache/cassandra/cql3/RoleSyntaxTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/RoleSyntaxTest.java b/test/unit/org/apache/cassandra/cql3/RoleSyntaxTest.java
deleted file mode 100644
index 02bfe61..0000000
--- a/test/unit/org/apache/cassandra/cql3/RoleSyntaxTest.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * 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;
-
-import org.junit.Test;
-
-public class RoleSyntaxTest extends CQLTester
-{
-    @Test
-    public void standardOptionsSyntaxTest() throws Throwable
-    {
-        assertValidSyntax("CREATE ROLE r WITH LOGIN = true AND SUPERUSER = false AND PASSWORD = 'foo'");
-        assertValidSyntax("CREATE ROLE r WITH PASSWORD = 'foo' AND LOGIN = true AND SUPERUSER = false");
-        assertValidSyntax("CREATE ROLE r WITH SUPERUSER = true AND PASSWORD = 'foo' AND LOGIN = false");
-        assertValidSyntax("CREATE ROLE r WITH LOGIN = true AND PASSWORD = 'foo' AND SUPERUSER = false");
-        assertValidSyntax("CREATE ROLE r WITH SUPERUSER = true AND PASSWORD = 'foo' AND LOGIN = false");
-
-        assertValidSyntax("ALTER ROLE r WITH LOGIN = true AND SUPERUSER = false AND PASSWORD = 'foo'");
-        assertValidSyntax("ALTER ROLE r WITH PASSWORD = 'foo' AND LOGIN = true AND SUPERUSER = false");
-        assertValidSyntax("ALTER ROLE r WITH SUPERUSER = true AND PASSWORD = 'foo' AND LOGIN = false");
-        assertValidSyntax("ALTER ROLE r WITH LOGIN = true AND PASSWORD = 'foo' AND SUPERUSER = false");
-        assertValidSyntax("ALTER ROLE r WITH SUPERUSER = true AND PASSWORD = 'foo' AND LOGIN = false");
-    }
-
-    @Test
-    public void customOptionsSyntaxTestl() throws Throwable
-    {
-        assertValidSyntax("CREATE ROLE r WITH OPTIONS = {'a':'b', 'b':1}");
-        assertInvalidSyntax("CREATE ROLE r WITH OPTIONS = 'term'");
-        assertInvalidSyntax("CREATE ROLE r WITH OPTIONS = 99");
-
-        assertValidSyntax("ALTER ROLE r WITH OPTIONS = {'a':'b', 'b':1}");
-        assertInvalidSyntax("ALTER ROLE r WITH OPTIONS = 'term'");
-        assertInvalidSyntax("ALTER ROLE r WITH OPTIONS = 99");
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/01115f72/test/unit/org/apache/cassandra/cql3/SSTableMetadataTrackingTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/SSTableMetadataTrackingTest.java b/test/unit/org/apache/cassandra/cql3/SSTableMetadataTrackingTest.java
deleted file mode 100644
index 7c3965f..0000000
--- a/test/unit/org/apache/cassandra/cql3/SSTableMetadataTrackingTest.java
+++ /dev/null
@@ -1,160 +0,0 @@
-/*
- * 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;
-
-import org.junit.Test;
-
-import org.apache.cassandra.db.ColumnFamilyStore;
-import org.apache.cassandra.db.Keyspace;
-import org.apache.cassandra.io.sstable.metadata.StatsMetadata;
-import static org.junit.Assert.assertEquals;
-
-public class SSTableMetadataTrackingTest extends CQLTester
-{
-    @Test
-    public void baseCheck() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a int, b int, c text, PRIMARY KEY (a, b))");
-        ColumnFamilyStore cfs = Keyspace.open(keyspace()).getColumnFamilyStore(currentTable());
-        execute("INSERT INTO %s (a,b,c) VALUES (1,1,'1') using timestamp 9999");
-        cfs.forceBlockingFlush();
-        StatsMetadata metadata = cfs.getSSTables().iterator().next().getSSTableMetadata();
-        assertEquals(9999, metadata.minTimestamp);
-        assertEquals(Integer.MAX_VALUE, metadata.maxLocalDeletionTime);
-        cfs.forceMajorCompaction();
-        metadata = cfs.getSSTables().iterator().next().getSSTableMetadata();
-        assertEquals(9999, metadata.minTimestamp);
-        assertEquals(Integer.MAX_VALUE, metadata.maxLocalDeletionTime);
-    }
-
-    @Test
-    public void testMinMaxtimestampRange() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a int, b int, c text, PRIMARY KEY (a, b))");
-        ColumnFamilyStore cfs = Keyspace.open(keyspace()).getColumnFamilyStore(currentTable());
-        execute("INSERT INTO %s (a,b,c) VALUES (1,1,'1') using timestamp 10000");
-        execute("DELETE FROM %s USING TIMESTAMP 9999 WHERE a = 1 and b = 1");
-        cfs.forceBlockingFlush();
-        StatsMetadata metadata = cfs.getSSTables().iterator().next().getSSTableMetadata();
-        assertEquals(9999, metadata.minTimestamp);
-        assertEquals(10000, metadata.maxTimestamp);
-        assertEquals(Integer.MAX_VALUE, metadata.maxLocalDeletionTime, 5);
-        cfs.forceMajorCompaction();
-        metadata = cfs.getSSTables().iterator().next().getSSTableMetadata();
-        assertEquals(9999, metadata.minTimestamp);
-        assertEquals(10000, metadata.maxTimestamp);
-        assertEquals(Integer.MAX_VALUE, metadata.maxLocalDeletionTime, 5);
-    }
-
-    @Test
-    public void testMinMaxtimestampRow() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a int, b int, c text, PRIMARY KEY (a, b))");
-        ColumnFamilyStore cfs = Keyspace.open(keyspace()).getColumnFamilyStore(currentTable());
-        execute("INSERT INTO %s (a,b,c) VALUES (1,1,'1') using timestamp 10000");
-        execute("DELETE FROM %s USING TIMESTAMP 9999 WHERE a = 1");
-        cfs.forceBlockingFlush();
-        StatsMetadata metadata = cfs.getSSTables().iterator().next().getSSTableMetadata();
-        assertEquals(9999, metadata.minTimestamp);
-        assertEquals(10000, metadata.maxTimestamp);
-        assertEquals(Integer.MAX_VALUE, metadata.maxLocalDeletionTime, 5);
-        cfs.forceMajorCompaction();
-        metadata = cfs.getSSTables().iterator().next().getSSTableMetadata();
-        assertEquals(9999, metadata.minTimestamp);
-        assertEquals(10000, metadata.maxTimestamp);
-        assertEquals(Integer.MAX_VALUE, metadata.maxLocalDeletionTime, 5);
-    }
-
-
-    @Test
-    public void testTrackMetadata_rangeTombstone() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a int, b int, c text, PRIMARY KEY (a, b)) WITH gc_grace_seconds = 10000");
-        ColumnFamilyStore cfs = Keyspace.open(keyspace()).getColumnFamilyStore(currentTable());
-        execute("DELETE FROM %s USING TIMESTAMP 9999 WHERE a = 1 and b = 1");
-        cfs.forceBlockingFlush();
-        assertEquals(1, cfs.getSSTables().size());
-        StatsMetadata metadata = cfs.getSSTables().iterator().next().getSSTableMetadata();
-        assertEquals(9999, metadata.minTimestamp);
-        assertEquals(9999, metadata.maxTimestamp);
-        assertEquals(System.currentTimeMillis()/1000, metadata.maxLocalDeletionTime, 5);
-        cfs.forceMajorCompaction();
-        StatsMetadata metadata2 = cfs.getSSTables().iterator().next().getSSTableMetadata();
-        assertEquals(metadata.maxLocalDeletionTime, metadata2.maxLocalDeletionTime);
-        assertEquals(metadata.minTimestamp, metadata2.minTimestamp);
-        assertEquals(metadata.maxTimestamp, metadata2.maxTimestamp);
-    }
-
-    @Test
-    public void testTrackMetadata_rowTombstone() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a int, b int, c text, PRIMARY KEY (a, b))");
-        ColumnFamilyStore cfs = Keyspace.open(keyspace()).getColumnFamilyStore(currentTable());
-        execute("DELETE FROM %s USING TIMESTAMP 9999 WHERE a = 1");
-
-        cfs.forceBlockingFlush();
-        assertEquals(1, cfs.getSSTables().size());
-        StatsMetadata metadata = cfs.getSSTables().iterator().next().getSSTableMetadata();
-        assertEquals(9999, metadata.minTimestamp);
-        assertEquals(9999, metadata.maxTimestamp);
-        assertEquals(System.currentTimeMillis()/1000, metadata.maxLocalDeletionTime, 5);
-        cfs.forceMajorCompaction();
-        StatsMetadata metadata2 = cfs.getSSTables().iterator().next().getSSTableMetadata();
-        assertEquals(metadata.maxLocalDeletionTime, metadata2.maxLocalDeletionTime);
-        assertEquals(metadata.minTimestamp, metadata2.minTimestamp);
-        assertEquals(metadata.maxTimestamp, metadata2.maxTimestamp);
-    }
-
-    @Test
-    public void testTrackMetadata_rowMarker() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a int, PRIMARY KEY (a))");
-        ColumnFamilyStore cfs = Keyspace.open(keyspace()).getColumnFamilyStore(currentTable());
-        execute("INSERT INTO %s (a) VALUES (1) USING TIMESTAMP 9999");
-
-        cfs.forceBlockingFlush();
-        assertEquals(1, cfs.getSSTables().size());
-        StatsMetadata metadata = cfs.getSSTables().iterator().next().getSSTableMetadata();
-        assertEquals(9999, metadata.minTimestamp);
-        assertEquals(9999, metadata.maxTimestamp);
-        assertEquals(Integer.MAX_VALUE, metadata.maxLocalDeletionTime);
-        cfs.forceMajorCompaction();
-        StatsMetadata metadata2 = cfs.getSSTables().iterator().next().getSSTableMetadata();
-        assertEquals(metadata.maxLocalDeletionTime, metadata2.maxLocalDeletionTime);
-        assertEquals(metadata.minTimestamp, metadata2.minTimestamp);
-        assertEquals(metadata.maxTimestamp, metadata2.maxTimestamp);
-    }
-    @Test
-    public void testTrackMetadata_rowMarkerDelete() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a int, PRIMARY KEY (a))");
-        ColumnFamilyStore cfs = Keyspace.open(keyspace()).getColumnFamilyStore(currentTable());
-        execute("DELETE FROM %s USING TIMESTAMP 9999 WHERE a=1");
-        cfs.forceBlockingFlush();
-        assertEquals(1, cfs.getSSTables().size());
-        StatsMetadata metadata = cfs.getSSTables().iterator().next().getSSTableMetadata();
-        assertEquals(9999, metadata.minTimestamp);
-        assertEquals(9999, metadata.maxTimestamp);
-        assertEquals(System.currentTimeMillis()/1000, metadata.maxLocalDeletionTime, 5);
-        cfs.forceMajorCompaction();
-        StatsMetadata metadata2 = cfs.getSSTables().iterator().next().getSSTableMetadata();
-        assertEquals(metadata.maxLocalDeletionTime, metadata2.maxLocalDeletionTime);
-        assertEquals(metadata.minTimestamp, metadata2.minTimestamp);
-        assertEquals(metadata.maxTimestamp, metadata2.maxTimestamp);
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/01115f72/test/unit/org/apache/cassandra/cql3/SecondaryIndexOnMapEntriesTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/SecondaryIndexOnMapEntriesTest.java b/test/unit/org/apache/cassandra/cql3/SecondaryIndexOnMapEntriesTest.java
deleted file mode 100644
index e502f6a..0000000
--- a/test/unit/org/apache/cassandra/cql3/SecondaryIndexOnMapEntriesTest.java
+++ /dev/null
@@ -1,337 +0,0 @@
-/*
- * 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;
-
-import org.apache.cassandra.exceptions.InvalidRequestException;
-import org.apache.commons.lang3.StringUtils;
-import org.junit.Test;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-public class SecondaryIndexOnMapEntriesTest extends CQLTester
-{
-    @Test
-    public void testShouldNotCreateIndexOnFrozenMaps() throws Throwable
-    {
-        createTable("CREATE TABLE %s (k TEXT PRIMARY KEY, v FROZEN<MAP<TEXT, TEXT>>)");
-        assertIndexInvalidForColumn("v");
-    }
-
-    @Test
-    public void testShouldNotCreateIndexOnNonMapTypes() throws Throwable
-    {
-        createTable("CREATE TABLE %s (k TEXT PRIMARY KEY, i INT, t TEXT, b BLOB, s SET<TEXT>, l LIST<TEXT>, tu TUPLE<TEXT>)");
-        assertIndexInvalidForColumn("i");
-        assertIndexInvalidForColumn("t");
-        assertIndexInvalidForColumn("b");
-        assertIndexInvalidForColumn("s");
-        assertIndexInvalidForColumn("l");
-        assertIndexInvalidForColumn("tu");
-    }
-
-    @Test
-    public void testShouldValidateMapKeyAndValueTypes() throws Throwable
-    {
-        createSimpleTableAndIndex();
-
-        String query = "SELECT * FROM %s WHERE v[?] = ?";
-        Object validKey = "valid key";
-        Object invalidKey = 31415;
-        Object validValue = 31415;
-        Object invalidValue = "invalid value";
-        assertInvalid(query, invalidKey, invalidValue);
-        assertInvalid(query, invalidKey, validValue);
-        assertInvalid(query, validKey, invalidValue);
-        assertReturnsNoRows(query, validKey, validValue);
-    }
-
-    @Test
-    public void testShouldFindRowsMatchingSingleEqualityRestriction() throws Throwable
-    {
-        createSimpleTableAndIndex();
-        Object[] foo = insertIntoSimpleTable("foo", map("a", 1,
-                                                        "c", 3));
-        Object[] bar = insertIntoSimpleTable("bar", map("a", 1,
-                                                        "b", 2));
-        Object[] baz = insertIntoSimpleTable("baz", map("b", 2,
-                                                        "c", 5,
-                                                        "d", 4));
-        Object[] qux = insertIntoSimpleTable("qux", map("b", 2,
-                                                        "d", 4));
-
-        assertRowsForConditions(entry("a", 1), bar, foo);
-        assertRowsForConditions(entry("b", 2), bar, baz, qux);
-        assertRowsForConditions(entry("c", 3), foo);
-        assertRowsForConditions(entry("c", 5), baz);
-        assertRowsForConditions(entry("d", 4), baz, qux);
-    }
-
-    @Test
-    public void testRequireFilteringDirectiveIfMultipleRestrictionsSpecified() throws Throwable
-    {
-        createSimpleTableAndIndex();
-        String baseQuery = "SELECT * FROM %s WHERE v['foo'] = 31415 AND v['baz'] = 31416";
-        assertInvalid(baseQuery);
-        assertReturnsNoRows(baseQuery + " ALLOW FILTERING");
-    }
-
-    @Test
-    public void testShouldFindRowsMatchingMultipleEqualityRestrictions() throws Throwable
-    {
-        createSimpleTableAndIndex();
-
-        Object[] foo = insertIntoSimpleTable("foo", map("k1", 1));
-        Object[] bar = insertIntoSimpleTable("bar", map("k1", 1,
-                                                        "k2", 2));
-        Object[] baz = insertIntoSimpleTable("baz", map("k2", 2,
-                                                        "k3", 3));
-        Object[] qux = insertIntoSimpleTable("qux", map("k2", 2,
-                                                        "k3", 3,
-                                                        "k4", 4));
-
-        assertRowsForConditions(entry("k1", 1),
-                                bar, foo);
-        assertRowsForConditions(entry("k1", 1).entry("k2", 2),
-                                bar);
-        assertNoRowsForConditions(entry("k1", 1).entry("k2", 2).entry("k3", 3));
-        assertRowsForConditions(entry("k2", 2).entry("k3", 3),
-                                baz, qux);
-        assertRowsForConditions(entry("k2", 2).entry("k3", 3).entry("k4", 4),
-                                qux);
-        assertRowsForConditions(entry("k3", 3).entry("k4", 4),
-                                qux);
-        assertNoRowsForConditions(entry("k3", 3).entry("k4", 4).entry("k5", 5));
-    }
-
-    @Test
-    public void testShouldFindRowsMatchingEqualityAndContainsRestrictions() throws Throwable
-    {
-        createSimpleTableAndIndex();
-
-        Object[] foo = insertIntoSimpleTable("foo", map("common", 31415,
-                                                        "k1", 1,
-                                                        "k2", 2,
-                                                        "k3", 3));
-        Object[] bar = insertIntoSimpleTable("bar", map("common", 31415,
-                                                        "k3", 3,
-                                                        "k4", 4,
-                                                        "k5", 5));
-        Object[] baz = insertIntoSimpleTable("baz", map("common", 31415,
-                                                        "k5", 5,
-                                                        "k6", 6,
-                                                        "k7", 7));
-
-        assertRowsForConditions(entry("common", 31415),
-                                bar, baz, foo);
-        assertRowsForConditions(entry("common", 31415).key("k1"),
-                                foo);
-        assertRowsForConditions(entry("common", 31415).key("k2"),
-                                foo);
-        assertRowsForConditions(entry("common", 31415).key("k3"),
-                                bar, foo);
-        assertRowsForConditions(entry("common", 31415).key("k3").value(2),
-                                foo);
-        assertRowsForConditions(entry("common", 31415).key("k3").value(3),
-                                bar, foo);
-        assertRowsForConditions(entry("common", 31415).key("k3").value(4),
-                                bar);
-        assertRowsForConditions(entry("common", 31415).key("k3").key("k5"),
-                                bar);
-        assertRowsForConditions(entry("common", 31415).key("k5"),
-                                bar, baz);
-        assertRowsForConditions(entry("common", 31415).key("k5").value(4),
-                                bar);
-        assertRowsForConditions(entry("common", 31415).key("k5").value(5),
-                                bar, baz);
-        assertRowsForConditions(entry("common", 31415).key("k5").value(6),
-                                baz);
-        assertNoRowsForConditions(entry("common", 31415).key("k5").value(8));
-    }
-
-    @Test
-    public void testShouldNotAcceptUnsupportedRelationsOnEntries() throws Throwable
-    {
-        createSimpleTableAndIndex();
-        assertInvalidRelation("< 31415");
-        assertInvalidRelation("<= 31415");
-        assertInvalidRelation("> 31415");
-        assertInvalidRelation(">= 31415");
-        assertInvalidRelation("IN (31415, 31416, 31417)");
-        assertInvalidRelation("CONTAINS 31415");
-        assertInvalidRelation("CONTAINS KEY 'foo'");
-    }
-
-    @Test
-    public void testShouldRecognizeAlteredOrDeletedMapEntries() throws Throwable
-    {
-        createSimpleTableAndIndex();
-        Object[] foo = insertIntoSimpleTable("foo", map("common", 31415,
-                                                        "target", 8192));
-        Object[] bar = insertIntoSimpleTable("bar", map("common", 31415,
-                                                        "target", 8192));
-        Object[] baz = insertIntoSimpleTable("baz", map("common", 31415,
-                                                        "target", 8192));
-
-        assertRowsForConditions(entry("target", 8192),
-                                bar, baz, foo);
-        baz = updateMapInSimpleTable(baz, "target", 4096);
-        assertRowsForConditions(entry("target", 8192),
-                                bar, foo);
-        bar = updateMapInSimpleTable(bar, "target", null);
-        assertRowsForConditions(entry("target", 8192),
-                                foo);
-        execute("DELETE FROM %s WHERE k = 'foo'");
-        assertNoRowsForConditions(entry("target", 8192));
-        assertRowsForConditions(entry("common", 31415),
-                                bar, baz);
-        assertRowsForConditions(entry("target", 4096),
-                                baz);
-    }
-
-    @Test
-    public void testShouldRejectQueriesForNullEntries() throws Throwable
-    {
-        createSimpleTableAndIndex();
-        assertInvalid("SELECT * FROM %s WHERE v['somekey'] = null");
-    }
-
-    @Test
-    public void testShouldTreatQueriesAgainstFrozenMapIndexesAsInvalid() throws Throwable
-    {
-        createTable("CREATE TABLE %s (k TEXT PRIMARY KEY, v FROZEN<MAP<TEXT, TEXT>>)");
-        createIndex("CREATE INDEX ON %s(FULL(V))");
-
-        try
-        {
-            execute("SELECT * FROM %s WHERE v['somekey'] = 'somevalue'");
-            fail("Expected index query to fail");
-        }
-        catch (InvalidRequestException e)
-        {
-            String expectedMessage = "Map-entry equality predicates on frozen map column v are not supported";
-            assertTrue("Expected error message to contain '" + expectedMessage + "' but got '" +
-                       e.getMessage() + "'", e.getMessage().contains(expectedMessage));
-        }
-    }
-
-    private void assertIndexInvalidForColumn(String colname) throws Throwable
-    {
-        String query = String.format("CREATE INDEX ON %%s(ENTRIES(%s))", colname);
-        assertInvalid(query);
-    }
-
-    private void assertReturnsNoRows(String query, Object... params) throws Throwable
-    {
-        assertRows(execute(query, params));
-    }
-
-    private void createSimpleTableAndIndex() throws Throwable
-    {
-        createTable("CREATE TABLE %s (k TEXT PRIMARY KEY, v MAP<TEXT, INT>)");
-        createIndex("CREATE INDEX ON %s(ENTRIES(v))");
-    }
-
-    private Object[] insertIntoSimpleTable(String key, Object value) throws Throwable
-    {
-        String query = "INSERT INTO %s (k, v) VALUES (?, ?)";
-        execute(query, key, value);
-        return row(key, value);
-    }
-
-    private void assertRowsForConditions(IndexWhereClause whereClause, Object[]... rows) throws Throwable
-    {
-        assertRows(execute("SELECT * FROM %s WHERE " + whereClause.text(), whereClause.params()), rows);
-    }
-
-    private void assertNoRowsForConditions(IndexWhereClause whereClause) throws Throwable
-    {
-        assertRowsForConditions(whereClause);
-    }
-
-    private void assertInvalidRelation(String rel) throws Throwable
-    {
-        String query = "SELECT * FROM %s WHERE v " + rel;
-        assertInvalid(query);
-    }
-
-    private Object[] updateMapInSimpleTable(Object[] row, String mapKey, Integer mapValue) throws Throwable
-    {
-        execute("UPDATE %s SET v[?] = ? WHERE k = ?", mapKey, mapValue, row[0]);
-        UntypedResultSet rawResults = execute("SELECT * FROM %s WHERE k = ?", row[0]);
-        Map<Object, Object> value = (Map<Object, Object>)row[1];
-        if (mapValue == null)
-        {
-            value.remove(mapKey);
-        }
-        else
-        {
-            value.put(mapKey, mapValue);
-        }
-        return row;
-    }
-
-    private IndexWhereClause entry(Object key, Object value)
-    {
-        return (new IndexWhereClause()).entry(key, value);
-    }
-
-    private static final class IndexWhereClause
-    {
-        private final List<String> preds = new ArrayList<>();
-        private final List<Object> params = new ArrayList<>();
-
-        public IndexWhereClause entry(Object key, Object value)
-        {
-            preds.add("v[?] = ?");
-            params.add(key);
-            params.add(value);
-            return this;
-        }
-
-        public IndexWhereClause key(Object key)
-        {
-            preds.add("v CONTAINS KEY ?");
-            params.add(key);
-            return this;
-        }
-
-        public IndexWhereClause value(Object value)
-        {
-            preds.add("v CONTAINS ?");
-            params.add(value);
-            return this;
-        }
-
-        public String text()
-        {
-            if (preds.size() == 1)
-                return preds.get(0);
-            return StringUtils.join(preds, " AND ") + " ALLOW FILTERING";
-        }
-
-        public Object[] params()
-        {
-            return params.toArray();
-        }
-    }
-}


[10/32] cassandra git commit: Merge branch 'cassandra-2.1' into cassandra-2.2

Posted by jm...@apache.org.
Merge branch 'cassandra-2.1' into cassandra-2.2


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

Branch: refs/heads/trunk
Commit: 20364f486f1442fccf134ee75ccab1151c0a102c
Parents: 0d4065e f797bfa
Author: Josh McKenzie <jo...@datastax.com>
Authored: Wed Jun 24 12:11:10 2015 -0400
Committer: Josh McKenzie <jo...@datastax.com>
Committed: Wed Jun 24 12:11:10 2015 -0400

----------------------------------------------------------------------

----------------------------------------------------------------------



[27/32] cassandra git commit: 2.2 commit for CASSANDRA-9160

Posted by jm...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/01115f72/test/unit/org/apache/cassandra/cql3/JsonTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/JsonTest.java b/test/unit/org/apache/cassandra/cql3/JsonTest.java
deleted file mode 100644
index 0380ddd..0000000
--- a/test/unit/org/apache/cassandra/cql3/JsonTest.java
+++ /dev/null
@@ -1,947 +0,0 @@
-/*
- * 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;
-
-import org.apache.cassandra.serializers.SimpleDateSerializer;
-import org.apache.cassandra.serializers.TimeSerializer;
-import org.apache.cassandra.utils.ByteBufferUtil;
-import org.junit.Test;
-
-import java.math.BigDecimal;
-import java.math.BigInteger;
-import java.net.InetAddress;
-import java.text.SimpleDateFormat;
-import java.util.Date;
-import java.util.UUID;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-public class JsonTest extends CQLTester
-{
-
-    @Test
-    public void testFromJsonFct() throws Throwable
-    {
-        String typeName = createType("CREATE TYPE %s (a int, b uuid, c set<text>)");
-        createTable("CREATE TABLE %s (" +
-                "k int PRIMARY KEY, " +
-                "asciival ascii, " +
-                "bigintval bigint, " +
-                "blobval blob, " +
-                "booleanval boolean, " +
-                "dateval date, " +
-                "decimalval decimal, " +
-                "doubleval double, " +
-                "floatval float, " +
-                "inetval inet, " +
-                "intval int, " +
-                "textval text, " +
-                "timeval time, " +
-                "timestampval timestamp, " +
-                "timeuuidval timeuuid, " +
-                "uuidval uuid," +
-                "varcharval varchar, " +
-                "varintval varint, " +
-                "listval list<int>, " +
-                "frozenlistval frozen<list<int>>, " +
-                "setval set<uuid>, " +
-                "frozensetval frozen<set<uuid>>, " +
-                "mapval map<ascii, int>," +
-                "frozenmapval frozen<map<ascii, int>>," +
-                "tupleval frozen<tuple<int, ascii, uuid>>," +
-                "udtval frozen<" + typeName + ">)");
-
-
-        // fromJson() can only be used when the receiver type is known
-        assertInvalidMessage("fromJson() cannot be used in the selection clause", "SELECT fromJson(asciival) FROM %s", 0, 0);
-
-        String func1 = createFunction(KEYSPACE, "int", "CREATE FUNCTION %s (a int) CALLED ON NULL INPUT RETURNS text LANGUAGE java AS $$ return a.toString(); $$");
-        createFunctionOverload(func1, "int", "CREATE FUNCTION %s (a text) CALLED ON NULL INPUT RETURNS text LANGUAGE java AS $$ return new String(a); $$");
-
-        assertInvalidMessage("Ambiguous call to function",
-                "INSERT INTO %s (k, textval) VALUES (?, " + func1 + "(fromJson(?)))", 0, "123");
-
-        // fails JSON parsing
-        assertInvalidMessage("Could not decode JSON string '\u038E\u0394\u03B4\u03E0'",
-                "INSERT INTO %s (k, asciival) VALUES (?, fromJson(?))", 0, "\u038E\u0394\u03B4\u03E0");
-
-        // handle nulls
-        execute("INSERT INTO %s (k, asciival) VALUES (?, fromJson(?))", 0, null);
-
-        // ================ ascii ================
-        execute("INSERT INTO %s (k, asciival) VALUES (?, fromJson(?))", 0, "\"ascii text\"");
-        assertRows(execute("SELECT k, asciival FROM %s WHERE k = ?", 0), row(0, "ascii text"));
-
-        execute("INSERT INTO %s (k, asciival) VALUES (?, fromJson(?))", 0, "\"ascii \\\" text\"");
-        assertRows(execute("SELECT k, asciival FROM %s WHERE k = ?", 0), row(0, "ascii \" text"));
-
-        assertInvalidMessage("Invalid ASCII character in string literal",
-                "INSERT INTO %s (k, asciival) VALUES (?, fromJson(?))", 0, "\"\\u1fff\\u2013\\u33B4\\u2014\"");
-
-        assertInvalidMessage("Expected an ascii string, but got a Integer",
-                "INSERT INTO %s (k, asciival) VALUES (?, fromJson(?))", 0, "123");
-
-        // test that we can use fromJson() in other valid places in queries
-        assertRows(execute("SELECT asciival FROM %s WHERE k = fromJson(?)", "0"), row("ascii \" text"));
-        execute("UPDATE %s SET asciival = fromJson(?) WHERE k = fromJson(?)", "\"ascii \\\" text\"", "0");
-        execute("DELETE FROM %s WHERE k = fromJson(?)", "0");
-
-        // ================ bigint ================
-        execute("INSERT INTO %s (k, bigintval) VALUES (?, fromJson(?))", 0, "123123123123");
-        assertRows(execute("SELECT k, bigintval FROM %s WHERE k = ?", 0), row(0, 123123123123L));
-
-        // strings are also accepted
-        execute("INSERT INTO %s (k, bigintval) VALUES (?, fromJson(?))", 0, "\"123123123123\"");
-        assertRows(execute("SELECT k, bigintval FROM %s WHERE k = ?", 0), row(0, 123123123123L));
-
-        // overflow (Long.MAX_VALUE + 1)
-        assertInvalidMessage("Expected a bigint value, but got a",
-                "INSERT INTO %s (k, bigintval) VALUES (?, fromJson(?))", 0, "9223372036854775808");
-
-        assertInvalidMessage("Expected a bigint value, but got a",
-                "INSERT INTO %s (k, bigintval) VALUES (?, fromJson(?))", 0, "123.456");
-
-        assertInvalidMessage("Unable to make long from",
-                "INSERT INTO %s (k, bigintval) VALUES (?, fromJson(?))", 0, "\"abc\"");
-
-        assertInvalidMessage("Expected a bigint value, but got a",
-                "INSERT INTO %s (k, bigintval) VALUES (?, fromJson(?))", 0, "[\"abc\"]");
-
-        // ================ blob ================
-        execute("INSERT INTO %s (k, blobval) VALUES (?, fromJson(?))", 0, "\"0x00000001\"");
-        assertRows(execute("SELECT k, blobval FROM %s WHERE k = ?", 0), row(0, ByteBufferUtil.bytes(1)));
-
-        assertInvalidMessage("Value 'xyzz' is not a valid blob representation",
-            "INSERT INTO %s (k, blobval) VALUES (?, fromJson(?))", 0, "\"xyzz\"");
-
-        assertInvalidMessage("String representation of blob is missing 0x prefix: 123",
-                "INSERT INTO %s (k, blobval) VALUES (?, fromJson(?))", 0, "\"123\"");
-
-        assertInvalidMessage("Value '0x123' is not a valid blob representation",
-                "INSERT INTO %s (k, blobval) VALUES (?, fromJson(?))", 0, "\"0x123\"");
-
-        assertInvalidMessage("Value '123' is not a valid blob representation",
-                "INSERT INTO %s (k, blobval) VALUES (?, fromJson(?))", 0, "123");
-
-        // ================ boolean ================
-        execute("INSERT INTO %s (k, booleanval) VALUES (?, fromJson(?))", 0, "true");
-        assertRows(execute("SELECT k, booleanval FROM %s WHERE k = ?", 0), row(0, true));
-
-        execute("INSERT INTO %s (k, booleanval) VALUES (?, fromJson(?))", 0, "false");
-        assertRows(execute("SELECT k, booleanval FROM %s WHERE k = ?", 0), row(0, false));
-
-        // strings are also accepted
-        execute("INSERT INTO %s (k, booleanval) VALUES (?, fromJson(?))", 0, "\"false\"");
-        assertRows(execute("SELECT k, booleanval FROM %s WHERE k = ?", 0), row(0, false));
-
-        assertInvalidMessage("Unable to make boolean from",
-                "INSERT INTO %s (k, booleanval) VALUES (?, fromJson(?))", 0, "\"abc\"");
-
-        assertInvalidMessage("Expected a boolean value, but got a Integer",
-                "INSERT INTO %s (k, booleanval) VALUES (?, fromJson(?))", 0, "123");
-
-        // ================ date ================
-        execute("INSERT INTO %s (k, dateval) VALUES (?, fromJson(?))", 0, "\"1987-03-23\"");
-        assertRows(execute("SELECT k, dateval FROM %s WHERE k = ?", 0), row(0, SimpleDateSerializer.dateStringToDays("1987-03-23")));
-
-        assertInvalidMessage("Expected a string representation of a date",
-                "INSERT INTO %s (k, dateval) VALUES (?, fromJson(?))", 0, "123");
-
-        assertInvalidMessage("Unable to coerce 'xyz' to a formatted date",
-                "INSERT INTO %s (k, dateval) VALUES (?, fromJson(?))", 0, "\"xyz\"");
-
-        // ================ decimal ================
-        execute("INSERT INTO %s (k, decimalval) VALUES (?, fromJson(?))", 0, "123123.123123");
-        assertRows(execute("SELECT k, decimalval FROM %s WHERE k = ?", 0), row(0, new BigDecimal("123123.123123")));
-
-        execute("INSERT INTO %s (k, decimalval) VALUES (?, fromJson(?))", 0, "123123");
-        assertRows(execute("SELECT k, decimalval FROM %s WHERE k = ?", 0), row(0, new BigDecimal("123123")));
-
-        // accept strings for numbers that cannot be represented as doubles
-        execute("INSERT INTO %s (k, decimalval) VALUES (?, fromJson(?))", 0, "\"123123.123123\"");
-        assertRows(execute("SELECT k, decimalval FROM %s WHERE k = ?", 0), row(0, new BigDecimal("123123.123123")));
-
-        execute("INSERT INTO %s (k, decimalval) VALUES (?, fromJson(?))", 0, "\"-1.23E-12\"");
-        assertRows(execute("SELECT k, decimalval FROM %s WHERE k = ?", 0), row(0, new BigDecimal("-1.23E-12")));
-
-        assertInvalidMessage("Value 'xyzz' is not a valid representation of a decimal value",
-                "INSERT INTO %s (k, decimalval) VALUES (?, fromJson(?))", 0, "\"xyzz\"");
-
-        assertInvalidMessage("Value 'true' is not a valid representation of a decimal value",
-                "INSERT INTO %s (k, decimalval) VALUES (?, fromJson(?))", 0, "true");
-
-        // ================ double ================
-        execute("INSERT INTO %s (k, doubleval) VALUES (?, fromJson(?))", 0, "123123.123123");
-        assertRows(execute("SELECT k, doubleval FROM %s WHERE k = ?", 0), row(0, 123123.123123d));
-
-        execute("INSERT INTO %s (k, doubleval) VALUES (?, fromJson(?))", 0, "123123");
-        assertRows(execute("SELECT k, doubleval FROM %s WHERE k = ?", 0), row(0, 123123.0d));
-
-        // strings are also accepted
-        execute("INSERT INTO %s (k, doubleval) VALUES (?, fromJson(?))", 0, "\"123123\"");
-        assertRows(execute("SELECT k, doubleval FROM %s WHERE k = ?", 0), row(0, 123123.0d));
-
-        assertInvalidMessage("Unable to make double from",
-                "INSERT INTO %s (k, doubleval) VALUES (?, fromJson(?))", 0, "\"xyzz\"");
-
-        assertInvalidMessage("Expected a double value, but got",
-                "INSERT INTO %s (k, doubleval) VALUES (?, fromJson(?))", 0, "true");
-
-        // ================ float ================
-        execute("INSERT INTO %s (k, floatval) VALUES (?, fromJson(?))", 0, "123123.123123");
-        assertRows(execute("SELECT k, floatval FROM %s WHERE k = ?", 0), row(0, 123123.123123f));
-
-        execute("INSERT INTO %s (k, floatval) VALUES (?, fromJson(?))", 0, "123123");
-        assertRows(execute("SELECT k, floatval FROM %s WHERE k = ?", 0), row(0, 123123.0f));
-
-        // strings are also accepted
-        execute("INSERT INTO %s (k, floatval) VALUES (?, fromJson(?))", 0, "\"123123.0\"");
-        assertRows(execute("SELECT k, floatval FROM %s WHERE k = ?", 0), row(0, 123123.0f));
-
-        assertInvalidMessage("Unable to make float from",
-                "INSERT INTO %s (k, floatval) VALUES (?, fromJson(?))", 0, "\"xyzz\"");
-
-        assertInvalidMessage("Expected a float value, but got a",
-                "INSERT INTO %s (k, floatval) VALUES (?, fromJson(?))", 0, "true");
-
-        // ================ inet ================
-        execute("INSERT INTO %s (k, inetval) VALUES (?, fromJson(?))", 0, "\"127.0.0.1\"");
-        assertRows(execute("SELECT k, inetval FROM %s WHERE k = ?", 0), row(0, InetAddress.getByName("127.0.0.1")));
-
-        execute("INSERT INTO %s (k, inetval) VALUES (?, fromJson(?))", 0, "\"::1\"");
-        assertRows(execute("SELECT k, inetval FROM %s WHERE k = ?", 0), row(0, InetAddress.getByName("::1")));
-
-        assertInvalidMessage("Unable to make inet address from 'xyzz'",
-                "INSERT INTO %s (k, inetval) VALUES (?, fromJson(?))", 0, "\"xyzz\"");
-
-        assertInvalidMessage("Expected a string representation of an inet value, but got a Integer",
-                "INSERT INTO %s (k, inetval) VALUES (?, fromJson(?))", 0, "123");
-
-        // ================ int ================
-        execute("INSERT INTO %s (k, intval) VALUES (?, fromJson(?))", 0, "123123");
-        assertRows(execute("SELECT k, intval FROM %s WHERE k = ?", 0), row(0, 123123));
-
-        // strings are also accepted
-        execute("INSERT INTO %s (k, intval) VALUES (?, fromJson(?))", 0, "\"123123\"");
-        assertRows(execute("SELECT k, intval FROM %s WHERE k = ?", 0), row(0, 123123));
-
-        // int overflow (2 ^ 32, or Integer.MAX_INT + 1)
-        assertInvalidMessage("Expected an int value, but got a",
-                "INSERT INTO %s (k, intval) VALUES (?, fromJson(?))", 0, "2147483648");
-
-        assertInvalidMessage("Expected an int value, but got a",
-                "INSERT INTO %s (k, intval) VALUES (?, fromJson(?))", 0, "123.456");
-
-        assertInvalidMessage("Unable to make int from",
-                "INSERT INTO %s (k, intval) VALUES (?, fromJson(?))", 0, "\"xyzz\"");
-
-        assertInvalidMessage("Expected an int value, but got a",
-                "INSERT INTO %s (k, intval) VALUES (?, fromJson(?))", 0, "true");
-
-        // ================ text (varchar) ================
-        execute("INSERT INTO %s (k, textval) VALUES (?, fromJson(?))", 0, "\"\"");
-        assertRows(execute("SELECT k, textval FROM %s WHERE k = ?", 0), row(0, ""));
-
-        execute("INSERT INTO %s (k, textval) VALUES (?, fromJson(?))", 0, "\"abcd\"");
-        assertRows(execute("SELECT k, textval FROM %s WHERE k = ?", 0), row(0, "abcd"));
-
-        execute("INSERT INTO %s (k, textval) VALUES (?, fromJson(?))", 0, "\"some \\\" text\"");
-        assertRows(execute("SELECT k, textval FROM %s WHERE k = ?", 0), row(0, "some \" text"));
-
-        execute("INSERT INTO %s (k, textval) VALUES (?, fromJson(?))", 0, "\"\\u2013\"");
-        assertRows(execute("SELECT k, textval FROM %s WHERE k = ?", 0), row(0, "\u2013"));
-
-        assertInvalidMessage("Expected a UTF-8 string, but got a Integer",
-                "INSERT INTO %s (k, textval) VALUES (?, fromJson(?))", 0, "123");
-
-        // ================ time ================
-        execute("INSERT INTO %s (k, timeval) VALUES (?, fromJson(?))", 0, "\"07:35:07.000111222\"");
-        assertRows(execute("SELECT k, timeval FROM %s WHERE k = ?", 0), row(0, TimeSerializer.timeStringToLong("07:35:07.000111222")));
-
-        assertInvalidMessage("Expected a string representation of a time value",
-                "INSERT INTO %s (k, timeval) VALUES (?, fromJson(?))", 0, "123456");
-
-        assertInvalidMessage("Unable to coerce 'xyz' to a formatted time",
-                "INSERT INTO %s (k, timeval) VALUES (?, fromJson(?))", 0, "\"xyz\"");
-
-        // ================ timestamp ================
-        execute("INSERT INTO %s (k, timestampval) VALUES (?, fromJson(?))", 0, "123123123123");
-        assertRows(execute("SELECT k, timestampval FROM %s WHERE k = ?", 0), row(0, new Date(123123123123L)));
-
-        execute("INSERT INTO %s (k, timestampval) VALUES (?, fromJson(?))", 0, "\"2014-01-01\"");
-        assertRows(execute("SELECT k, timestampval FROM %s WHERE k = ?", 0), row(0, new SimpleDateFormat("y-M-d").parse("2014-01-01")));
-
-        assertInvalidMessage("Expected a long or a datestring representation of a timestamp value, but got a Double",
-                "INSERT INTO %s (k, timestampval) VALUES (?, fromJson(?))", 0, "123.456");
-
-        assertInvalidMessage("Unable to coerce 'abcd' to a formatted date",
-                "INSERT INTO %s (k, timestampval) VALUES (?, fromJson(?))", 0, "\"abcd\"");
-
-        // ================ timeuuid ================
-        execute("INSERT INTO %s (k, timeuuidval) VALUES (?, fromJson(?))", 0, "\"6bddc89a-5644-11e4-97fc-56847afe9799\"");
-        assertRows(execute("SELECT k, timeuuidval FROM %s WHERE k = ?", 0), row(0, UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799")));
-
-        execute("INSERT INTO %s (k, timeuuidval) VALUES (?, fromJson(?))", 0, "\"6BDDC89A-5644-11E4-97FC-56847AFE9799\"");
-        assertRows(execute("SELECT k, timeuuidval FROM %s WHERE k = ?", 0), row(0, UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799")));
-
-        assertInvalidMessage("TimeUUID supports only version 1 UUIDs",
-                "INSERT INTO %s (k, timeuuidval) VALUES (?, fromJson(?))", 0, "\"00000000-0000-0000-0000-000000000000\"");
-
-        assertInvalidMessage("Expected a string representation of a timeuuid, but got a Integer",
-                "INSERT INTO %s (k, timeuuidval) VALUES (?, fromJson(?))", 0, "123");
-
-         // ================ uuidval ================
-        execute("INSERT INTO %s (k, uuidval) VALUES (?, fromJson(?))", 0, "\"6bddc89a-5644-11e4-97fc-56847afe9799\"");
-        assertRows(execute("SELECT k, uuidval FROM %s WHERE k = ?", 0), row(0, UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799")));
-
-        execute("INSERT INTO %s (k, uuidval) VALUES (?, fromJson(?))", 0, "\"6BDDC89A-5644-11E4-97FC-56847AFE9799\"");
-        assertRows(execute("SELECT k, uuidval FROM %s WHERE k = ?", 0), row(0, UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799")));
-
-        assertInvalidMessage("Unable to make UUID from",
-                "INSERT INTO %s (k, uuidval) VALUES (?, fromJson(?))", 0, "\"00000000-0000-0000-zzzz-000000000000\"");
-
-        assertInvalidMessage("Expected a string representation of a uuid, but got a Integer",
-                "INSERT INTO %s (k, uuidval) VALUES (?, fromJson(?))", 0, "123");
-
-        // ================ varint ================
-        execute("INSERT INTO %s (k, varintval) VALUES (?, fromJson(?))", 0, "123123123123");
-        assertRows(execute("SELECT k, varintval FROM %s WHERE k = ?", 0), row(0, new BigInteger("123123123123")));
-
-        // accept strings for numbers that cannot be represented as longs
-        execute("INSERT INTO %s (k, varintval) VALUES (?, fromJson(?))", 0, "\"1234567890123456789012345678901234567890\"");
-        assertRows(execute("SELECT k, varintval FROM %s WHERE k = ?", 0), row(0, new BigInteger("1234567890123456789012345678901234567890")));
-
-        assertInvalidMessage("Value '123123.123' is not a valid representation of a varint value",
-                "INSERT INTO %s (k, varintval) VALUES (?, fromJson(?))", 0, "123123.123");
-
-        assertInvalidMessage("Value 'xyzz' is not a valid representation of a varint value",
-                "INSERT INTO %s (k, varintval) VALUES (?, fromJson(?))", 0, "\"xyzz\"");
-
-        assertInvalidMessage("Value '' is not a valid representation of a varint value",
-                "INSERT INTO %s (k, varintval) VALUES (?, fromJson(?))", 0, "\"\"");
-
-        assertInvalidMessage("Value 'true' is not a valid representation of a varint value",
-                "INSERT INTO %s (k, varintval) VALUES (?, fromJson(?))", 0, "true");
-
-        // ================ lists ================
-        execute("INSERT INTO %s (k, listval) VALUES (?, fromJson(?))", 0, "[1, 2, 3]");
-        assertRows(execute("SELECT k, listval FROM %s WHERE k = ?", 0), row(0, list(1, 2, 3)));
-
-        execute("INSERT INTO %s (k, listval) VALUES (?, fromJson(?))", 0, "[]");
-        assertRows(execute("SELECT k, listval FROM %s WHERE k = ?", 0), row(0, null));
-
-        assertInvalidMessage("Expected a list, but got a Integer",
-                "INSERT INTO %s (k, listval) VALUES (?, fromJson(?))", 0, "123");
-
-        assertInvalidMessage("Unable to make int from",
-                "INSERT INTO %s (k, listval) VALUES (?, fromJson(?))", 0, "[\"abc\"]");
-
-        assertInvalidMessage("Invalid null element in list",
-                "INSERT INTO %s (k, listval) VALUES (?, fromJson(?))", 0, "[null]");
-
-        // frozen
-        execute("INSERT INTO %s (k, frozenlistval) VALUES (?, fromJson(?))", 0, "[1, 2, 3]");
-        assertRows(execute("SELECT k, frozenlistval FROM %s WHERE k = ?", 0), row(0, list(1, 2, 3)));
-
-        // ================ sets ================
-        execute("INSERT INTO %s (k, setval) VALUES (?, fromJson(?))",
-                0, "[\"6bddc89a-5644-11e4-97fc-56847afe9798\", \"6bddc89a-5644-11e4-97fc-56847afe9799\"]");
-        assertRows(execute("SELECT k, setval FROM %s WHERE k = ?", 0),
-                row(0, set(UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9798"), (UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799"))))
-        );
-
-        // duplicates are okay, just like in CQL
-        execute("INSERT INTO %s (k, setval) VALUES (?, fromJson(?))",
-                0, "[\"6bddc89a-5644-11e4-97fc-56847afe9798\", \"6bddc89a-5644-11e4-97fc-56847afe9798\", \"6bddc89a-5644-11e4-97fc-56847afe9799\"]");
-        assertRows(execute("SELECT k, setval FROM %s WHERE k = ?", 0),
-                row(0, set(UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9798"), (UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799"))))
-        );
-
-        execute("INSERT INTO %s (k, setval) VALUES (?, fromJson(?))", 0, "[]");
-        assertRows(execute("SELECT k, setval FROM %s WHERE k = ?", 0), row(0, null));
-
-        assertInvalidMessage("Expected a list (representing a set), but got a Integer",
-                "INSERT INTO %s (k, setval) VALUES (?, fromJson(?))", 0, "123");
-
-        assertInvalidMessage("Unable to make UUID from",
-                "INSERT INTO %s (k, setval) VALUES (?, fromJson(?))", 0, "[\"abc\"]");
-
-        assertInvalidMessage("Invalid null element in set",
-                "INSERT INTO %s (k, setval) VALUES (?, fromJson(?))", 0, "[null]");
-
-        // frozen
-        execute("INSERT INTO %s (k, frozensetval) VALUES (?, fromJson(?))",
-                0, "[\"6bddc89a-5644-11e4-97fc-56847afe9798\", \"6bddc89a-5644-11e4-97fc-56847afe9799\"]");
-        assertRows(execute("SELECT k, frozensetval FROM %s WHERE k = ?", 0),
-                row(0, set(UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9798"), (UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799"))))
-        );
-
-        execute("INSERT INTO %s (k, frozensetval) VALUES (?, fromJson(?))",
-                0, "[\"6bddc89a-5644-11e4-97fc-56847afe9799\", \"6bddc89a-5644-11e4-97fc-56847afe9798\"]");
-        assertRows(execute("SELECT k, frozensetval FROM %s WHERE k = ?", 0),
-                row(0, set(UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9798"), (UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799"))))
-        );
-
-        // ================ maps ================
-        execute("INSERT INTO %s (k, mapval) VALUES (?, fromJson(?))", 0, "{\"a\": 1, \"b\": 2}");
-        assertRows(execute("SELECT k, mapval FROM %s WHERE k = ?", 0), row(0, map("a", 1, "b", 2)));
-
-        execute("INSERT INTO %s (k, mapval) VALUES (?, fromJson(?))", 0, "{}");
-        assertRows(execute("SELECT k, mapval FROM %s WHERE k = ?", 0), row(0, null));
-
-        assertInvalidMessage("Expected a map, but got a Integer",
-                "INSERT INTO %s (k, mapval) VALUES (?, fromJson(?))", 0, "123");
-
-        assertInvalidMessage("Invalid ASCII character in string literal",
-                "INSERT INTO %s (k, mapval) VALUES (?, fromJson(?))", 0, "{\"\\u1fff\\u2013\\u33B4\\u2014\": 1}");
-
-        assertInvalidMessage("Invalid null value in map",
-                "INSERT INTO %s (k, mapval) VALUES (?, fromJson(?))", 0, "{\"a\": null}");
-
-        // frozen
-        execute("INSERT INTO %s (k, frozenmapval) VALUES (?, fromJson(?))", 0, "{\"a\": 1, \"b\": 2}");
-        assertRows(execute("SELECT k, frozenmapval FROM %s WHERE k = ?", 0), row(0, map("a", 1, "b", 2)));
-
-        execute("INSERT INTO %s (k, frozenmapval) VALUES (?, fromJson(?))", 0, "{\"b\": 2, \"a\": 1}");
-        assertRows(execute("SELECT k, frozenmapval FROM %s WHERE k = ?", 0), row(0, map("a", 1, "b", 2)));
-
-        // ================ tuples ================
-        execute("INSERT INTO %s (k, tupleval) VALUES (?, fromJson(?))", 0, "[1, \"foobar\", \"6bddc89a-5644-11e4-97fc-56847afe9799\"]");
-        assertRows(execute("SELECT k, tupleval FROM %s WHERE k = ?", 0),
-            row(0, tuple(1, "foobar", UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799")))
-        );
-
-        execute("INSERT INTO %s (k, tupleval) VALUES (?, fromJson(?))", 0, "[1, null, \"6bddc89a-5644-11e4-97fc-56847afe9799\"]");
-        assertRows(execute("SELECT k, tupleval FROM %s WHERE k = ?", 0),
-                row(0, tuple(1, null, UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799")))
-        );
-
-        assertInvalidMessage("Tuple contains extra items",
-                "INSERT INTO %s (k, tupleval) VALUES (?, fromJson(?))",
-                0, "[1, \"foobar\", \"6bddc89a-5644-11e4-97fc-56847afe9799\", 1, 2, 3]");
-
-        assertInvalidMessage("Tuple is missing items",
-                "INSERT INTO %s (k, tupleval) VALUES (?, fromJson(?))",
-                0, "[1, \"foobar\"]");
-
-        assertInvalidMessage("Unable to make int from",
-                "INSERT INTO %s (k, tupleval) VALUES (?, fromJson(?))",
-                0, "[\"not an int\", \"foobar\", \"6bddc89a-5644-11e4-97fc-56847afe9799\"]");
-
-        // ================ UDTs ================
-        execute("INSERT INTO %s (k, udtval) VALUES (?, fromJson(?))", 0, "{\"a\": 1, \"b\": \"6bddc89a-5644-11e4-97fc-56847afe9799\", \"c\": [\"foo\", \"bar\"]}");
-        assertRows(execute("SELECT k, udtval.a, udtval.b, udtval.c FROM %s WHERE k = ?", 0),
-                row(0, 1, UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799"), set("bar", "foo"))
-        );
-
-        // order of fields shouldn't matter
-        execute("INSERT INTO %s (k, udtval) VALUES (?, fromJson(?))", 0, "{\"b\": \"6bddc89a-5644-11e4-97fc-56847afe9799\", \"a\": 1, \"c\": [\"foo\", \"bar\"]}");
-        assertRows(execute("SELECT k, udtval.a, udtval.b, udtval.c FROM %s WHERE k = ?", 0),
-                row(0, 1, UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799"), set("bar", "foo"))
-        );
-
-        // test nulls
-        execute("INSERT INTO %s (k, udtval) VALUES (?, fromJson(?))", 0, "{\"a\": null, \"b\": \"6bddc89a-5644-11e4-97fc-56847afe9799\", \"c\": [\"foo\", \"bar\"]}");
-        assertRows(execute("SELECT k, udtval.a, udtval.b, udtval.c FROM %s WHERE k = ?", 0),
-                row(0, null, UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799"), set("bar", "foo"))
-        );
-
-        // test missing fields
-        execute("INSERT INTO %s (k, udtval) VALUES (?, fromJson(?))", 0, "{\"a\": 1, \"b\": \"6bddc89a-5644-11e4-97fc-56847afe9799\"}");
-        assertRows(execute("SELECT k, udtval.a, udtval.b, udtval.c FROM %s WHERE k = ?", 0),
-                row(0, 1, UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799"), null)
-        );
-
-        assertInvalidMessage("Unknown field", "INSERT INTO %s (k, udtval) VALUES (?, fromJson(?))", 0, "{\"xxx\": 1}");
-        assertInvalidMessage("Unable to make int from",
-                "INSERT INTO %s (k, udtval) VALUES (?, fromJson(?))", 0, "{\"a\": \"foobar\"}");
-    }
-
-    @Test
-    public void testToJsonFct() throws Throwable
-    {
-        String typeName = createType("CREATE TYPE %s (a int, b uuid, c set<text>)");
-        createTable("CREATE TABLE %s (" +
-                "k int PRIMARY KEY, " +
-                "asciival ascii, " +
-                "bigintval bigint, " +
-                "blobval blob, " +
-                "booleanval boolean, " +
-                "dateval date, " +
-                "decimalval decimal, " +
-                "doubleval double, " +
-                "floatval float, " +
-                "inetval inet, " +
-                "intval int, " +
-                "textval text, " +
-                "timeval time, " +
-                "timestampval timestamp, " +
-                "timeuuidval timeuuid, " +
-                "uuidval uuid," +
-                "varcharval varchar, " +
-                "varintval varint, " +
-                "listval list<int>, " +
-                "frozenlistval frozen<list<int>>, " +
-                "setval set<uuid>, " +
-                "frozensetval frozen<set<uuid>>, " +
-                "mapval map<ascii, int>, " +
-                "frozenmapval frozen<map<ascii, int>>, " +
-                "tupleval frozen<tuple<int, ascii, uuid>>," +
-                "udtval frozen<" + typeName + ">)");
-
-        // toJson() can only be used in selections
-        assertInvalidMessage("toJson() may only be used within the selection clause",
-                "INSERT INTO %s (k, asciival) VALUES (?, toJson(?))", 0, 0);
-        assertInvalidMessage("toJson() may only be used within the selection clause",
-                "UPDATE %s SET asciival = toJson(?) WHERE k = ?", 0, 0);
-        assertInvalidMessage("toJson() may only be used within the selection clause",
-                "DELETE FROM %s WHERE k = fromJson(toJson(?))", 0);
-
-        // ================ ascii ================
-        execute("INSERT INTO %s (k, asciival) VALUES (?, ?)", 0, "ascii text");
-        assertRows(execute("SELECT k, toJson(asciival) FROM %s WHERE k = ?", 0), row(0, "\"ascii text\""));
-
-        execute("INSERT INTO %s (k, asciival) VALUES (?, ?)", 0, "");
-        assertRows(execute("SELECT k, toJson(asciival) FROM %s WHERE k = ?", 0), row(0, "\"\""));
-
-        // ================ bigint ================
-        execute("INSERT INTO %s (k, bigintval) VALUES (?, ?)", 0, 123123123123L);
-        assertRows(execute("SELECT k, toJson(bigintval) FROM %s WHERE k = ?", 0), row(0, "123123123123"));
-
-        execute("INSERT INTO %s (k, bigintval) VALUES (?, ?)", 0, 0L);
-        assertRows(execute("SELECT k, toJson(bigintval) FROM %s WHERE k = ?", 0), row(0, "0"));
-
-        execute("INSERT INTO %s (k, bigintval) VALUES (?, ?)", 0, -123123123123L);
-        assertRows(execute("SELECT k, toJson(bigintval) FROM %s WHERE k = ?", 0), row(0, "-123123123123"));
-
-        // ================ blob ================
-        execute("INSERT INTO %s (k, blobval) VALUES (?, ?)", 0, ByteBufferUtil.bytes(1));
-        assertRows(execute("SELECT k, toJson(blobval) FROM %s WHERE k = ?", 0), row(0, "\"0x00000001\""));
-
-        execute("INSERT INTO %s (k, blobval) VALUES (?, ?)", 0, ByteBufferUtil.EMPTY_BYTE_BUFFER);
-        assertRows(execute("SELECT k, toJson(blobval) FROM %s WHERE k = ?", 0), row(0, "\"0x\""));
-
-        // ================ boolean ================
-        execute("INSERT INTO %s (k, booleanval) VALUES (?, ?)", 0, true);
-        assertRows(execute("SELECT k, toJson(booleanval) FROM %s WHERE k = ?", 0), row(0, "true"));
-
-        execute("INSERT INTO %s (k, booleanval) VALUES (?, ?)", 0, false);
-        assertRows(execute("SELECT k, toJson(booleanval) FROM %s WHERE k = ?", 0), row(0, "false"));
-
-        // ================ date ================
-        execute("INSERT INTO %s (k, dateval) VALUES (?, ?)", 0, SimpleDateSerializer.dateStringToDays("1987-03-23"));
-        assertRows(execute("SELECT k, toJson(dateval) FROM %s WHERE k = ?", 0), row(0, "\"1987-03-23\""));
-
-        // ================ decimal ================
-        execute("INSERT INTO %s (k, decimalval) VALUES (?, ?)", 0, new BigDecimal("123123.123123"));
-        assertRows(execute("SELECT k, toJson(decimalval) FROM %s WHERE k = ?", 0), row(0, "123123.123123"));
-
-        execute("INSERT INTO %s (k, decimalval) VALUES (?, ?)", 0, new BigDecimal("-1.23E-12"));
-        assertRows(execute("SELECT k, toJson(decimalval) FROM %s WHERE k = ?", 0), row(0, "-1.23E-12"));
-
-        // ================ double ================
-        execute("INSERT INTO %s (k, doubleval) VALUES (?, ?)", 0, 123123.123123d);
-        assertRows(execute("SELECT k, toJson(doubleval) FROM %s WHERE k = ?", 0), row(0, "123123.123123"));
-
-        execute("INSERT INTO %s (k, doubleval) VALUES (?, ?)", 0, 123123d);
-        assertRows(execute("SELECT k, toJson(doubleval) FROM %s WHERE k = ?", 0), row(0, "123123.0"));
-
-        // ================ float ================
-        execute("INSERT INTO %s (k, floatval) VALUES (?, ?)", 0, 123.123f);
-        assertRows(execute("SELECT k, toJson(floatval) FROM %s WHERE k = ?", 0), row(0, "123.123"));
-
-        execute("INSERT INTO %s (k, floatval) VALUES (?, ?)", 0, 123123f);
-        assertRows(execute("SELECT k, toJson(floatval) FROM %s WHERE k = ?", 0), row(0, "123123.0"));
-
-        // ================ inet ================
-        execute("INSERT INTO %s (k, inetval) VALUES (?, ?)", 0, InetAddress.getByName("127.0.0.1"));
-        assertRows(execute("SELECT k, toJson(inetval) FROM %s WHERE k = ?", 0), row(0, "\"127.0.0.1\""));
-
-        execute("INSERT INTO %s (k, inetval) VALUES (?, ?)", 0, InetAddress.getByName("::1"));
-        assertRows(execute("SELECT k, toJson(inetval) FROM %s WHERE k = ?", 0), row(0, "\"0:0:0:0:0:0:0:1\""));
-
-        // ================ int ================
-        execute("INSERT INTO %s (k, intval) VALUES (?, ?)", 0, 123123);
-        assertRows(execute("SELECT k, toJson(intval) FROM %s WHERE k = ?", 0), row(0, "123123"));
-
-        execute("INSERT INTO %s (k, intval) VALUES (?, ?)", 0, 0);
-        assertRows(execute("SELECT k, toJson(intval) FROM %s WHERE k = ?", 0), row(0, "0"));
-
-        execute("INSERT INTO %s (k, intval) VALUES (?, ?)", 0, -123123);
-        assertRows(execute("SELECT k, toJson(intval) FROM %s WHERE k = ?", 0), row(0, "-123123"));
-
-        // ================ text (varchar) ================
-        execute("INSERT INTO %s (k, textval) VALUES (?, ?)", 0, "");
-        assertRows(execute("SELECT k, toJson(textval) FROM %s WHERE k = ?", 0), row(0, "\"\""));
-
-        execute("INSERT INTO %s (k, textval) VALUES (?, ?)", 0, "abcd");
-        assertRows(execute("SELECT k, toJson(textval) FROM %s WHERE k = ?", 0), row(0, "\"abcd\""));
-
-        execute("INSERT INTO %s (k, textval) VALUES (?, ?)", 0, "\u8422");
-        assertRows(execute("SELECT k, toJson(textval) FROM %s WHERE k = ?", 0), row(0, "\"\u8422\""));
-
-        execute("INSERT INTO %s (k, textval) VALUES (?, ?)", 0, "\u0000");
-        assertRows(execute("SELECT k, toJson(textval) FROM %s WHERE k = ?", 0), row(0, "\"\\u0000\""));
-
-        // ================ timestamp ================
-        execute("INSERT INTO %s (k, timeval) VALUES (?, ?)", 0, 123L);
-        assertRows(execute("SELECT k, toJson(timeval) FROM %s WHERE k = ?", 0), row(0, "\"00:00:00.000000123\""));
-
-        // ================ timestamp ================
-        execute("INSERT INTO %s (k, timestampval) VALUES (?, ?)", 0, new SimpleDateFormat("y-M-d").parse("2014-01-01"));
-        assertRows(execute("SELECT k, toJson(timestampval) FROM %s WHERE k = ?", 0), row(0, "\"2014-01-01 00:00:00.000\""));
-
-        // ================ timeuuid ================
-        execute("INSERT INTO %s (k, timeuuidval) VALUES (?, ?)", 0, UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799"));
-        assertRows(execute("SELECT k, toJson(timeuuidval) FROM %s WHERE k = ?", 0), row(0, "\"6bddc89a-5644-11e4-97fc-56847afe9799\""));
-
-         // ================ uuidval ================
-        execute("INSERT INTO %s (k, uuidval) VALUES (?, ?)", 0, UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799"));
-        assertRows(execute("SELECT k, toJson(uuidval) FROM %s WHERE k = ?", 0), row(0, "\"6bddc89a-5644-11e4-97fc-56847afe9799\""));
-
-        // ================ varint ================
-        execute("INSERT INTO %s (k, varintval) VALUES (?, ?)", 0, new BigInteger("123123123123123123123"));
-        assertRows(execute("SELECT k, toJson(varintval) FROM %s WHERE k = ?", 0), row(0, "123123123123123123123"));
-
-        // ================ lists ================
-        execute("INSERT INTO %s (k, listval) VALUES (?, ?)", 0, list(1, 2, 3));
-        assertRows(execute("SELECT k, toJson(listval) FROM %s WHERE k = ?", 0), row(0, "[1, 2, 3]"));
-
-        execute("INSERT INTO %s (k, listval) VALUES (?, ?)", 0, list());
-        assertRows(execute("SELECT k, toJson(listval) FROM %s WHERE k = ?", 0), row(0, "null"));
-
-        // frozen
-        execute("INSERT INTO %s (k, frozenlistval) VALUES (?, ?)", 0, list(1, 2, 3));
-        assertRows(execute("SELECT k, toJson(frozenlistval) FROM %s WHERE k = ?", 0), row(0, "[1, 2, 3]"));
-
-        // ================ sets ================
-        execute("INSERT INTO %s (k, setval) VALUES (?, ?)",
-                0, set(UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9798"), (UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799"))));
-        assertRows(execute("SELECT k, toJson(setval) FROM %s WHERE k = ?", 0),
-                row(0, "[\"6bddc89a-5644-11e4-97fc-56847afe9798\", \"6bddc89a-5644-11e4-97fc-56847afe9799\"]")
-        );
-
-        execute("INSERT INTO %s (k, setval) VALUES (?, ?)", 0, set());
-        assertRows(execute("SELECT k, toJson(setval) FROM %s WHERE k = ?", 0), row(0, "null"));
-
-        // frozen
-        execute("INSERT INTO %s (k, frozensetval) VALUES (?, ?)",
-                0, set(UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9798"), (UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799"))));
-        assertRows(execute("SELECT k, toJson(frozensetval) FROM %s WHERE k = ?", 0),
-                row(0, "[\"6bddc89a-5644-11e4-97fc-56847afe9798\", \"6bddc89a-5644-11e4-97fc-56847afe9799\"]")
-        );
-
-        // ================ maps ================
-        execute("INSERT INTO %s (k, mapval) VALUES (?, ?)", 0, map("a", 1, "b", 2));
-        assertRows(execute("SELECT k, toJson(mapval) FROM %s WHERE k = ?", 0), row(0, "{\"a\": 1, \"b\": 2}"));
-
-        execute("INSERT INTO %s (k, mapval) VALUES (?, ?)", 0, map());
-        assertRows(execute("SELECT k, toJson(mapval) FROM %s WHERE k = ?", 0), row(0, "null"));
-
-        // frozen
-        execute("INSERT INTO %s (k, frozenmapval) VALUES (?, ?)", 0, map("a", 1, "b", 2));
-        assertRows(execute("SELECT k, toJson(frozenmapval) FROM %s WHERE k = ?", 0), row(0, "{\"a\": 1, \"b\": 2}"));
-
-        // ================ tuples ================
-        execute("INSERT INTO %s (k, tupleval) VALUES (?, ?)", 0, tuple(1, "foobar", UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799")));
-        assertRows(execute("SELECT k, toJson(tupleval) FROM %s WHERE k = ?", 0),
-            row(0, "[1, \"foobar\", \"6bddc89a-5644-11e4-97fc-56847afe9799\"]")
-        );
-
-        execute("INSERT INTO %s (k, tupleval) VALUES (?, ?)", 0, tuple(1, "foobar", null));
-        assertRows(execute("SELECT k, toJson(tupleval) FROM %s WHERE k = ?", 0),
-                row(0, "[1, \"foobar\", null]")
-        );
-
-        // ================ UDTs ================
-        execute("INSERT INTO %s (k, udtval) VALUES (?, {a: ?, b: ?, c: ?})", 0, 1, UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799"), set("foo", "bar"));
-        assertRows(execute("SELECT k, toJson(udtval) FROM %s WHERE k = ?", 0),
-                row(0, "{\"a\": 1, \"b\": \"6bddc89a-5644-11e4-97fc-56847afe9799\", \"c\": [\"bar\", \"foo\"]}")
-        );
-
-        execute("INSERT INTO %s (k, udtval) VALUES (?, {a: ?, b: ?})", 0, 1, UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799"));
-        assertRows(execute("SELECT k, toJson(udtval) FROM %s WHERE k = ?", 0),
-                row(0, "{\"a\": 1, \"b\": \"6bddc89a-5644-11e4-97fc-56847afe9799\", \"c\": null}")
-        );
-    }
-
-    @Test
-    public void testSelectJsonSyntax() throws Throwable
-    {
-        // tests SELECT JSON statements
-        createTable("CREATE TABLE %s (k int primary key, v int)");
-        execute("INSERT INTO %s (k, v) VALUES (0, 0)");
-        execute("INSERT INTO %s (k, v) VALUES (1, 1)");
-
-        assertRows(execute("SELECT JSON * FROM %s"),
-                row("{\"k\": 0, \"v\": 0}"),
-                row("{\"k\": 1, \"v\": 1}")
-        );
-
-        assertRows(execute("SELECT JSON k, v FROM %s"),
-                row("{\"k\": 0, \"v\": 0}"),
-                row("{\"k\": 1, \"v\": 1}")
-        );
-
-        assertRows(execute("SELECT JSON v, k FROM %s"),
-                row("{\"v\": 0, \"k\": 0}"),
-                row("{\"v\": 1, \"k\": 1}")
-        );
-
-        assertRows(execute("SELECT JSON v as foo, k as bar FROM %s"),
-                row("{\"foo\": 0, \"bar\": 0}"),
-                row("{\"foo\": 1, \"bar\": 1}")
-        );
-
-        assertRows(execute("SELECT JSON ttl(v), k FROM %s"),
-                row("{\"ttl(v)\": null, \"k\": 0}"),
-                row("{\"ttl(v)\": null, \"k\": 1}")
-        );
-
-        assertRows(execute("SELECT JSON ttl(v) as foo, k FROM %s"),
-                row("{\"foo\": null, \"k\": 0}"),
-                row("{\"foo\": null, \"k\": 1}")
-        );
-
-        assertRows(execute("SELECT JSON count(*) FROM %s"),
-                row("{\"count\": 2}")
-        );
-
-        assertRows(execute("SELECT JSON count(*) as foo FROM %s"),
-                row("{\"foo\": 2}")
-        );
-
-        assertRows(execute("SELECT JSON toJson(blobAsInt(intAsBlob(v))) FROM %s LIMIT 1"),
-                row("{\"system.tojson(system.blobasint(system.intasblob(v)))\": \"0\"}")
-        );
-    }
-
-    @Test
-    public void testInsertJsonSyntax() throws Throwable
-    {
-        createTable("CREATE TABLE %s (k int primary key, v int)");
-        execute("INSERT INTO %s JSON ?", "{\"k\": 0, \"v\": 0}");
-        assertRows(execute("SELECT * FROM %s"),
-                row(0, 0)
-        );
-
-        // without specifying column names
-        execute("INSERT INTO %s JSON ?", "{\"k\": 0, \"v\": 0}");
-        assertRows(execute("SELECT * FROM %s"),
-                row(0, 0)
-        );
-
-        execute("INSERT INTO %s JSON ?", "{\"k\": 0, \"v\": null}");
-        assertRows(execute("SELECT * FROM %s"),
-                row(0, null)
-        );
-
-        execute("INSERT INTO %s JSON ?", "{\"v\": 1, \"k\": 0}");
-        assertRows(execute("SELECT * FROM %s"),
-                row(0, 1)
-        );
-
-        execute("INSERT INTO %s JSON ?", "{\"k\": 0}");
-        assertRows(execute("SELECT * FROM %s"),
-                row(0, null)
-        );
-
-        if (USE_PREPARED_VALUES)
-            assertInvalidMessage("Got null for INSERT JSON values", "INSERT INTO %s JSON ?", new Object[]{null});
-
-        assertInvalidMessage("Got null for INSERT JSON values", "INSERT INTO %s JSON ?", "null");
-        assertInvalidMessage("Could not decode JSON string as a map", "INSERT INTO %s JSON ?", "\"notamap\"");
-        assertInvalidMessage("Could not decode JSON string as a map", "INSERT INTO %s JSON ?", "12.34");
-        assertInvalidMessage("JSON values map contains unrecognized column",
-                "INSERT INTO %s JSON ?",
-                "{\"k\": 0, \"v\": 0, \"zzz\": 0}");
-
-        assertInvalidMessage("Unable to make int from",
-                "INSERT INTO %s JSON ?",
-                "{\"k\": 0, \"v\": \"notanint\"}");
-    }
-
-    @Test
-    public void testCaseSensitivity() throws Throwable
-    {
-        createTable("CREATE TABLE %s (k int primary key, \"Foo\" int)");
-        execute("INSERT INTO %s JSON ?", "{\"k\": 0, \"\\\"Foo\\\"\": 0}");
-        execute("INSERT INTO %s JSON ?", "{\"K\": 0, \"\\\"Foo\\\"\": 0}");
-        execute("INSERT INTO %s JSON ?", "{\"\\\"k\\\"\": 0, \"\\\"Foo\\\"\": 0}");
-
-        // results should preserve and quote case-sensitive identifiers
-        assertRows(execute("SELECT JSON * FROM %s"), row("{\"k\": 0, \"\\\"Foo\\\"\": 0}"));
-        assertRows(execute("SELECT JSON k, \"Foo\" as foo FROM %s"), row("{\"k\": 0, \"foo\": 0}"));
-        assertRows(execute("SELECT JSON k, \"Foo\" as \"Bar\" FROM %s"), row("{\"k\": 0, \"\\\"Bar\\\"\": 0}"));
-
-        assertInvalid("INSERT INTO %s JSON ?", "{\"k\": 0, \"foo\": 0}");
-        assertInvalid("INSERT INTO %s JSON ?", "{\"k\": 0, \"\\\"foo\\\"\": 0}");
-
-        // user-defined types also need to handle case-sensitivity
-        String typeName = createType("CREATE TYPE %s (a int, \"Foo\" int)");
-        createTable("CREATE TABLE %s (k int primary key, v frozen<" + typeName + ">)");
-
-        execute("INSERT INTO %s JSON ?", "{\"k\": 0, \"v\": {\"a\": 0, \"\\\"Foo\\\"\": 0}}");
-        assertRows(execute("SELECT JSON k, v FROM %s"), row("{\"k\": 0, \"v\": {\"a\": 0, \"\\\"Foo\\\"\": 0}}"));
-
-        execute("INSERT INTO %s JSON ?", "{\"k\": 0, \"v\": {\"A\": 0, \"\\\"Foo\\\"\": 0}}");
-        assertRows(execute("SELECT JSON k, v FROM %s"), row("{\"k\": 0, \"v\": {\"a\": 0, \"\\\"Foo\\\"\": 0}}"));
-    }
-
-    @Test
-    public void testInsertJsonSyntaxWithCollections() throws Throwable
-    {
-        createTable("CREATE TABLE %s (" +
-                "k int PRIMARY KEY, " +
-                "m map<text, boolean>, " +
-                "mf frozen<map<text, boolean>>, " +
-                "s set<int>, " +
-                "sf frozen<set<int>>, " +
-                "l list<int>, " +
-                "lf frozen<list<int>>)");
-
-        // map
-        execute("INSERT INTO %s JSON ?", "{\"k\": 0, \"m\": {\"a\": true, \"b\": false}}");
-        assertRows(execute("SELECT k, m FROM %s"), row(0, map("a", true, "b", false)));
-
-        // frozen map
-        execute("INSERT INTO %s JSON ?", "{\"k\": 0, \"mf\": {\"a\": true, \"b\": false}}");
-        assertRows(execute("SELECT k, mf FROM %s"), row(0, map("a", true, "b", false)));
-
-        // set
-        execute("INSERT INTO %s JSON ?", "{\"k\": 0, \"s\": [3, 1, 2]}");
-        assertRows(execute("SELECT k, s FROM %s"), row(0, set(1, 2, 3)));
-
-        // frozen set
-        execute("INSERT INTO %s JSON ?", "{\"k\": 0, \"sf\": [3, 1, 2]}");
-        assertRows(execute("SELECT k, sf FROM %s"), row(0, set(1, 2, 3)));
-
-        // list
-        execute("INSERT INTO %s JSON ?", "{\"k\": 0, \"l\": [1, 2, 3]}");
-        assertRows(execute("SELECT k, l FROM %s"), row(0, list(1, 2, 3)));
-
-        // frozen list
-        execute("INSERT INTO %s JSON ?", "{\"k\": 0, \"lf\": [1, 2, 3]}");
-        assertRows(execute("SELECT k, lf FROM %s"), row(0, list(1, 2, 3)));
-    }
-
-    @Test
-    public void testInsertJsonSyntaxWithNonNativeMapKeys() throws Throwable
-    {
-        // JSON doesn't allow non-string keys, so we accept string representations of any type as map keys and
-        // return maps with string keys when necessary.
-
-        String typeName = createType("CREATE TYPE %s (a int)");
-        createTable("CREATE TABLE %s (" +
-                "k int PRIMARY KEY, " +
-                "intmap map<int, boolean>, " +
-                "bigintmap map<bigint, boolean>, " +
-                "varintmap map<varint, boolean>, " +
-                "booleanmap map<boolean, boolean>, " +
-                "floatmap map<float, boolean>, " +
-                "doublemap map<double, boolean>, " +
-                "decimalmap map<decimal, boolean>, " +
-                "tuplemap map<frozen<tuple<int, text>>, boolean>, " +
-                "udtmap map<frozen<" + typeName + ">, boolean>, " +
-                "setmap map<frozen<set<int>>, boolean>, " +
-                "listmap map<frozen<list<int>>, boolean>, " +
-                "textsetmap map<frozen<set<text>>, boolean>, " +
-                "nestedsetmap map<frozen<map<set<text>, text>>, boolean>, " +
-                "frozensetmap frozen<map<set<int>, boolean>>)");
-
-        // int keys
-        execute("INSERT INTO %s JSON ?", "{\"k\": 0, \"intmap\": {\"0\": true, \"1\": false}}");
-        assertRows(execute("SELECT JSON k, intmap FROM %s"), row("{\"k\": 0, \"intmap\": {\"0\": true, \"1\": false}}"));
-
-        // bigint keys
-        execute("INSERT INTO %s JSON ?", "{\"k\": 0, \"bigintmap\": {\"0\": true, \"1\": false}}");
-        assertRows(execute("SELECT JSON k, bigintmap FROM %s"), row("{\"k\": 0, \"bigintmap\": {\"0\": true, \"1\": false}}"));
-
-        // varint keys
-        execute("INSERT INTO %s JSON ?", "{\"k\": 0, \"varintmap\": {\"0\": true, \"1\": false}}");
-        assertRows(execute("SELECT JSON k, varintmap FROM %s"), row("{\"k\": 0, \"varintmap\": {\"0\": true, \"1\": false}}"));
-
-        // boolean keys
-        execute("INSERT INTO %s JSON ?", "{\"k\": 0, \"booleanmap\": {\"true\": true, \"false\": false}}");
-        assertRows(execute("SELECT JSON k, booleanmap FROM %s"), row("{\"k\": 0, \"booleanmap\": {\"false\": false, \"true\": true}}"));
-
-        // float keys
-        execute("INSERT INTO %s JSON ?", "{\"k\": 0, \"floatmap\": {\"1.23\": true, \"4.56\": false}}");
-        assertRows(execute("SELECT JSON k, floatmap FROM %s"), row("{\"k\": 0, \"floatmap\": {\"1.23\": true, \"4.56\": false}}"));
-
-        // double keys
-        execute("INSERT INTO %s JSON ?", "{\"k\": 0, \"doublemap\": {\"1.23\": true, \"4.56\": false}}");
-        assertRows(execute("SELECT JSON k, doublemap FROM %s"), row("{\"k\": 0, \"doublemap\": {\"1.23\": true, \"4.56\": false}}"));
-
-        // decimal keys
-        execute("INSERT INTO %s JSON ?", "{\"k\": 0, \"decimalmap\": {\"1.23\": true, \"4.56\": false}}");
-        assertRows(execute("SELECT JSON k, decimalmap FROM %s"), row("{\"k\": 0, \"decimalmap\": {\"1.23\": true, \"4.56\": false}}"));
-
-        // tuple<int, text> keys
-        execute("INSERT INTO %s JSON ?", "{\"k\": 0, \"tuplemap\": {\"[0, \\\"a\\\"]\": true, \"[1, \\\"b\\\"]\": false}}");
-        assertRows(execute("SELECT JSON k, tuplemap FROM %s"), row("{\"k\": 0, \"tuplemap\": {\"[0, \\\"a\\\"]\": true, \"[1, \\\"b\\\"]\": false}}"));
-
-        // UDT keys
-        execute("INSERT INTO %s JSON ?", "{\"k\": 0, \"udtmap\": {\"{\\\"a\\\": 0}\": true, \"{\\\"a\\\": 1}\": false}}");
-        assertRows(execute("SELECT JSON k, udtmap FROM %s"), row("{\"k\": 0, \"udtmap\": {\"{\\\"a\\\": 0}\": true, \"{\\\"a\\\": 1}\": false}}"));
-
-        // set<int> keys
-        execute("INSERT INTO %s JSON ?", "{\"k\": 0, \"setmap\": {\"[0, 1, 2]\": true, \"[3, 4, 5]\": false}}");
-        assertRows(execute("SELECT JSON k, setmap FROM %s"), row("{\"k\": 0, \"setmap\": {\"[0, 1, 2]\": true, \"[3, 4, 5]\": false}}"));
-
-        // list<int> keys
-        execute("INSERT INTO %s JSON ?", "{\"k\": 0, \"listmap\": {\"[0, 1, 2]\": true, \"[3, 4, 5]\": false}}");
-        assertRows(execute("SELECT JSON k, listmap FROM %s"), row("{\"k\": 0, \"listmap\": {\"[0, 1, 2]\": true, \"[3, 4, 5]\": false}}"));
-
-        // set<text> keys
-        execute("INSERT INTO %s JSON ?", "{\"k\": 0, \"textsetmap\": {\"[\\\"0\\\", \\\"1\\\"]\": true, \"[\\\"3\\\", \\\"4\\\"]\": false}}");
-        assertRows(execute("SELECT JSON k, textsetmap FROM %s"), row("{\"k\": 0, \"textsetmap\": {\"[\\\"0\\\", \\\"1\\\"]\": true, \"[\\\"3\\\", \\\"4\\\"]\": false}}"));
-
-        // map<set<text>, text> keys
-        String innerKey1 = "[\"0\", \"1\"]";
-        String fullKey1 = String.format("{\"%s\": \"%s\"}", new String(Json.JSON_STRING_ENCODER.quoteAsString(innerKey1)), "a");
-        String stringKey1 = new String(Json.JSON_STRING_ENCODER.quoteAsString(fullKey1));
-        String innerKey2 = "[\"3\", \"4\"]";
-        String fullKey2 = String.format("{\"%s\": \"%s\"}", new String(Json.JSON_STRING_ENCODER.quoteAsString(innerKey2)), "b");
-        String stringKey2 = new String(Json.JSON_STRING_ENCODER.quoteAsString(fullKey2));
-        execute("INSERT INTO %s JSON ?", "{\"k\": 0, \"nestedsetmap\": {\"" + stringKey1 + "\": true, \"" + stringKey2 + "\": false}}");
-        assertRows(execute("SELECT JSON k, nestedsetmap FROM %s"), row("{\"k\": 0, \"nestedsetmap\": {\"" + stringKey1 + "\": true, \"" + stringKey2 + "\": false}}"));
-
-        // set<int> keys in a frozen map
-        execute("INSERT INTO %s JSON ?", "{\"k\": 0, \"frozensetmap\": {\"[0, 1, 2]\": true, \"[3, 4, 5]\": false}}");
-        assertRows(execute("SELECT JSON k, frozensetmap FROM %s"), row("{\"k\": 0, \"frozensetmap\": {\"[0, 1, 2]\": true, \"[3, 4, 5]\": false}}"));
-    }
-
-    @Test
-    public void testInsertJsonSyntaxWithTuplesAndUDTs() throws Throwable
-    {
-        String typeName = createType("CREATE TYPE %s (a int, b frozen<set<int>>, c tuple<int, int>)");
-        createTable("CREATE TABLE %s (" +
-                "k int PRIMARY KEY, " +
-                "a frozen<" + typeName + ">, " +
-                "b tuple<int, boolean>)");
-
-        execute("INSERT INTO %s JSON ?", "{\"k\": 0, \"a\": {\"a\": 0, \"b\": [1, 2, 3], \"c\": [0, 1]}, \"b\": [0, true]}");
-        assertRows(execute("SELECT k, a.a, a.b, a.c, b FROM %s"), row(0, 0, set(1, 2, 3), tuple(0, 1), tuple(0, true)));
-
-        execute("INSERT INTO %s JSON ?", "{\"k\": 0, \"a\": {\"a\": 0, \"b\": [1, 2, 3], \"c\": null}, \"b\": null}");
-        assertRows(execute("SELECT k, a.a, a.b, a.c, b FROM %s"), row(0, 0, set(1, 2, 3), null, null));
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/01115f72/test/unit/org/apache/cassandra/cql3/ModificationTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/ModificationTest.java b/test/unit/org/apache/cassandra/cql3/ModificationTest.java
deleted file mode 100644
index 6397a15..0000000
--- a/test/unit/org/apache/cassandra/cql3/ModificationTest.java
+++ /dev/null
@@ -1,112 +0,0 @@
-/*
- * 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;
-
-import org.junit.Test;
-
-public class ModificationTest extends CQLTester
-{
-    @Test
-    public void testModificationWithUnset() throws Throwable
-    {
-        createTable("CREATE TABLE %s (k int PRIMARY KEY, s text, i int)");
-
-        // insert using nulls
-        execute("INSERT INTO %s (k, s, i) VALUES (10, ?, ?)", "text", 10);
-        execute("INSERT INTO %s (k, s, i) VALUES (10, ?, ?)", null, null);
-        assertRows(execute("SELECT s, i FROM %s WHERE k = 10"),
-                row(null, null) // sending null deletes the data
-        );
-        // insert using UNSET
-        execute("INSERT INTO %s (k, s, i) VALUES (11, ?, ?)", "text", 10);
-        execute("INSERT INTO %s (k, s, i) VALUES (11, ?, ?)", unset(), unset());
-        assertRows(execute("SELECT s, i FROM %s WHERE k=11"),
-                row("text", 10) // unset columns does not delete the existing data
-        );
-
-        assertInvalidMessage("Invalid unset value for column k", "UPDATE %s SET i = 0 WHERE k = ?", unset());
-        assertInvalidMessage("Invalid unset value for column k", "DELETE FROM %s WHERE k = ?", unset());
-        assertInvalidMessage("Invalid unset value for argument in call to function blobasint", "SELECT * FROM %s WHERE k = blobAsInt(?)", unset());
-    }
-
-    @Test
-    public void testTtlWithUnset() throws Throwable
-    {
-        createTable("CREATE TABLE %s (k int PRIMARY KEY, i int)");
-        execute("INSERT INTO %s (k, i) VALUES (1, 1) USING TTL ?", unset()); // treat as 'unlimited'
-        assertRows(execute("SELECT ttl(i) FROM %s"),
-                row(new Object[] {null})
-        );
-    }
-
-    @Test
-    public void testTimestampWithUnset() throws Throwable
-    {
-        createTable("CREATE TABLE %s (k int PRIMARY KEY, i int)");
-        execute("INSERT INTO %s (k, i) VALUES (1, 1) USING TIMESTAMP ?", unset()); // treat as 'now'
-    }
-
-    @Test
-    public void testCounterUpdatesWithUnset() throws Throwable
-    {
-        createTable("CREATE TABLE %s (k int PRIMARY KEY, c counter)");
-
-        // set up
-        execute("UPDATE %s SET c = c + 1 WHERE k = 10");
-        assertRows(execute("SELECT c FROM %s WHERE k = 10"),
-                row(1L)
-        );
-        // increment
-        execute("UPDATE %s SET c = c + ? WHERE k = 10", 1L);
-        assertRows(execute("SELECT c FROM %s WHERE k = 10"),
-                row(2L)
-        );
-        execute("UPDATE %s SET c = c + ? WHERE k = 10", unset());
-        assertRows(execute("SELECT c FROM %s WHERE k = 10"),
-                row(2L) // no change to the counter value
-        );
-        // decrement
-        execute("UPDATE %s SET c = c - ? WHERE k = 10", 1L);
-        assertRows(execute("SELECT c FROM %s WHERE k = 10"),
-                row(1L)
-        );
-        execute("UPDATE %s SET c = c - ? WHERE k = 10", unset());
-        assertRows(execute("SELECT c FROM %s WHERE k = 10"),
-                row(1L) // no change to the counter value
-        );
-    }
-
-    @Test
-    public void testBatchWithUnset() throws Throwable
-    {
-        createTable("CREATE TABLE %s (k int PRIMARY KEY, s text, i int)");
-
-        // test batch and update
-        String qualifiedTable = keyspace() + "." + currentTable();
-        execute("BEGIN BATCH " +
-                 "INSERT INTO %s (k, s, i) VALUES (100, 'batchtext', 7); " +
-                 "INSERT INTO " + qualifiedTable + " (k, s, i) VALUES (111, 'batchtext', 7); " +
-             "UPDATE " + qualifiedTable + " SET s=?, i=? WHERE k = 100; " +
-                 "UPDATE " + qualifiedTable + " SET s=?, i=? WHERE k=111; " +
-                 "APPLY BATCH;", null, unset(), unset(), null);
-        assertRows(execute("SELECT k, s, i FROM %s where k in (100,111)"),
-            row(100, null, 7),
-            row(111, "batchtext", null)
-        );
-    }
-}


[22/32] cassandra git commit: 2.2 commit for CASSANDRA-9160

Posted by jm...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/01115f72/test/unit/org/apache/cassandra/cql3/UseStatementTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/UseStatementTest.java b/test/unit/org/apache/cassandra/cql3/UseStatementTest.java
deleted file mode 100644
index 66b4b42..0000000
--- a/test/unit/org/apache/cassandra/cql3/UseStatementTest.java
+++ /dev/null
@@ -1,29 +0,0 @@
-/*
- * 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;
-
-import org.junit.Test;
-
-public class UseStatementTest extends CQLTester
-{
-    @Test
-    public void testUseStatementWithBindVariable() throws Throwable
-    {
-        assertInvalidSyntaxMessage("Bind variables cannot be used for keyspace names", "USE ?");
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/01115f72/test/unit/org/apache/cassandra/cql3/UserTypesTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/UserTypesTest.java b/test/unit/org/apache/cassandra/cql3/UserTypesTest.java
deleted file mode 100644
index bfd3e9f..0000000
--- a/test/unit/org/apache/cassandra/cql3/UserTypesTest.java
+++ /dev/null
@@ -1,334 +0,0 @@
-/*
- * 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;
-
-import org.junit.Test;
-
-public class UserTypesTest extends CQLTester
-{
-    @Test
-    public void testInvalidField() throws Throwable
-    {
-        String myType = createType("CREATE TYPE %s (f int)");
-        createTable("CREATE TABLE %s (k int PRIMARY KEY, v frozen<" + myType + ">)");
-
-        // 's' is not a field of myType
-        assertInvalid("INSERT INTO %s (k, v) VALUES (?, {s : ?})", 0, 1);
-    }
-
-    @Test
-    public void testCassandra8105() throws Throwable
-    {
-        String ut1 = createType("CREATE TYPE %s (a int, b int)");
-        String ut2 = createType("CREATE TYPE %s (j frozen<" + KEYSPACE + "." + ut1 + ">, k int)");
-        createTable("CREATE TABLE %s (x int PRIMARY KEY, y set<frozen<" + KEYSPACE + "." + ut2 + ">>)");
-        execute("INSERT INTO %s (x, y) VALUES (1, { { k: 1 } })");
-
-        String ut3 = createType("CREATE TYPE %s (a int, b int)");
-        String ut4 = createType("CREATE TYPE %s (j frozen<" + KEYSPACE + "." + ut3 + ">, k int)");
-        createTable("CREATE TABLE %s (x int PRIMARY KEY, y list<frozen<" + KEYSPACE + "." + ut4 + ">>)");
-        execute("INSERT INTO %s (x, y) VALUES (1, [ { k: 1 } ])");
-
-        String ut5 = createType("CREATE TYPE %s (a int, b int)");
-        String ut6 = createType("CREATE TYPE %s (i int, j frozen<" + KEYSPACE + "." + ut5 + ">)");
-        createTable("CREATE TABLE %s (x int PRIMARY KEY, y set<frozen<" + KEYSPACE + "." + ut6 + ">>)");
-        execute("INSERT INTO %s (x, y) VALUES (1, { { i: 1 } })");
-    }
-
-    @Test
-    public void testFor7684() throws Throwable
-    {
-        String myType = createType("CREATE TYPE %s (x double)");
-        createTable("CREATE TABLE %s (k int, v frozen<" + myType + ">, b boolean static, PRIMARY KEY (k, v))");
-
-        execute("INSERT INTO %s(k, v) VALUES (?, {x:?})", 1, -104.99251);
-        execute("UPDATE %s SET b = ? WHERE k = ?", true, 1);
-
-        assertRows(execute("SELECT v.x FROM %s WHERE k = ? AND v = {x:?}", 1, -104.99251),
-            row(-104.99251)
-        );
-
-        flush();
-
-        assertRows(execute("SELECT v.x FROM %s WHERE k = ? AND v = {x:?}", 1, -104.99251),
-            row(-104.99251)
-        );
-    }
-
-    @Test
-    public void testCreateInvalidTablesWithUDT() throws Throwable
-    {
-        String myType = createType("CREATE TYPE %s (f int)");
-
-        // Using a UDT without frozen shouldn't work
-        assertInvalidMessage("Non-frozen User-Defined types are not supported, please use frozen<>",
-                             "CREATE TABLE " + KEYSPACE + ".wrong (k int PRIMARY KEY, v " + KEYSPACE + '.' + myType + ")");
-
-        assertInvalidMessage("Statement on keyspace " + KEYSPACE + " cannot refer to a user type in keyspace otherkeyspace;" +
-                             " user types can only be used in the keyspace they are defined in",
-                             "CREATE TABLE " + KEYSPACE + ".wrong (k int PRIMARY KEY, v frozen<otherKeyspace.myType>)");
-
-        assertInvalidMessage("Unknown type " + KEYSPACE + ".unknowntype",
-                             "CREATE TABLE " + KEYSPACE + ".wrong (k int PRIMARY KEY, v frozen<" + KEYSPACE + '.' + "unknownType>)");
-    }
-
-    @Test
-    public void testAlterUDT() throws Throwable
-    {
-        String myType = KEYSPACE + '.' + createType("CREATE TYPE %s (a int)");
-        createTable("CREATE TABLE %s (a int PRIMARY KEY, b frozen<" + myType + ">)");
-        execute("INSERT INTO %s (a, b) VALUES (1, {a: 1})");
-
-        assertRows(execute("SELECT b.a FROM %s"), row(1));
-
-        flush();
-
-        execute("ALTER TYPE " + myType + " ADD b int");
-        execute("INSERT INTO %s (a, b) VALUES (2, {a: 2, b :2})");
-
-        assertRows(execute("SELECT b.a, b.b FROM %s"),
-                   row(1, null),
-                   row(2, 2));
-
-        flush();
-
-        assertRows(execute("SELECT b.a, b.b FROM %s"),
-                   row(1, null),
-                   row(2, 2));
-    }
-
-    @Test
-    public void testUDTWithUnsetValues() throws Throwable
-    {
-        // set up
-        String myType = createType("CREATE TYPE %s (x int, y int)");
-        String myOtherType = createType("CREATE TYPE %s (a frozen<" + myType + ">)");
-        createTable("CREATE TABLE %s (k int PRIMARY KEY, v frozen<" + myType + ">, z frozen<" + myOtherType + ">)");
-
-        assertInvalidMessage("Invalid unset value for field 'y' of user defined type " + myType,
-                "INSERT INTO %s (k, v) VALUES (10, {x:?, y:?})", 1, unset());
-
-        assertInvalidMessage("Invalid unset value for field 'y' of user defined type " + myType,
-                "INSERT INTO %s (k, v, z) VALUES (10, {x:?, y:?}, {a:{x: ?, y: ?}})", 1, 1, 1, unset());
-    }
-
-    @Test
-    public void testAlteringUserTypeNestedWithinMap() throws Throwable
-    {
-        // test frozen and non-frozen collections
-        String[] columnTypePrefixes = {"frozen<map<text, ", "map<text, frozen<"};
-        for (String columnTypePrefix : columnTypePrefixes)
-        {
-            String ut1 = createType("CREATE TYPE %s (a int)");
-            String columnType = columnTypePrefix + KEYSPACE + "." + ut1 + ">>";
-
-            createTable("CREATE TABLE %s (x int PRIMARY KEY, y " + columnType + ")");
-
-            execute("INSERT INTO %s (x, y) VALUES(1, {'firstValue':{a:1}})");
-            assertRows(execute("SELECT * FROM %s"), row(1, map("firstValue", userType(1))));
-            flush();
-
-            execute("ALTER TYPE " + KEYSPACE + "." + ut1 + " ADD b int");
-            execute("INSERT INTO %s (x, y) VALUES(2, {'secondValue':{a:2, b:2}})");
-            execute("INSERT INTO %s (x, y) VALUES(3, {'thirdValue':{a:3}})");
-            execute("INSERT INTO %s (x, y) VALUES(4, {'fourthValue':{b:4}})");
-
-            assertRows(execute("SELECT * FROM %s"),
-                    row(1, map("firstValue", userType(1))),
-                    row(2, map("secondValue", userType(2, 2))),
-                    row(3, map("thirdValue", userType(3, null))),
-                    row(4, map("fourthValue", userType(null, 4))));
-
-            flush();
-
-            assertRows(execute("SELECT * FROM %s"),
-                    row(1, map("firstValue", userType(1))),
-                    row(2, map("secondValue", userType(2, 2))),
-                    row(3, map("thirdValue", userType(3, null))),
-                    row(4, map("fourthValue", userType(null, 4))));
-        }
-    }
-
-    @Test
-    public void testAlteringUserTypeNestedWithinSet() throws Throwable
-    {
-        // test frozen and non-frozen collections
-        String[] columnTypePrefixes = {"frozen<set<", "set<frozen<"};
-        for (String columnTypePrefix : columnTypePrefixes)
-        {
-            String ut1 = createType("CREATE TYPE %s (a int)");
-            String columnType = columnTypePrefix + KEYSPACE + "." + ut1 + ">>";
-
-            createTable("CREATE TABLE %s (x int PRIMARY KEY, y " + columnType + ")");
-
-            execute("INSERT INTO %s (x, y) VALUES(1, {1} )");
-            assertRows(execute("SELECT * FROM %s"), row(1, set(userType(1))));
-            flush();
-
-            execute("ALTER TYPE " + KEYSPACE + "." + ut1 + " ADD b int");
-            execute("INSERT INTO %s (x, y) VALUES(2, {{a:2, b:2}})");
-            execute("INSERT INTO %s (x, y) VALUES(3, {{a:3}})");
-            execute("INSERT INTO %s (x, y) VALUES(4, {{b:4}})");
-
-            assertRows(execute("SELECT * FROM %s"),
-                    row(1, set(userType(1))),
-                    row(2, set(userType(2, 2))),
-                    row(3, set(userType(3, null))),
-                    row(4, set(userType(null, 4))));
-
-            flush();
-
-            assertRows(execute("SELECT * FROM %s"),
-                    row(1, set(userType(1))),
-                    row(2, set(userType(2, 2))),
-                    row(3, set(userType(3, null))),
-                    row(4, set(userType(null, 4))));
-        }
-    }
-
-    @Test
-    public void testAlteringUserTypeNestedWithinList() throws Throwable
-    {
-        // test frozen and non-frozen collections
-        String[] columnTypePrefixes = {"frozen<list<", "list<frozen<"};
-        for (String columnTypePrefix : columnTypePrefixes)
-        {
-            String ut1 = createType("CREATE TYPE %s (a int)");
-            String columnType = columnTypePrefix + KEYSPACE + "." + ut1 + ">>";
-
-            createTable("CREATE TABLE %s (x int PRIMARY KEY, y " + columnType + ")");
-
-            execute("INSERT INTO %s (x, y) VALUES(1, [1] )");
-            assertRows(execute("SELECT * FROM %s"), row(1, list(userType(1))));
-            flush();
-
-            execute("ALTER TYPE " + KEYSPACE + "." + ut1 + " ADD b int");
-            execute("INSERT INTO %s (x, y) VALUES(2, [{a:2, b:2}])");
-            execute("INSERT INTO %s (x, y) VALUES(3, [{a:3}])");
-            execute("INSERT INTO %s (x, y) VALUES(4, [{b:4}])");
-
-            assertRows(execute("SELECT * FROM %s"),
-                    row(1, list(userType(1))),
-                    row(2, list(userType(2, 2))),
-                    row(3, list(userType(3, null))),
-                    row(4, list(userType(null, 4))));
-
-            flush();
-
-            assertRows(execute("SELECT * FROM %s"),
-                    row(1, list(userType(1))),
-                    row(2, list(userType(2, 2))),
-                    row(3, list(userType(3, null))),
-                    row(4, list(userType(null, 4))));
-        }
-    }
-
-    @Test
-    public void testAlteringUserTypeNestedWithinTuple() throws Throwable
-    {
-        String type = createType("CREATE TYPE %s (a int, b int)");
-
-        createTable("CREATE TABLE %s (a int PRIMARY KEY, b frozen<tuple<int, " + KEYSPACE + "." + type + ">>)");
-
-        execute("INSERT INTO %s (a, b) VALUES(1, (1, {a:1, b:1}))");
-        assertRows(execute("SELECT * FROM %s"), row(1, tuple(1, userType(1, 1))));
-        flush();
-
-        execute("ALTER TYPE " + KEYSPACE + "." + type + " ADD c int");
-        execute("INSERT INTO %s (a, b) VALUES(2, (2, {a: 2, b: 2, c: 2}))");
-        execute("INSERT INTO %s (a, b) VALUES(3, (3, {a: 3, b: 3}))");
-        execute("INSERT INTO %s (a, b) VALUES(4, (4, {b:4}))");
-
-        assertRows(execute("SELECT * FROM %s"),
-                   row(1, tuple(1, userType(1, 1))),
-                   row(2, tuple(2, userType(2, 2, 2))),
-                   row(3, tuple(3, userType(3, 3, null))),
-                   row(4, tuple(4, userType(null, 4, null))));
-
-        flush();
-
-        assertRows(execute("SELECT * FROM %s"),
-                   row(1, tuple(1, userType(1, 1))),
-                   row(2, tuple(2, userType(2, 2, 2))),
-                   row(3, tuple(3, userType(3, 3, null))),
-                   row(4, tuple(4, userType(null, 4, null))));
-    }
-
-    @Test
-    public void testAlteringUserTypeNestedWithinNestedTuple() throws Throwable
-    {
-        String type = createType("CREATE TYPE %s (a int, b int)");
-
-        createTable("CREATE TABLE %s (a int PRIMARY KEY, b frozen<tuple<int, tuple<int, " + KEYSPACE + "." + type + ">>>)");
-
-        execute("INSERT INTO %s (a, b) VALUES(1, (1, (1, {a:1, b:1})))");
-        assertRows(execute("SELECT * FROM %s"), row(1, tuple(1, tuple(1, userType(1, 1)))));
-        flush();
-
-        execute("ALTER TYPE " + KEYSPACE + "." + type + " ADD c int");
-        execute("INSERT INTO %s (a, b) VALUES(2, (2, (1, {a: 2, b: 2, c: 2})))");
-        execute("INSERT INTO %s (a, b) VALUES(3, (3, (1, {a: 3, b: 3})))");
-        execute("INSERT INTO %s (a, b) VALUES(4, (4, (1, {b:4})))");
-
-        assertRows(execute("SELECT * FROM %s"),
-                   row(1, tuple(1, tuple(1, userType(1, 1)))),
-                   row(2, tuple(2, tuple(1, userType(2, 2, 2)))),
-                   row(3, tuple(3, tuple(1, userType(3, 3, null)))),
-                   row(4, tuple(4, tuple(1, userType(null, 4, null)))));
-
-        flush();
-
-        assertRows(execute("SELECT * FROM %s"),
-                   row(1, tuple(1, tuple(1, userType(1, 1)))),
-                   row(2, tuple(2, tuple(1, userType(2, 2, 2)))),
-                   row(3, tuple(3, tuple(1, userType(3, 3, null)))),
-                   row(4, tuple(4, tuple(1, userType(null, 4, null)))));
-    }
-
-    @Test
-    public void testAlteringUserTypeNestedWithinUserType() throws Throwable
-    {
-        String type = createType("CREATE TYPE %s (a int, b int)");
-        String otherType = createType("CREATE TYPE %s (x frozen<" + KEYSPACE + "." + type + ">)");
-
-        createTable("CREATE TABLE %s (a int PRIMARY KEY, b frozen<" + KEYSPACE + "." + otherType + ">)");
-
-        execute("INSERT INTO %s (a, b) VALUES(1, {x: {a:1, b:1}})");
-        assertRows(execute("SELECT b.x.a, b.x.b FROM %s"), row(1, 1));
-        flush();
-
-        execute("ALTER TYPE " + KEYSPACE + "." + type + " ADD c int");
-        execute("INSERT INTO %s (a, b) VALUES(2, {x: {a: 2, b: 2, c: 2}})");
-        execute("INSERT INTO %s (a, b) VALUES(3, {x: {a: 3, b: 3}})");
-        execute("INSERT INTO %s (a, b) VALUES(4, {x: {b:4}})");
-
-        assertRows(execute("SELECT b.x.a, b.x.b, b.x.c FROM %s"),
-                   row(1, 1, null),
-                   row(2, 2, 2),
-                   row(3, 3, null),
-                   row(null, 4, null));
-
-        flush();
-
-        assertRows(execute("SELECT b.x.a, b.x.b, b.x.c FROM %s"),
-                   row(1, 1, null),
-                   row(2, 2, 2),
-                   row(3, 3, null),
-                   row(null, 4, null));
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/01115f72/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 5bacf0d..2f1d361 100644
--- a/test/unit/org/apache/cassandra/cql3/selection/SelectionColumnMappingTest.java
+++ b/test/unit/org/apache/cassandra/cql3/selection/SelectionColumnMappingTest.java
@@ -3,13 +3,16 @@ package org.apache.cassandra.cql3.selection;
 import java.util.Collections;
 
 import com.google.common.collect.ImmutableList;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.cql3.*;
 import org.apache.cassandra.cql3.statements.SelectStatement;
 import org.apache.cassandra.db.marshal.*;
+import org.apache.cassandra.dht.ByteOrderedPartitioner;
 import org.apache.cassandra.exceptions.RequestValidationException;
 import org.apache.cassandra.service.ClientState;
 import org.apache.cassandra.utils.ByteBufferUtil;
@@ -25,6 +28,12 @@ public class SelectionColumnMappingTest extends CQLTester
     UserType userType;
     String functionName;
 
+    @BeforeClass
+    public static void setUpClass()
+    {
+        DatabaseDescriptor.setPartitioner(ByteOrderedPartitioner.instance);
+    }
+
     @Test
     public void testSelectionColumnMapping() throws Throwable
     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/01115f72/test/unit/org/apache/cassandra/cql3/validation/entities/CollectionsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/CollectionsTest.java b/test/unit/org/apache/cassandra/cql3/validation/entities/CollectionsTest.java
new file mode 100644
index 0000000..2e72c39
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/CollectionsTest.java
@@ -0,0 +1,588 @@
+/*
+ * 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.validation.entities;
+
+import java.util.UUID;
+
+import org.junit.Test;
+
+import org.apache.cassandra.cql3.CQLTester;
+
+public class CollectionsTest extends CQLTester
+{
+    @Test
+    public void testMapBulkRemoval() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, m map<text, text>)");
+
+        execute("INSERT INTO %s(k, m) VALUES (?, ?)", 0, map("k1", "v1", "k2", "v2", "k3", "v3"));
+
+        assertRows(execute("SELECT * FROM %s"),
+            row(0, map("k1", "v1", "k2", "v2", "k3", "v3"))
+        );
+
+        execute("UPDATE %s SET m = m - ? WHERE k = ?", set("k2"), 0);
+
+        assertRows(execute("SELECT * FROM %s"),
+            row(0, map("k1", "v1", "k3", "v3"))
+        );
+
+        execute("UPDATE %s SET m = m + ?, m = m - ? WHERE k = ?", map("k4", "v4"), set("k3"), 0);
+
+        assertRows(execute("SELECT * FROM %s"),
+            row(0, map("k1", "v1", "k4", "v4"))
+        );
+    }
+
+    @Test
+    public void testInvalidCollectionsMix() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, l list<text>, s set<text>, m map<text, text>)");
+
+        // Note: we force the non-prepared form for some of those tests because a list and a set
+        // have the same serialized format in practice and CQLTester don't validate that the type
+        // of what's passed as a value in the prepared case, so the queries would work (which is ok,
+        // CQLTester is just a "dumb" client).
+
+        assertInvalid("UPDATE %s SET l = l + { 'a', 'b' } WHERE k = 0");
+        assertInvalid("UPDATE %s SET l = l - { 'a', 'b' } WHERE k = 0");
+        assertInvalid("UPDATE %s SET l = l + ? WHERE k = 0", map("a", "b", "c", "d"));
+        assertInvalid("UPDATE %s SET l = l - ? WHERE k = 0", map("a", "b", "c", "d"));
+
+        assertInvalid("UPDATE %s SET s = s + [ 'a', 'b' ] WHERE k = 0");
+        assertInvalid("UPDATE %s SET s = s - [ 'a', 'b' ] WHERE k = 0");
+        assertInvalid("UPDATE %s SET s = s + ? WHERE k = 0", map("a", "b", "c", "d"));
+        assertInvalid("UPDATE %s SET s = s - ? WHERE k = 0", map("a", "b", "c", "d"));
+
+        assertInvalid("UPDATE %s SET m = m + ? WHERE k = 0", list("a", "b"));
+        assertInvalid("UPDATE %s SET m = m - [ 'a', 'b' ] WHERE k = 0");
+        assertInvalid("UPDATE %s SET m = m + ? WHERE k = 0", set("a", "b"));
+        assertInvalid("UPDATE %s SET m = m - ? WHERE k = 0", map("a", "b", "c", "d"));
+    }
+
+    @Test
+    public void testSets() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, s set<text>)");
+
+        execute("INSERT INTO %s(k, s) VALUES (0, ?)", set("v1", "v2", "v3", "v4"));
+
+        assertRows(execute("SELECT s FROM %s WHERE k = 0"),
+            row(set("v1", "v2", "v3", "v4"))
+        );
+
+        execute("DELETE s[?] FROM %s WHERE k = 0", "v1");
+
+        assertRows(execute("SELECT s FROM %s WHERE k = 0"),
+                   row(set("v2", "v3", "v4"))
+        );
+
+        // Full overwrite
+        execute("UPDATE %s SET s = ? WHERE k = 0", set("v6", "v5"));
+
+        assertRows(execute("SELECT s FROM %s WHERE k = 0"),
+                   row(set("v5", "v6"))
+        );
+
+        execute("UPDATE %s SET s = s + ? WHERE k = 0", set("v7"));
+
+        assertRows(execute("SELECT s FROM %s WHERE k = 0"),
+                   row(set("v5", "v6", "v7"))
+        );
+
+        execute("UPDATE %s SET s = s - ? WHERE k = 0", set("v6", "v5"));
+
+        assertRows(execute("SELECT s FROM %s WHERE k = 0"),
+                   row(set("v7"))
+        );
+
+        execute("DELETE s[?] FROM %s WHERE k = 0", set("v7"));
+
+        // Deleting an element that does not exist will succeed
+        execute("DELETE s[?] FROM %s WHERE k = 0", set("v7"));
+
+        execute("DELETE s FROM %s WHERE k = 0");
+
+        assertRows(execute("SELECT s FROM %s WHERE k = 0"),
+                   row((Object) null)
+        );
+    }
+
+    @Test
+    public void testMaps() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, m map<text, int>)");
+
+        execute("INSERT INTO %s(k, m) VALUES (0, ?)", map("v1", 1, "v2", 2));
+
+        assertRows(execute("SELECT m FROM %s WHERE k = 0"),
+            row(map("v1", 1, "v2", 2))
+        );
+
+        execute("UPDATE %s SET m[?] = ?, m[?] = ? WHERE k = 0", "v3", 3, "v4", 4);
+
+        assertRows(execute("SELECT m FROM %s WHERE k = 0"),
+            row(map("v1", 1, "v2", 2, "v3", 3, "v4", 4))
+        );
+
+        execute("DELETE m[?] FROM %s WHERE k = 0", "v1");
+
+        assertRows(execute("SELECT m FROM %s WHERE k = 0"),
+            row(map("v2", 2, "v3", 3, "v4", 4))
+        );
+
+        // Full overwrite
+        execute("UPDATE %s SET m = ? WHERE k = 0", map("v6", 6, "v5", 5));
+
+        assertRows(execute("SELECT m FROM %s WHERE k = 0"),
+                   row(map("v5", 5, "v6", 6))
+        );
+
+        execute("UPDATE %s SET m = m + ? WHERE k = 0", map("v7", 7));
+
+        assertRows(execute("SELECT m FROM %s WHERE k = 0"),
+                   row(map("v5", 5, "v6", 6, "v7", 7))
+        );
+
+        execute("DELETE m[?] FROM %s WHERE k = 0", "v7");
+
+        assertRows(execute("SELECT m FROM %s WHERE k = 0"),
+                   row(map("v5", 5, "v6", 6))
+        );
+
+        execute("DELETE m[?] FROM %s WHERE k = 0", "v6");
+
+        assertRows(execute("SELECT m FROM %s WHERE k = 0"),
+                   row(map("v5", 5))
+        );
+
+        execute("DELETE m[?] FROM %s WHERE k = 0", "v5");
+
+        assertRows(execute("SELECT m FROM %s WHERE k = 0"),
+                   row((Object) null)
+        );
+
+        // Deleting a non-existing key should succeed
+        execute("DELETE m[?] FROM %s WHERE k = 0", "v5");
+
+        assertRows(execute("SELECT m FROM %s WHERE k = 0"),
+                   row((Object) null)
+        );
+
+        // The empty map is parsed as an empty set (because we don't have enough info at parsing
+        // time when we see a {}) and special cased later. This test checks this work properly
+        execute("UPDATE %s SET m = {} WHERE k = 0");
+
+        assertRows(execute("SELECT m FROM %s WHERE k = 0"),
+            row((Object)null)
+        );
+    }
+
+    @Test
+    public void testLists() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, l list<text>)");
+
+        execute("INSERT INTO %s(k, l) VALUES (0, ?)", list("v1", "v2", "v3"));
+
+        assertRows(execute("SELECT l FROM %s WHERE k = 0"), row(list("v1", "v2", "v3")));
+
+        execute("DELETE l[?] FROM %s WHERE k = 0", 1);
+
+        assertRows(execute("SELECT l FROM %s WHERE k = 0"), row(list("v1", "v3")));
+
+        execute("UPDATE %s SET l[?] = ? WHERE k = 0", 1, "v4");
+
+        assertRows(execute("SELECT l FROM %s WHERE k = 0"), row(list("v1", "v4")));
+
+        // Full overwrite
+        execute("UPDATE %s SET l = ? WHERE k = 0", list("v6", "v5"));
+
+        assertRows(execute("SELECT l FROM %s WHERE k = 0"), row(list("v6", "v5")));
+
+        execute("UPDATE %s SET l = l + ? WHERE k = 0", list("v7", "v8"));
+
+        assertRows(execute("SELECT l FROM %s WHERE k = 0"), row(list("v6", "v5", "v7", "v8")));
+
+        execute("UPDATE %s SET l = ? + l WHERE k = 0", list("v9"));
+
+        assertRows(execute("SELECT l FROM %s WHERE k = 0"), row(list("v9", "v6", "v5", "v7", "v8")));
+
+        execute("UPDATE %s SET l = l - ? WHERE k = 0", list("v5", "v8"));
+
+        assertRows(execute("SELECT l FROM %s WHERE k = 0"), row(list("v9", "v6", "v7")));
+
+        execute("DELETE l FROM %s WHERE k = 0");
+
+        assertRows(execute("SELECT l FROM %s WHERE k = 0"), row((Object) null));
+
+        assertInvalidMessage("Attempted to delete an element from a list which is null",
+                             "DELETE l[0] FROM %s WHERE k=0 ");
+
+        assertInvalidMessage("Attempted to set an element on a list which is null",
+                             "UPDATE %s SET l[0] = ? WHERE k=0", list("v10"));
+
+        execute("UPDATE %s SET l = l - ? WHERE k=0 ", list("v11"));
+
+        assertRows(execute("SELECT l FROM %s WHERE k = 0"), row((Object) null));
+    }
+
+    @Test
+    public void testMapWithUnsetValues() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, m map<text,text>)");
+        // set up
+        Object m = map("k", "v");
+        execute("INSERT INTO %s (k, m) VALUES (10, ?)", m);
+        assertRows(execute("SELECT m FROM %s WHERE k = 10"),
+                   row(m)
+        );
+
+        // test putting an unset map, should not delete the contents
+        execute("INSERT INTO %s (k, m) VALUES (10, ?)", unset());
+        assertRows(execute("SELECT m FROM %s WHERE k = 10"),
+                   row(m)
+        );
+        // test unset variables in a map update operaiotn, should not delete the contents
+        execute("UPDATE %s SET m['k'] = ? WHERE k = 10", unset());
+        assertRows(execute("SELECT m FROM %s WHERE k = 10"),
+                   row(m)
+        );
+        assertInvalidMessage("Invalid unset map key", "UPDATE %s SET m[?] = 'foo' WHERE k = 10", unset());
+
+        // test unset value for map key
+        assertInvalidMessage("Invalid unset map key", "DELETE m[?] FROM %s WHERE k = 10", unset());
+    }
+
+    @Test
+    public void testListWithUnsetValues() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, l list<text>)");
+        // set up
+        Object l = list("foo", "foo");
+        execute("INSERT INTO %s (k, l) VALUES (10, ?)", l);
+        assertRows(execute("SELECT l FROM %s WHERE k = 10"),
+                   row(l)
+        );
+
+        // replace list with unset value
+        execute("INSERT INTO %s (k, l) VALUES (10, ?)", unset());
+        assertRows(execute("SELECT l FROM %s WHERE k = 10"),
+                   row(l)
+        );
+
+        // add to position
+        execute("UPDATE %s SET l[1] = ? WHERE k = 10", unset());
+        assertRows(execute("SELECT l FROM %s WHERE k = 10"),
+                   row(l)
+        );
+
+        // set in index
+        assertInvalidMessage("Invalid unset value for list index", "UPDATE %s SET l[?] = 'foo' WHERE k = 10", unset());
+
+        // remove element by index
+        execute("DELETE l[?] FROM %s WHERE k = 10", unset());
+        assertRows(execute("SELECT l FROM %s WHERE k = 10"),
+                   row(l)
+        );
+
+        // remove all occurrences of element
+        execute("UPDATE %s SET l = l - ? WHERE k = 10", unset());
+        assertRows(execute("SELECT l FROM %s WHERE k = 10"),
+                   row(l)
+        );
+
+        // select with in clause
+        assertInvalidMessage("Invalid unset value for column k", "SELECT * FROM %s WHERE k IN ?", unset());
+        assertInvalidMessage("Invalid unset value for column k", "SELECT * FROM %s WHERE k IN (?)", unset());
+    }
+
+    @Test
+    public void testSetWithUnsetValues() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, s set<text>)");
+
+        Object s = set("bar", "baz", "foo");
+        execute("INSERT INTO %s (k, s) VALUES (10, ?)", s);
+        assertRows(execute("SELECT s FROM %s WHERE k = 10"),
+                   row(s)
+        );
+
+        // replace set with unset value
+        execute("INSERT INTO %s (k, s) VALUES (10, ?)", unset());
+        assertRows(execute("SELECT s FROM %s WHERE k = 10"),
+                   row(s)
+        );
+
+        // add to set
+        execute("UPDATE %s SET s = s + ? WHERE k = 10", unset());
+        assertRows(execute("SELECT s FROM %s WHERE k = 10"),
+                   row(s)
+        );
+
+        // remove all occurrences of element
+        execute("UPDATE %s SET s = s - ? WHERE k = 10", unset());
+        assertRows(execute("SELECT s FROM %s WHERE k = 10"),
+                   row(s)
+        );
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.set_test()
+     */
+    @Test
+    public void testSet() throws Throwable
+    {
+        createTable("CREATE TABLE %s ( fn text, ln text, tags set<text>, PRIMARY KEY (fn, ln) )");
+
+        execute("UPDATE %s SET tags = tags + { 'foo' } WHERE fn='Tom' AND ln='Bombadil'");
+        execute("UPDATE %s SET tags = tags + { 'bar' } WHERE fn='Tom' AND ln='Bombadil'");
+        execute("UPDATE %s SET tags = tags + { 'foo' } WHERE fn='Tom' AND ln='Bombadil'");
+        execute("UPDATE %s SET tags = tags + { 'foobar' } WHERE fn='Tom' AND ln='Bombadil'");
+        execute("UPDATE %s SET tags = tags - { 'bar' } WHERE fn='Tom' AND ln='Bombadil'");
+
+        assertRows(execute("SELECT tags FROM %s"),
+                   row(set("foo", "foobar")));
+
+        execute("UPDATE %s SET tags = { 'a', 'c', 'b' } WHERE fn='Bilbo' AND ln='Baggins'");
+        assertRows(execute("SELECT tags FROM %s WHERE fn='Bilbo' AND ln='Baggins'"),
+                   row(set("a", "b", "c")));
+
+        execute("UPDATE %s SET tags = { 'm', 'n' } WHERE fn='Bilbo' AND ln='Baggins'");
+        assertRows(execute("SELECT tags FROM %s WHERE fn='Bilbo' AND ln='Baggins'"),
+                   row(set("m", "n")));
+
+        execute("DELETE tags['m'] FROM %s WHERE fn='Bilbo' AND ln='Baggins'");
+        assertRows(execute("SELECT tags FROM %s WHERE fn='Bilbo' AND ln='Baggins'"),
+                   row(set("n")));
+
+        execute("DELETE tags FROM %s WHERE fn='Bilbo' AND ln='Baggins'");
+        assertEmpty(execute("SELECT tags FROM %s WHERE fn='Bilbo' AND ln='Baggins'"));
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.map_test()
+     */
+    @Test
+    public void testMap() throws Throwable
+    {
+        createTable("CREATE TABLE %s (fn text, ln text, m map<text, int>, PRIMARY KEY (fn, ln))");
+
+        execute("UPDATE %s SET m['foo'] = 3 WHERE fn='Tom' AND ln='Bombadil'");
+        execute("UPDATE %s SET m['bar'] = 4 WHERE fn='Tom' AND ln='Bombadil'");
+        execute("UPDATE %s SET m['woot'] = 5 WHERE fn='Tom' AND ln='Bombadil'");
+        execute("UPDATE %s SET m['bar'] = 6 WHERE fn='Tom' AND ln='Bombadil'");
+        execute("DELETE m['foo'] FROM %s WHERE fn='Tom' AND ln='Bombadil'");
+
+        assertRows(execute("SELECT m FROM %s"),
+                   row(map("bar", 6, "woot", 5)));
+
+        execute("UPDATE %s SET m = { 'a' : 4 , 'c' : 3, 'b' : 2 } WHERE fn='Bilbo' AND ln='Baggins'");
+        assertRows(execute("SELECT m FROM %s WHERE fn='Bilbo' AND ln='Baggins'"),
+                   row(map("a", 4, "b", 2, "c", 3)));
+
+        execute("UPDATE %s SET m =  { 'm' : 4 , 'n' : 1, 'o' : 2 } WHERE fn='Bilbo' AND ln='Baggins'");
+        assertRows(execute("SELECT m FROM %s WHERE fn='Bilbo' AND ln='Baggins'"),
+                   row(map("m", 4, "n", 1, "o", 2)));
+
+        execute("UPDATE %s SET m = {} WHERE fn='Bilbo' AND ln='Baggins'");
+        assertEmpty(execute("SELECT m FROM %s WHERE fn='Bilbo' AND ln='Baggins'"));
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.list_test()
+     */
+    @Test
+    public void testList() throws Throwable
+    {
+        createTable("CREATE TABLE %s (fn text, ln text, tags list<text>, PRIMARY KEY (fn, ln))");
+
+        execute("UPDATE %s SET tags = tags + [ 'foo' ] WHERE fn='Tom' AND ln='Bombadil'");
+        execute("UPDATE %s SET tags = tags + [ 'bar' ] WHERE fn='Tom' AND ln='Bombadil'");
+        execute("UPDATE %s SET tags = tags + [ 'foo' ] WHERE fn='Tom' AND ln='Bombadil'");
+        execute("UPDATE %s SET tags = tags + [ 'foobar' ] WHERE fn='Tom' AND ln='Bombadil'");
+
+        assertRows(execute("SELECT tags FROM %s"),
+                   row(list("foo", "bar", "foo", "foobar")));
+
+        execute("UPDATE %s SET tags = [ 'a', 'c', 'b', 'c' ] WHERE fn='Bilbo' AND ln='Baggins'");
+        assertRows(execute("SELECT tags FROM %s WHERE fn='Bilbo' AND ln='Baggins'"),
+                   row(list("a", "c", "b", "c")));
+
+        execute("UPDATE %s SET tags = [ 'm', 'n' ] + tags WHERE fn='Bilbo' AND ln='Baggins'");
+        assertRows(execute("SELECT tags FROM %s WHERE fn='Bilbo' AND ln='Baggins'"),
+                   row(list("m", "n", "a", "c", "b", "c")));
+
+        execute("UPDATE %s SET tags[2] = 'foo', tags[4] = 'bar' WHERE fn='Bilbo' AND ln='Baggins'");
+        assertRows(execute("SELECT tags FROM %s WHERE fn='Bilbo' AND ln='Baggins'"),
+                   row(list("m", "n", "foo", "c", "bar", "c")));
+
+        execute("DELETE tags[2] FROM %s WHERE fn='Bilbo' AND ln='Baggins'");
+        assertRows(execute("SELECT tags FROM %s WHERE fn='Bilbo' AND ln='Baggins'"),
+                   row(list("m", "n", "c", "bar", "c")));
+
+        execute("UPDATE %s SET tags = tags - [ 'bar' ] WHERE fn='Bilbo' AND ln='Baggins'");
+        assertRows(execute("SELECT tags FROM %s WHERE fn='Bilbo' AND ln='Baggins'"),
+                   row(list("m", "n", "c", "c")));
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.multi_collection_test()
+     */
+    @Test
+    public void testMultiCollections() throws Throwable
+    {
+        UUID id = UUID.fromString("b017f48f-ae67-11e1-9096-005056c00008");
+
+        createTable("CREATE TABLE %s (k uuid PRIMARY KEY, L list<int>, M map<text, int>, S set<int> )");
+
+        execute("UPDATE %s SET L = [1, 3, 5] WHERE k = ?", id);
+        execute("UPDATE %s SET L = L + [7, 11, 13] WHERE k = ?;", id);
+        execute("UPDATE %s SET S = {1, 3, 5} WHERE k = ?", id);
+        execute("UPDATE %s SET S = S + {7, 11, 13} WHERE k = ?", id);
+        execute("UPDATE %s SET M = {'foo': 1, 'bar' : 3} WHERE k = ?", id);
+        execute("UPDATE %s SET M = M + {'foobar' : 4} WHERE k = ?", id);
+
+        assertRows(execute("SELECT L, M, S FROM %s WHERE k = ?", id),
+                   row(list(1, 3, 5, 7, 11, 13),
+                       map("bar", 3, "foo", 1, "foobar", 4),
+                       set(1, 3, 5, 7, 11, 13)));
+    }
+
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.collection_and_regular_test()
+     */
+    @Test
+    public void testCollectionAndRegularColumns() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, l list<int>, c int)");
+
+        execute("INSERT INTO %s (k, l, c) VALUES(3, [0, 1, 2], 4)");
+        execute("UPDATE %s SET l[0] = 1, c = 42 WHERE k = 3");
+        assertRows(execute("SELECT l, c FROM %s WHERE k = 3"),
+                   row(list(1, 1, 2), 42));
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.multi_list_set_test()
+     */
+    @Test
+    public void testMultipleLists() throws Throwable
+    {
+        createTable(" CREATE TABLE %s (k int PRIMARY KEY, l1 list<int>, l2 list<int>)");
+
+        execute("INSERT INTO %s (k, l1, l2) VALUES (0, [1, 2, 3], [4, 5, 6])");
+        execute("UPDATE %s SET l2[1] = 42, l1[1] = 24  WHERE k = 0");
+
+        assertRows(execute("SELECT l1, l2 FROM %s WHERE k = 0"),
+                   row(list(1, 24, 3), list(4, 42, 6)));
+    }
+
+    /**
+     * Test you can add columns in a table with collections (#4982 bug),
+     * migrated from cql_tests.py:TestCQL.alter_with_collections_test()
+     */
+    @Test
+    public void testAlterCollections() throws Throwable
+    {
+        createTable("CREATE TABLE %s (key int PRIMARY KEY, aset set<text>)");
+        execute("ALTER TABLE %s ADD c text");
+        execute("ALTER TABLE %s ADD alist list<text>");
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.collection_compact_test()
+     */
+    @Test
+    public void testCompactCollections() throws Throwable
+    {
+        String tableName = KEYSPACE + "." + createTableName();
+        assertInvalid(String.format("CREATE TABLE %s (user ascii PRIMARY KEY, mails list < text >) WITH COMPACT STORAGE;", tableName));
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.collection_function_test()
+     */
+    @Test
+    public void testFunctionsOnCollections() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, l set<int>)");
+
+        assertInvalid("SELECT ttl(l) FROM %s WHERE k = 0");
+        assertInvalid("SELECT writetime(l) FROM %s WHERE k = 0");
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.bug_5376()
+     */
+    @Test
+    public void testInClauseWithCollections() throws Throwable
+    {
+        createTable("CREATE TABLE %s (key text, c bigint, v text, x set < text >, PRIMARY KEY(key, c) )");
+
+        assertInvalid("select * from %s where key = 'foo' and c in (1,3,4)");
+    }
+
+    /**
+     * Test for bug #5795,
+     * migrated from cql_tests.py:TestCQL.nonpure_function_collection_test()
+     */
+    @Test
+    public void testNonPureFunctionCollection() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, v list<timeuuid>)");
+
+        // we just want to make sure this doesn't throw
+        execute("INSERT INTO %s (k, v) VALUES (0, [now()])");
+    }
+
+    /**
+     * Test for 5805 bug,
+     * migrated from cql_tests.py:TestCQL.collection_flush_test()
+     */
+    @Test
+    public void testCollectionFlush() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, s set<int>)");
+
+        execute("INSERT INTO %s (k, s) VALUES (1, {1})");
+        flush();
+
+        execute("INSERT INTO %s (k, s) VALUES (1, {2})");
+        flush();
+
+        assertRows(execute("SELECT * FROM %s"),
+                   row(1, set(2)));
+    }
+
+    /**
+     * Test for 6276,
+     * migrated from cql_tests.py:TestCQL.drop_and_readd_collection_test()
+     */
+    @Test
+    public void testDropAndReaddCollection() throws Throwable
+    {
+        createTable("create table %s (k int primary key, v set<text>, x int)");
+        execute("insert into %s (k, v) VALUES (0, {'fffffffff'})");
+        flush();
+        execute("alter table %s drop v");
+        assertInvalid("alter table %s add v set<int>");
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/01115f72/test/unit/org/apache/cassandra/cql3/validation/entities/CountersTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/CountersTest.java b/test/unit/org/apache/cassandra/cql3/validation/entities/CountersTest.java
new file mode 100644
index 0000000..e5ff251
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/CountersTest.java
@@ -0,0 +1,115 @@
+/*
+ * 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.validation.entities;
+
+import org.junit.Test;
+
+import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+
+public class CountersTest extends CQLTester
+{
+    /**
+     * Check for a table with counters,
+     * migrated from cql_tests.py:TestCQL.counters_test()
+     */
+    @Test
+    public void testCounters() throws Throwable
+    {
+        createTable("CREATE TABLE %s (userid int, url text, total counter, PRIMARY KEY (userid, url)) WITH COMPACT STORAGE");
+
+        execute("UPDATE %s SET total = total + 1 WHERE userid = 1 AND url = 'http://foo.com'");
+        assertRows(execute("SELECT total FROM %s WHERE userid = 1 AND url = 'http://foo.com'"),
+                   row(1L));
+
+        execute("UPDATE %s SET total = total - 4 WHERE userid = 1 AND url = 'http://foo.com'");
+        assertRows(execute("SELECT total FROM %s WHERE userid = 1 AND url = 'http://foo.com'"),
+                   row(-3L));
+
+        execute("UPDATE %s SET total = total+1 WHERE userid = 1 AND url = 'http://foo.com'");
+        assertRows(execute("SELECT total FROM %s WHERE userid = 1 AND url = 'http://foo.com'"),
+                   row(-2L));
+
+        execute("UPDATE %s SET total = total -2 WHERE userid = 1 AND url = 'http://foo.com'");
+        assertRows(execute("SELECT total FROM %s WHERE userid = 1 AND url = 'http://foo.com'"),
+                   row(-4L));
+    }
+
+    /**
+     * Test for the validation bug of #4706,
+     * migrated from cql_tests.py:TestCQL.validate_counter_regular_test()
+     */
+    @Test
+    public void testRegularCounters() throws Throwable
+    {
+        assertInvalidThrowMessage("Cannot add a non counter column",
+                                  ConfigurationException.class,
+                                  String.format("CREATE TABLE %s.%s (id bigint PRIMARY KEY, count counter, things set<text>)", KEYSPACE, createTableName()));
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.collection_counter_test()
+     */
+    @Test
+    public void testCountersOnCollections() throws Throwable
+    {
+        String tableName = KEYSPACE + "." + createTableName();
+        assertInvalidThrow(InvalidRequestException.class,
+                           String.format("CREATE TABLE %s (k int PRIMARY KEY, l list<counter>)", tableName));
+
+        tableName = KEYSPACE + "." + createTableName();
+        assertInvalidThrow(InvalidRequestException.class,
+                           String.format("CREATE TABLE %s (k int PRIMARY KEY, s set<counter>)", tableName));
+
+        tableName = KEYSPACE + "." + createTableName();
+        assertInvalidThrow(InvalidRequestException.class,
+                           String.format("CREATE TABLE %s (k int PRIMARY KEY, m map<text, counter>)", tableName));
+    }
+
+    @Test
+    public void testCounterUpdatesWithUnset() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, c counter)");
+
+        // set up
+        execute("UPDATE %s SET c = c + 1 WHERE k = 10");
+        assertRows(execute("SELECT c FROM %s WHERE k = 10"),
+                   row(1L)
+        );
+        // increment
+        execute("UPDATE %s SET c = c + ? WHERE k = 10", 1L);
+        assertRows(execute("SELECT c FROM %s WHERE k = 10"),
+                   row(2L)
+        );
+        execute("UPDATE %s SET c = c + ? WHERE k = 10", unset());
+        assertRows(execute("SELECT c FROM %s WHERE k = 10"),
+                   row(2L) // no change to the counter value
+        );
+        // decrement
+        execute("UPDATE %s SET c = c - ? WHERE k = 10", 1L);
+        assertRows(execute("SELECT c FROM %s WHERE k = 10"),
+                   row(1L)
+        );
+        execute("UPDATE %s SET c = c - ? WHERE k = 10", unset());
+        assertRows(execute("SELECT c FROM %s WHERE k = 10"),
+                   row(1L) // no change to the counter value
+        );
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/01115f72/test/unit/org/apache/cassandra/cql3/validation/entities/DateTypeTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/DateTypeTest.java b/test/unit/org/apache/cassandra/cql3/validation/entities/DateTypeTest.java
new file mode 100644
index 0000000..7fa5e67
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/DateTypeTest.java
@@ -0,0 +1,39 @@
+/*
+ * 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.validation.entities;
+
+import org.junit.Test;
+
+import org.apache.cassandra.cql3.CQLTester;
+
+public class DateTypeTest extends CQLTester
+{
+    /**
+     * Check dates are correctly recognized and validated,
+     * migrated from cql_tests.py:TestCQL.date_test()
+     */
+    @Test
+    public void testDate() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, t timestamp)");
+
+        execute("INSERT INTO %s (k, t) VALUES (0, '2011-02-03')");
+        assertInvalid("INSERT INTO %s (k, t) VALUES (0, '2011-42-42')");
+    }
+}


[25/32] cassandra git commit: 2.2 commit for CASSANDRA-9160

Posted by jm...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/01115f72/test/unit/org/apache/cassandra/cql3/SelectWithTokenFunctionTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/SelectWithTokenFunctionTest.java b/test/unit/org/apache/cassandra/cql3/SelectWithTokenFunctionTest.java
deleted file mode 100644
index cde4f92..0000000
--- a/test/unit/org/apache/cassandra/cql3/SelectWithTokenFunctionTest.java
+++ /dev/null
@@ -1,233 +0,0 @@
-/*
- * 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;
-
-import java.util.Arrays;
-
-import org.junit.Test;
-
-public class SelectWithTokenFunctionTest extends CQLTester
-{
-    @Test
-    public void testTokenFunctionWithSingleColumnPartitionKey() throws Throwable
-    {
-        createTable("CREATE TABLE IF NOT EXISTS %s (a int PRIMARY KEY, b text)");
-        execute("INSERT INTO %s (a, b) VALUES (0, 'a')");
-
-        assertRows(execute("SELECT * FROM %s WHERE token(a) >= token(?)", 0), row(0, "a"));
-        assertRows(execute("SELECT * FROM %s WHERE token(a) >= token(?) and token(a) < token(?)", 0, 1), row(0, "a"));
-        assertInvalid("SELECT * FROM %s WHERE token(a) > token(?)", "a");
-        assertInvalidMessage("The token() function must contains only partition key components",
-                             "SELECT * FROM %s WHERE token(a, b) >= token(?, ?)", "b", 0);
-        assertInvalidMessage("More than one restriction was found for the start bound on a",
-                             "SELECT * FROM %s WHERE token(a) >= token(?) and token(a) >= token(?)", 0, 1);
-        assertInvalidMessage("Columns \"a\" cannot be restricted by both an equality and an inequality relation",
-                             "SELECT * FROM %s WHERE token(a) >= token(?) and token(a) = token(?)", 0, 1);
-        assertInvalidSyntax("SELECT * FROM %s WHERE token(a) = token(?) and token(a) IN (token(?))", 0, 1);
-
-        assertInvalidMessage("More than one restriction was found for the start bound on a",
-                             "SELECT * FROM %s WHERE token(a) > token(?) AND token(a) > token(?)", 1, 2);
-        assertInvalidMessage("More than one restriction was found for the end bound on a",
-                             "SELECT * FROM %s WHERE token(a) <= token(?) AND token(a) < token(?)", 1, 2);
-        assertInvalidMessage("Columns \"a\" cannot be restricted by both an equality and an inequality relation",
-                             "SELECT * FROM %s WHERE token(a) > token(?) AND token(a) = token(?)", 1, 2);
-        assertInvalidMessage("a cannot be restricted by more than one relation if it includes an Equal",
-                             "SELECT * FROM %s WHERE  token(a) = token(?) AND token(a) > token(?)", 1, 2);
-    }
-
-    @Test
-    public void testTokenFunctionWithPartitionKeyAndClusteringKeyArguments() throws Throwable
-    {
-        createTable("CREATE TABLE IF NOT EXISTS %s (a int, b text, PRIMARY KEY (a, b))");
-        assertInvalidMessage("The token() function must contains only partition key components",
-                             "SELECT * FROM %s WHERE token(a, b) > token(0, 'c')");
-    }
-
-    @Test
-    public void testTokenFunctionWithMultiColumnPartitionKey() throws Throwable
-    {
-        createTable("CREATE TABLE IF NOT EXISTS %s (a int, b text, PRIMARY KEY ((a, b)))");
-        execute("INSERT INTO %s (a, b) VALUES (0, 'a')");
-        execute("INSERT INTO %s (a, b) VALUES (0, 'b')");
-        execute("INSERT INTO %s (a, b) VALUES (0, 'c')");
-
-        assertRows(execute("SELECT * FROM %s WHERE token(a, b) > token(?, ?)", 0, "a"),
-                   row(0, "b"),
-                   row(0, "c"));
-        assertRows(execute("SELECT * FROM %s WHERE token(a, b) > token(?, ?) and token(a, b) < token(?, ?)",
-                           0, "a",
-                           0, "d"),
-                   row(0, "b"),
-                   row(0, "c"));
-        assertInvalidMessage("The token() function must be applied to all partition key components or none of them",
-                             "SELECT * FROM %s WHERE token(a) > token(?) and token(b) > token(?)", 0, "a");
-        assertInvalidMessage("The token() function must be applied to all partition key components or none of them",
-                             "SELECT * FROM %s WHERE token(a) > token(?, ?) and token(a) < token(?, ?) and token(b) > token(?, ?) ",
-                             0, "a", 0, "d", 0, "a");
-        assertInvalidMessage("The token function arguments must be in the partition key order: a, b",
-                             "SELECT * FROM %s WHERE token(b, a) > token(0, 'c')");
-        assertInvalidMessage("The token() function must be applied to all partition key components or none of them",
-                             "SELECT * FROM %s WHERE token(a, b) > token(?, ?) and token(b) < token(?, ?)", 0, "a", 0, "a");
-        assertInvalidMessage("The token() function must be applied to all partition key components or none of them",
-                             "SELECT * FROM %s WHERE token(a) > token(?, ?) and token(b) > token(?, ?)", 0, "a", 0, "a");
-    }
-
-    @Test
-    public void testSingleColumnPartitionKeyWithTokenNonTokenRestrictionsMix() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a int primary key, b int)");
-
-        execute("INSERT INTO %s (a, b) VALUES (0, 0);");
-        execute("INSERT INTO %s (a, b) VALUES (1, 1);");
-        execute("INSERT INTO %s (a, b) VALUES (2, 2);");
-        execute("INSERT INTO %s (a, b) VALUES (3, 3);");
-        execute("INSERT INTO %s (a, b) VALUES (4, 4);");
-        assertRows(execute("SELECT * FROM %s WHERE a IN (?, ?);", 1, 3),
-                   row(1, 1),
-                   row(3, 3));
-        assertRows(execute("SELECT * FROM %s WHERE token(a)> token(?) and token(a) <= token(?);", 1, 3),
-                   row(2, 2),
-                   row(3, 3));
-        assertRows(execute("SELECT * FROM %s WHERE token(a)= token(2);"),
-                   row(2, 2));
-        assertRows(execute("SELECT * FROM %s WHERE token(a) > token(?) AND token(a) <= token(?) AND a IN (?, ?);",
-                           1, 3, 1, 3),
-                   row(3, 3));
-        assertRows(execute("SELECT * FROM %s WHERE token(a) < token(?) AND token(a) >= token(?) AND a IN (?, ?);",
-                           1, 3, 1, 3),
-                   row(3, 3));
-        assertInvalidMessage("Only EQ and IN relation are supported on the partition key (unless you use the token() function)",
-                             "SELECT * FROM %s WHERE token(a) > token(?) AND token(a) <= token(?) AND a > ?;", 1, 3, 1);
-
-        assertRows(execute("SELECT * FROM %s WHERE token(a) > token(?) AND token(a) <= token(?) AND a IN ?;",
-                           1, 3, Arrays.asList(1, 3)),
-                   row(3, 3));
-        assertRows(execute("SELECT * FROM %s WHERE token(a) > token(?) AND a = ?;", 1, 3),
-                   row(3, 3));
-        assertRows(execute("SELECT * FROM %s WHERE a = ? AND token(a) > token(?);", 3, 1),
-                   row(3, 3));
-        assertEmpty(execute("SELECT * FROM %s WHERE token(a) > token(?) AND a = ?;", 3, 1));
-        assertEmpty(execute("SELECT * FROM %s WHERE a = ? AND token(a) > token(?);", 1, 3));
-        assertRows(execute("SELECT * FROM %s WHERE token(a) > token(?) AND a IN (?, ?);", 2, 1, 3),
-                   row(3, 3));
-        assertRows(execute("SELECT * FROM %s WHERE token(a) > token(?) AND token(a) < token(?) AND a IN (?, ?) ;", 2, 5, 1, 3),
-                   row(3, 3));
-        assertRows(execute("SELECT * FROM %s WHERE a IN (?, ?) AND token(a) > token(?) AND token(a) < token(?);", 1, 3, 2, 5),
-                   row(3, 3));
-        assertRows(execute("SELECT * FROM %s WHERE token(a) > token(?) AND a IN (?, ?) AND token(a) < token(?);", 2, 1, 3, 5),
-                   row(3, 3));
-        assertEmpty(execute("SELECT * FROM %s WHERE a IN (?, ?) AND token(a) > token(?);", 1, 3, 3));
-        assertRows(execute("SELECT * FROM %s WHERE token(a) <= token(?) AND a = ?;", 2, 2),
-                   row(2, 2));
-        assertEmpty(execute("SELECT * FROM %s WHERE token(a) <= token(?) AND a = ?;", 2, 3));
-        assertEmpty(execute("SELECT * FROM %s WHERE token(a) = token(?) AND a = ?;", 2, 3));
-        assertRows(execute("SELECT * FROM %s WHERE token(a) >= token(?) AND token(a) <= token(?) AND a = ?;", 2, 2, 2),
-                   row(2, 2));
-        assertEmpty(execute("SELECT * FROM %s WHERE token(a) >= token(?) AND token(a) < token(?) AND a = ?;", 2, 2, 2));
-        assertEmpty(execute("SELECT * FROM %s WHERE token(a) > token(?) AND token(a) <= token(?) AND a = ?;", 2, 2, 2));
-        assertEmpty(execute("SELECT * FROM %s WHERE token(a) > token(?) AND token(a) < token(?) AND a = ?;", 2, 2, 2));
-    }
-
-    @Test
-    public void testMultiColumnPartitionKeyWithTokenNonTokenRestrictionsMix() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a int, b int, c int, primary key((a, b)))");
-
-        execute("INSERT INTO %s (a, b, c) VALUES (0, 0, 0);");
-        execute("INSERT INTO %s (a, b, c) VALUES (0, 1, 1);");
-        execute("INSERT INTO %s (a, b, c) VALUES (0, 2, 2);");
-        execute("INSERT INTO %s (a, b, c) VALUES (1, 0, 3);");
-        execute("INSERT INTO %s (a, b, c) VALUES (1, 1, 4);");
-
-        assertRows(execute("SELECT * FROM %s WHERE token(a, b) > token(?, ?);", 0, 0),
-                   row(0, 1, 1),
-                   row(0, 2, 2),
-                   row(1, 0, 3),
-                   row(1, 1, 4));
-
-        assertRows(execute("SELECT * FROM %s WHERE token(a, b) > token(?, ?) AND a = ? AND b IN (?, ?);",
-                           0, 0, 1, 0, 1),
-                   row(1, 0, 3),
-                   row(1, 1, 4));
-
-        assertRows(execute("SELECT * FROM %s WHERE a = ? AND token(a, b) > token(?, ?) AND b IN (?, ?);",
-                           1, 0, 0, 0, 1),
-                   row(1, 0, 3),
-                   row(1, 1, 4));
-
-        assertRows(execute("SELECT * FROM %s WHERE b IN (?, ?) AND token(a, b) > token(?, ?) AND a = ?;",
-                           0, 1, 0, 0, 1),
-                   row(1, 0, 3),
-                   row(1, 1, 4));
-
-        assertEmpty(execute("SELECT * FROM %s WHERE b IN (?, ?) AND token(a, b) > token(?, ?) AND token(a, b) < token(?, ?) AND a = ?;",
-                            0, 1, 0, 0, 0, 0, 1));
-
-        assertEmpty(execute("SELECT * FROM %s WHERE b IN (?, ?) AND token(a, b) > token(?, ?) AND token(a, b) <= token(?, ?) AND a = ?;",
-                            0, 1, 0, 0, 0, 0, 1));
-
-        assertEmpty(execute("SELECT * FROM %s WHERE b IN (?, ?) AND token(a, b) >= token(?, ?) AND token(a, b) < token(?, ?) AND a = ?;",
-                            0, 1, 0, 0, 0, 0, 1));
-
-        assertEmpty(execute("SELECT * FROM %s WHERE b IN (?, ?) AND token(a, b) = token(?, ?) AND a = ?;",
-                            0, 1, 0, 0, 1));
-
-        assertInvalidMessage("Partition key parts: b must be restricted as other parts are",
-                             "SELECT * FROM %s WHERE token(a, b) > token(?, ?) AND a = ?;", 0, 0, 1);
-    }
-
-    @Test
-    public void testMultiColumnPartitionKeyWithIndexAndTokenNonTokenRestrictionsMix() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a int, b int, c int, primary key((a, b)))");
-        createIndex("CREATE INDEX ON %s(b)");
-        createIndex("CREATE INDEX ON %s(c)");
-
-        execute("INSERT INTO %s (a, b, c) VALUES (0, 0, 0);");
-        execute("INSERT INTO %s (a, b, c) VALUES (0, 1, 1);");
-        execute("INSERT INTO %s (a, b, c) VALUES (0, 2, 2);");
-        execute("INSERT INTO %s (a, b, c) VALUES (1, 0, 3);");
-        execute("INSERT INTO %s (a, b, c) VALUES (1, 1, 4);");
-
-        assertRows(execute("SELECT * FROM %s WHERE b = ?;", 1),
-                   row(0, 1, 1),
-                   row(1, 1, 4));
-
-        assertRows(execute("SELECT * FROM %s WHERE token(a, b) > token(?, ?) AND b = ?;", 0, 0, 1),
-                   row(0, 1, 1),
-                   row(1, 1, 4));
-
-        assertRows(execute("SELECT * FROM %s WHERE b = ? AND token(a, b) > token(?, ?);", 1, 0, 0),
-                   row(0, 1, 1),
-                   row(1, 1, 4));
-
-        assertRows(execute("SELECT * FROM %s WHERE b = ? AND token(a, b) > token(?, ?) and c = ? ALLOW FILTERING;", 1, 0, 0, 4),
-                   row(1, 1, 4));
-    }
-
-    @Test
-    public void testTokenFunctionWithCompoundPartitionAndClusteringCols() throws Throwable
-    {
-        createTable("CREATE TABLE IF NOT EXISTS %s (a int, b int, c int, d int, PRIMARY KEY ((a, b), c, d))");
-        // just test that the queries don't error
-        execute("SELECT * FROM %s WHERE token(a, b) > token(0, 0) AND c > 10 ALLOW FILTERING;");
-        execute("SELECT * FROM %s WHERE c > 10 AND token(a, b) > token(0, 0) ALLOW FILTERING;");
-        execute("SELECT * FROM %s WHERE token(a, b) > token(0, 0) AND (c, d) > (0, 0) ALLOW FILTERING;");
-        execute("SELECT * FROM %s WHERE (c, d) > (0, 0) AND token(a, b) > token(0, 0) ALLOW FILTERING;");
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/01115f72/test/unit/org/apache/cassandra/cql3/SelectionOrderingTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/SelectionOrderingTest.java b/test/unit/org/apache/cassandra/cql3/SelectionOrderingTest.java
deleted file mode 100644
index 301aaf4..0000000
--- a/test/unit/org/apache/cassandra/cql3/SelectionOrderingTest.java
+++ /dev/null
@@ -1,233 +0,0 @@
-/*
- * 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;
-
-import org.junit.Test;
-
-public class SelectionOrderingTest extends CQLTester
-{
-
-    @Test
-    public void testNormalSelectionOrderSingleClustering() throws Throwable
-    {
-        for (String descOption : new String[]{"", " WITH CLUSTERING ORDER BY (b DESC)"})
-        {
-            createTable("CREATE TABLE %s (a int, b int, c int, PRIMARY KEY (a, b))" + descOption);
-            execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, 0, 0);
-            execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, 1, 1);
-            execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, 2, 2);
-
-            assertRows(execute("SELECT * FROM %s WHERE a=? ORDER BY b ASC", 0),
-                    row(0, 0, 0),
-                    row(0, 1, 1),
-                    row(0, 2, 2)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a=? ORDER BY b DESC", 0),
-                    row(0, 2, 2),
-                    row(0, 1, 1),
-                    row(0, 0, 0)
-            );
-
-            // order by the only column in the selection
-            assertRows(execute("SELECT b FROM %s WHERE a=? ORDER BY b ASC", 0),
-                    row(0), row(1), row(2));
-
-            assertRows(execute("SELECT b FROM %s WHERE a=? ORDER BY b DESC", 0),
-                    row(2), row(1), row(0));
-
-            // order by a column not in the selection
-            assertRows(execute("SELECT c FROM %s WHERE a=? ORDER BY b ASC", 0),
-                    row(0), row(1), row(2));
-
-            assertRows(execute("SELECT c FROM %s WHERE a=? ORDER BY b DESC", 0),
-                    row(2), row(1), row(0));
-        }
-    }
-
-    @Test
-    public void testFunctionSelectionOrderSingleClustering() throws Throwable
-    {
-        for (String descOption : new String[]{"", " WITH CLUSTERING ORDER BY (b DESC)"})
-        {
-            createTable("CREATE TABLE %s (a int, b int, c int, PRIMARY KEY (a, b))" + descOption);
-            execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, 0, 0);
-            execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, 1, 1);
-            execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, 2, 2);
-
-            // order by the only column in the selection
-            assertRows(execute("SELECT blobAsInt(intAsBlob(b)) FROM %s WHERE a=? ORDER BY b ASC", 0),
-                    row(0), row(1), row(2));
-
-            assertRows(execute("SELECT blobAsInt(intAsBlob(b)) FROM %s WHERE a=? ORDER BY b DESC", 0),
-                    row(2), row(1), row(0));
-
-            // order by a column not in the selection
-            assertRows(execute("SELECT blobAsInt(intAsBlob(c)) FROM %s WHERE a=? ORDER BY b ASC", 0),
-                    row(0), row(1), row(2));
-
-            assertRows(execute("SELECT blobAsInt(intAsBlob(c)) FROM %s WHERE a=? ORDER BY b DESC", 0),
-                    row(2), row(1), row(0));
-
-            assertInvalid("SELECT * FROM %s WHERE a=? ORDER BY c ASC", 0);
-            assertInvalid("SELECT * FROM %s WHERE a=? ORDER BY c DESC", 0);
-        }
-    }
-
-    @Test
-    public void testFieldSelectionOrderSingleClustering() throws Throwable
-    {
-        String type = createType("CREATE TYPE %s (a int)");
-
-        for (String descOption : new String[]{"", " WITH CLUSTERING ORDER BY (b DESC)"})
-        {
-            createTable("CREATE TABLE %s (a int, b int, c frozen<" + type + "   >, PRIMARY KEY (a, b))" + descOption);
-            execute("INSERT INTO %s (a, b, c) VALUES (?, ?, {a: ?})", 0, 0, 0);
-            execute("INSERT INTO %s (a, b, c) VALUES (?, ?, {a: ?})", 0, 1, 1);
-            execute("INSERT INTO %s (a, b, c) VALUES (?, ?, {a: ?})", 0, 2, 2);
-
-            // order by a column not in the selection
-            assertRows(execute("SELECT c.a FROM %s WHERE a=? ORDER BY b ASC", 0),
-                    row(0), row(1), row(2));
-
-            assertRows(execute("SELECT c.a FROM %s WHERE a=? ORDER BY b DESC", 0),
-                    row(2), row(1), row(0));
-
-            assertRows(execute("SELECT blobAsInt(intAsBlob(c.a)) FROM %s WHERE a=? ORDER BY b DESC", 0),
-                    row(2), row(1), row(0));
-            dropTable("DROP TABLE %s");
-        }
-    }
-
-    @Test
-    public void testNormalSelectionOrderMultipleClustering() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a int, b int, c int, d int, PRIMARY KEY (a, b, c))");
-        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 0, 0);
-        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 1, 1);
-        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 2, 2);
-        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 0, 3);
-        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 1, 4);
-        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 2, 5);
-
-        assertRows(execute("SELECT * FROM %s WHERE a=? ORDER BY b ASC", 0),
-                row(0, 0, 0, 0),
-                row(0, 0, 1, 1),
-                row(0, 0, 2, 2),
-                row(0, 1, 0, 3),
-                row(0, 1, 1, 4),
-                row(0, 1, 2, 5)
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE a=? ORDER BY b DESC", 0),
-                row(0, 1, 2, 5),
-                row(0, 1, 1, 4),
-                row(0, 1, 0, 3),
-                row(0, 0, 2, 2),
-                row(0, 0, 1, 1),
-                row(0, 0, 0, 0)
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE a=? ORDER BY b DESC, c DESC", 0),
-                row(0, 1, 2, 5),
-                row(0, 1, 1, 4),
-                row(0, 1, 0, 3),
-                row(0, 0, 2, 2),
-                row(0, 0, 1, 1),
-                row(0, 0, 0, 0)
-        );
-
-        assertInvalid("SELECT * FROM %s WHERE a=? ORDER BY c ASC", 0);
-        assertInvalid("SELECT * FROM %s WHERE a=? ORDER BY c DESC", 0);
-        assertInvalid("SELECT * FROM %s WHERE a=? ORDER BY b ASC, c DESC", 0);
-        assertInvalid("SELECT * FROM %s WHERE a=? ORDER BY b DESC, c ASC", 0);
-        assertInvalid("SELECT * FROM %s WHERE a=? ORDER BY d ASC", 0);
-
-        // select and order by b
-        assertRows(execute("SELECT b FROM %s WHERE a=? ORDER BY b ASC", 0),
-                row(0), row(0), row(0), row(1), row(1), row(1));
-        assertRows(execute("SELECT b FROM %s WHERE a=? ORDER BY b DESC", 0),
-                row(1), row(1), row(1), row(0), row(0), row(0));
-
-        // select c, order by b
-        assertRows(execute("SELECT c FROM %s WHERE a=? ORDER BY b ASC", 0),
-                row(0), row(1), row(2), row(0), row(1), row(2));
-        assertRows(execute("SELECT c FROM %s WHERE a=? ORDER BY b DESC", 0),
-                row(2), row(1), row(0), row(2), row(1), row(0));
-
-        // select c, order by b, c
-        assertRows(execute("SELECT c FROM %s WHERE a=? ORDER BY b ASC, c ASC", 0),
-                row(0), row(1), row(2), row(0), row(1), row(2));
-        assertRows(execute("SELECT c FROM %s WHERE a=? ORDER BY b DESC, c DESC", 0),
-                row(2), row(1), row(0), row(2), row(1), row(0));
-
-        // select d, order by b, c
-        assertRows(execute("SELECT d FROM %s WHERE a=? ORDER BY b ASC, c ASC", 0),
-                row(0), row(1), row(2), row(3), row(4), row(5));
-        assertRows(execute("SELECT d FROM %s WHERE a=? ORDER BY b DESC, c DESC", 0),
-                row(5), row(4), row(3), row(2), row(1), row(0));
-    }
-
-    @Test
-    public void testFunctionSelectionOrderMultipleClustering() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a int, b int, c int, d int, PRIMARY KEY (a, b, c))");
-        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 0, 0);
-        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 1, 1);
-        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 2, 2);
-        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 0, 3);
-        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 1, 4);
-        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 2, 5);
-
-        assertInvalid("SELECT blobAsInt(intAsBlob(b)) FROM %s WHERE a=? ORDER BY c ASC", 0);
-        assertInvalid("SELECT blobAsInt(intAsBlob(b)) FROM %s WHERE a=? ORDER BY c DESC", 0);
-        assertInvalid("SELECT blobAsInt(intAsBlob(b)) FROM %s WHERE a=? ORDER BY b ASC, c DESC", 0);
-        assertInvalid("SELECT blobAsInt(intAsBlob(b)) FROM %s WHERE a=? ORDER BY b DESC, c ASC", 0);
-        assertInvalid("SELECT blobAsInt(intAsBlob(b)) FROM %s WHERE a=? ORDER BY d ASC", 0);
-
-        // select and order by b
-        assertRows(execute("SELECT blobAsInt(intAsBlob(b)) FROM %s WHERE a=? ORDER BY b ASC", 0),
-                row(0), row(0), row(0), row(1), row(1), row(1));
-        assertRows(execute("SELECT blobAsInt(intAsBlob(b)) FROM %s WHERE a=? ORDER BY b DESC", 0),
-                row(1), row(1), row(1), row(0), row(0), row(0));
-
-        assertRows(execute("SELECT b, blobAsInt(intAsBlob(b)) FROM %s WHERE a=? ORDER BY b ASC", 0),
-                row(0, 0), row(0, 0), row(0, 0), row(1, 1), row(1, 1), row(1, 1));
-        assertRows(execute("SELECT b, blobAsInt(intAsBlob(b)) FROM %s WHERE a=? ORDER BY b DESC", 0),
-                row(1, 1), row(1, 1), row(1, 1), row(0, 0), row(0, 0), row(0, 0));
-
-        // select c, order by b
-        assertRows(execute("SELECT blobAsInt(intAsBlob(c)) FROM %s WHERE a=? ORDER BY b ASC", 0),
-                row(0), row(1), row(2), row(0), row(1), row(2));
-        assertRows(execute("SELECT blobAsInt(intAsBlob(c)) FROM %s WHERE a=? ORDER BY b DESC", 0),
-                row(2), row(1), row(0), row(2), row(1), row(0));
-
-        // select c, order by b, c
-        assertRows(execute("SELECT blobAsInt(intAsBlob(c)) FROM %s WHERE a=? ORDER BY b ASC, c ASC", 0),
-                row(0), row(1), row(2), row(0), row(1), row(2));
-        assertRows(execute("SELECT blobAsInt(intAsBlob(c)) FROM %s WHERE a=? ORDER BY b DESC, c DESC", 0),
-                row(2), row(1), row(0), row(2), row(1), row(0));
-
-        // select d, order by b, c
-        assertRows(execute("SELECT blobAsInt(intAsBlob(d)) FROM %s WHERE a=? ORDER BY b ASC, c ASC", 0),
-                row(0), row(1), row(2), row(3), row(4), row(5));
-        assertRows(execute("SELECT blobAsInt(intAsBlob(d)) FROM %s WHERE a=? ORDER BY b DESC, c DESC", 0),
-                row(5), row(4), row(3), row(2), row(1), row(0));
-
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/01115f72/test/unit/org/apache/cassandra/cql3/SingleColumnRelationTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/SingleColumnRelationTest.java b/test/unit/org/apache/cassandra/cql3/SingleColumnRelationTest.java
deleted file mode 100644
index 2b8fbd4..0000000
--- a/test/unit/org/apache/cassandra/cql3/SingleColumnRelationTest.java
+++ /dev/null
@@ -1,553 +0,0 @@
-/*
- * 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;
-
-import java.util.Arrays;
-
-import org.junit.Test;
-
-import java.util.ArrayList;
-import java.util.List;
-
-public class SingleColumnRelationTest extends CQLTester
-{
-    @Test
-    public void testInvalidCollectionEqualityRelation() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a int PRIMARY KEY, b set<int>, c list<int>, d map<int, int>)");
-        createIndex("CREATE INDEX ON %s (b)");
-        createIndex("CREATE INDEX ON %s (c)");
-        createIndex("CREATE INDEX ON %s (d)");
-
-        assertInvalidMessage("Collection column 'b' (set<int>) cannot be restricted by a '=' relation",
-                             "SELECT * FROM %s WHERE a = 0 AND b=?", set(0));
-        assertInvalidMessage("Collection column 'c' (list<int>) cannot be restricted by a '=' relation",
-                             "SELECT * FROM %s WHERE a = 0 AND c=?", list(0));
-        assertInvalidMessage("Collection column 'd' (map<int, int>) cannot be restricted by a '=' relation",
-                             "SELECT * FROM %s WHERE a = 0 AND d=?", map(0, 0));
-    }
-
-    @Test
-    public void testInvalidCollectionNonEQRelation() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a int PRIMARY KEY, b set<int>, c int)");
-        createIndex("CREATE INDEX ON %s (c)");
-        execute("INSERT INTO %s (a, b, c) VALUES (0, {0}, 0)");
-
-        // non-EQ operators
-        assertInvalidMessage("Collection column 'b' (set<int>) cannot be restricted by a '>' relation",
-                             "SELECT * FROM %s WHERE c = 0 AND b > ?", set(0));
-        assertInvalidMessage("Collection column 'b' (set<int>) cannot be restricted by a '>=' relation",
-                             "SELECT * FROM %s WHERE c = 0 AND b >= ?", set(0));
-        assertInvalidMessage("Collection column 'b' (set<int>) cannot be restricted by a '<' relation",
-                             "SELECT * FROM %s WHERE c = 0 AND b < ?", set(0));
-        assertInvalidMessage("Collection column 'b' (set<int>) cannot be restricted by a '<=' relation",
-                             "SELECT * FROM %s WHERE c = 0 AND b <= ?", set(0));
-        assertInvalidMessage("Collection column 'b' (set<int>) cannot be restricted by a 'IN' relation",
-                             "SELECT * FROM %s WHERE c = 0 AND b IN (?)", set(0));
-    }
-
-    @Test
-    public void testClusteringColumnRelations() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a text, b int, c int, d int, primary key(a, b, c))");
-        execute("insert into %s (a, b, c, d) values (?, ?, ?, ?)", "first", 1, 5, 1);
-        execute("insert into %s (a, b, c, d) values (?, ?, ?, ?)", "first", 2, 6, 2);
-        execute("insert into %s (a, b, c, d) values (?, ?, ?, ?)", "first", 3, 7, 3);
-        execute("insert into %s (a, b, c, d) values (?, ?, ?, ?)", "second", 4, 8, 4);
-
-        testSelectQueriesWithClusteringColumnRelations();
-    }
-
-    @Test
-    public void testClusteringColumnRelationsWithCompactStorage() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a text, b int, c int, d int, primary key(a, b, c)) WITH COMPACT STORAGE;");
-        execute("insert into %s (a, b, c, d) values (?, ?, ?, ?)", "first", 1, 5, 1);
-        execute("insert into %s (a, b, c, d) values (?, ?, ?, ?)", "first", 2, 6, 2);
-        execute("insert into %s (a, b, c, d) values (?, ?, ?, ?)", "first", 3, 7, 3);
-        execute("insert into %s (a, b, c, d) values (?, ?, ?, ?)", "second", 4, 8, 4);
-
-        testSelectQueriesWithClusteringColumnRelations();
-    }
-
-    private void testSelectQueriesWithClusteringColumnRelations() throws Throwable
-    {
-        assertRows(execute("select * from %s where a in (?, ?)", "first", "second"),
-                   row("first", 1, 5, 1),
-                   row("first", 2, 6, 2),
-                   row("first", 3, 7, 3),
-                   row("second", 4, 8, 4));
-
-        assertRows(execute("select * from %s where a = ? and b = ? and c in (?, ?)", "first", 2, 6, 7),
-                   row("first", 2, 6, 2));
-
-        assertRows(execute("select * from %s where a = ? and b in (?, ?) and c in (?, ?)", "first", 2, 3, 6, 7),
-                   row("first", 2, 6, 2),
-                   row("first", 3, 7, 3));
-
-        assertRows(execute("select * from %s where a = ? and b in (?, ?) and c in (?, ?)", "first", 3, 2, 7, 6),
-                   row("first", 2, 6, 2),
-                   row("first", 3, 7, 3));
-
-        assertRows(execute("select * from %s where a = ? and c in (?, ?) and b in (?, ?)", "first", 7, 6, 3, 2),
-                   row("first", 2, 6, 2),
-                   row("first", 3, 7, 3));
-
-        assertRows(execute("select c, d from %s where a = ? and c in (?, ?) and b in (?, ?)", "first", 7, 6, 3, 2),
-                   row(6, 2),
-                   row(7, 3));
-
-        assertRows(execute("select c, d from %s where a = ? and c in (?, ?) and b in (?, ?, ?)", "first", 7, 6, 3, 2, 3),
-                   row(6, 2),
-                   row(7, 3));
-
-        assertRows(execute("select * from %s where a = ? and b in (?, ?) and c = ?", "first", 3, 2, 7),
-                   row("first", 3, 7, 3));
-
-        assertRows(execute("select * from %s where a = ? and b in ? and c in ?",
-                           "first", Arrays.asList(3, 2), Arrays.asList(7, 6)),
-                   row("first", 2, 6, 2),
-                   row("first", 3, 7, 3));
-
-        assertInvalidMessage("Invalid null value for column b",
-                             "select * from %s where a = ? and b in ? and c in ?", "first", null, Arrays.asList(7, 6));
-
-        assertRows(execute("select * from %s where a = ? and c >= ? and b in (?, ?)", "first", 6, 3, 2),
-                   row("first", 2, 6, 2),
-                   row("first", 3, 7, 3));
-
-        assertRows(execute("select * from %s where a = ? and c > ? and b in (?, ?)", "first", 6, 3, 2),
-                   row("first", 3, 7, 3));
-
-        assertRows(execute("select * from %s where a = ? and c <= ? and b in (?, ?)", "first", 6, 3, 2),
-                   row("first", 2, 6, 2));
-
-        assertRows(execute("select * from %s where a = ? and c < ? and b in (?, ?)", "first", 7, 3, 2),
-                   row("first", 2, 6, 2));
-//---
-        assertRows(execute("select * from %s where a = ? and c >= ? and c <= ? and b in (?, ?)", "first", 6, 7, 3, 2),
-                   row("first", 2, 6, 2),
-                   row("first", 3, 7, 3));
-
-        assertRows(execute("select * from %s where a = ? and c > ? and c <= ? and b in (?, ?)", "first", 6, 7, 3, 2),
-                   row("first", 3, 7, 3));
-
-        assertEmpty(execute("select * from %s where a = ? and c > ? and c < ? and b in (?, ?)", "first", 6, 7, 3, 2));
-
-        assertInvalidMessage("Column \"c\" cannot be restricted by both an equality and an inequality relation",
-                             "select * from %s where a = ? and c > ? and c = ? and b in (?, ?)", "first", 6, 7, 3, 2);
-
-        assertInvalidMessage("c cannot be restricted by more than one relation if it includes an Equal",
-                             "select * from %s where a = ? and c = ? and c > ?  and b in (?, ?)", "first", 6, 7, 3, 2);
-
-        assertRows(execute("select * from %s where a = ? and c in (?, ?) and b in (?, ?) order by b DESC",
-                           "first", 7, 6, 3, 2),
-                   row("first", 3, 7, 3),
-                   row("first", 2, 6, 2));
-
-        assertInvalidMessage("More than one restriction was found for the start bound on b",
-                             "select * from %s where a = ? and b > ? and b > ?", "first", 6, 3, 2);
-
-        assertInvalidMessage("More than one restriction was found for the end bound on b",
-                             "select * from %s where a = ? and b < ? and b <= ?", "first", 6, 3, 2);
-    }
-
-    @Test
-    public void testPartitionKeyColumnRelations() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a text, b int, c int, d int, primary key((a, b), c))");
-        execute("insert into %s (a, b, c, d) values (?, ?, ?, ?)", "first", 1, 1, 1);
-        execute("insert into %s (a, b, c, d) values (?, ?, ?, ?)", "first", 2, 2, 2);
-        execute("insert into %s (a, b, c, d) values (?, ?, ?, ?)", "first", 3, 3, 3);
-        execute("insert into %s (a, b, c, d) values (?, ?, ?, ?)", "first", 4, 4, 4);
-        execute("insert into %s (a, b, c, d) values (?, ?, ?, ?)", "second", 1, 1, 1);
-        execute("insert into %s (a, b, c, d) values (?, ?, ?, ?)", "second", 4, 4, 4);
-
-        assertRows(execute("select * from %s where a = ? and b = ?", "first", 2),
-                   row("first", 2, 2, 2));
-
-        assertRows(execute("select * from %s where a in (?, ?) and b in (?, ?)", "first", "second", 2, 3),
-                   row("first", 2, 2, 2),
-                   row("first", 3, 3, 3));
-
-        assertRows(execute("select * from %s where a in (?, ?) and b = ?", "first", "second", 4),
-                   row("first", 4, 4, 4),
-                   row("second", 4, 4, 4));
-
-        assertRows(execute("select * from %s where a = ? and b in (?, ?)", "first", 3, 4),
-                   row("first", 3, 3, 3),
-                   row("first", 4, 4, 4));
-
-        assertRows(execute("select * from %s where a in (?, ?) and b in (?, ?)", "first", "second", 1, 4),
-                   row("first", 1, 1, 1),
-                   row("first", 4, 4, 4),
-                   row("second", 1, 1, 1),
-                   row("second", 4, 4, 4));
-
-        assertInvalidMessage("Partition key parts: b must be restricted as other parts are",
-                             "select * from %s where a in (?, ?)", "first", "second");
-        assertInvalidMessage("Partition key parts: b must be restricted as other parts are",
-                             "select * from %s where a = ?", "first");
-        assertInvalidMessage("b cannot be restricted by more than one relation if it includes a IN",
-                             "select * from %s where a = ? AND b IN (?, ?) AND b = ?", "first", 2, 2, 3);
-        assertInvalidMessage("b cannot be restricted by more than one relation if it includes an Equal",
-                             "select * from %s where a = ? AND b = ? AND b IN (?, ?)", "first", 2, 2, 3);
-        assertInvalidMessage("a cannot be restricted by more than one relation if it includes a IN",
-                             "select * from %s where a IN (?, ?) AND a = ? AND b = ?", "first", "second", "first", 3);
-        assertInvalidMessage("a cannot be restricted by more than one relation if it includes an Equal",
-                             "select * from %s where a = ? AND a IN (?, ?) AND b IN (?, ?)", "first", "second", "first", 2, 3);
-    }
-
-    @Test
-    public void testClusteringColumnRelationsWithClusteringOrder() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a text, b int, c int, d int, primary key(a, b, c)) WITH CLUSTERING ORDER BY (b DESC);");
-        execute("insert into %s (a, b, c, d) values (?, ?, ?, ?)", "first", 1, 5, 1);
-        execute("insert into %s (a, b, c, d) values (?, ?, ?, ?)", "first", 2, 6, 2);
-        execute("insert into %s (a, b, c, d) values (?, ?, ?, ?)", "first", 3, 7, 3);
-        execute("insert into %s (a, b, c, d) values (?, ?, ?, ?)", "second", 4, 8, 4);
-
-        assertRows(execute("select * from %s where a = ? and c in (?, ?) and b in (?, ?) order by b DESC",
-                           "first", 7, 6, 3, 2),
-                   row("first", 3, 7, 3),
-                   row("first", 2, 6, 2));
-
-        assertRows(execute("select * from %s where a = ? and c in (?, ?) and b in (?, ?) order by b ASC",
-                           "first", 7, 6, 3, 2),
-                   row("first", 2, 6, 2),
-                   row("first", 3, 7, 3));
-    }
-
-    @Test
-    public void testAllowFilteringWithClusteringColumn() throws Throwable
-    {
-        createTable("CREATE TABLE %s (k int, c int, v int, PRIMARY KEY (k, c))");
-
-        execute("INSERT INTO %s (k, c, v) VALUES(?, ?, ?)", 1, 2, 1);
-        execute("INSERT INTO %s (k, c, v) VALUES(?, ?, ?)", 1, 3, 2);
-        execute("INSERT INTO %s (k, c, v) VALUES(?, ?, ?)", 2, 2, 3);
-
-        // Don't require filtering, always allowed
-        assertRows(execute("SELECT * FROM %s WHERE k = ?", 1),
-                   row(1, 2, 1),
-                   row(1, 3, 2));
-
-        assertRows(execute("SELECT * FROM %s WHERE k = ? AND c > ?", 1, 2), row(1, 3, 2));
-
-        assertRows(execute("SELECT * FROM %s WHERE k = ? AND c = ?", 1, 2), row(1, 2, 1));
-
-        assertRows(execute("SELECT * FROM %s WHERE k = ? ALLOW FILTERING", 1),
-                   row(1, 2, 1),
-                   row(1, 3, 2));
-
-        assertRows(execute("SELECT * FROM %s WHERE k = ? AND c > ? ALLOW FILTERING", 1, 2), row(1, 3, 2));
-
-        assertRows(execute("SELECT * FROM %s WHERE k = ? AND c = ? ALLOW FILTERING", 1, 2), row(1, 2, 1));
-
-        // Require filtering, allowed only with ALLOW FILTERING
-        assertInvalidMessage("Cannot execute this query as it might involve data filtering",
-                             "SELECT * FROM %s WHERE c = ?", 2);
-        assertInvalidMessage("Cannot execute this query as it might involve data filtering",
-                             "SELECT * FROM %s WHERE c > ? AND c <= ?", 2, 4);
-
-        assertRows(execute("SELECT * FROM %s WHERE c = ? ALLOW FILTERING", 2),
-                   row(1, 2, 1),
-                   row(2, 2, 3));
-
-        assertRows(execute("SELECT * FROM %s WHERE c > ? AND c <= ? ALLOW FILTERING", 2, 4), row(1, 3, 2));
-    }
-
-    @Test
-    public void testAllowFilteringWithIndexedColumn() throws Throwable
-    {
-        createTable("CREATE TABLE %s (k int PRIMARY KEY, a int, b int)");
-        createIndex("CREATE INDEX ON %s(a)");
-
-        execute("INSERT INTO %s(k, a, b) VALUES(?, ?, ?)", 1, 10, 100);
-        execute("INSERT INTO %s(k, a, b) VALUES(?, ?, ?)", 2, 20, 200);
-        execute("INSERT INTO %s(k, a, b) VALUES(?, ?, ?)", 3, 30, 300);
-        execute("INSERT INTO %s(k, a, b) VALUES(?, ?, ?)", 4, 40, 400);
-
-        // Don't require filtering, always allowed
-        assertRows(execute("SELECT * FROM %s WHERE k = ?", 1), row(1, 10, 100));
-        assertRows(execute("SELECT * FROM %s WHERE a = ?", 20), row(2, 20, 200));
-        assertRows(execute("SELECT * FROM %s WHERE k = ? ALLOW FILTERING", 1), row(1, 10, 100));
-        assertRows(execute("SELECT * FROM %s WHERE a = ? ALLOW FILTERING", 20), row(2, 20, 200));
-
-        assertInvalid("SELECT * FROM %s WHERE a = ? AND b = ?");
-        assertRows(execute("SELECT * FROM %s WHERE a = ? AND b = ? ALLOW FILTERING", 20, 200), row(2, 20, 200));
-    }
-
-    @Test
-    public void testIndexQueriesOnComplexPrimaryKey() throws Throwable
-    {
-        createTable("CREATE TABLE %s (pk0 int, pk1 int, ck0 int, ck1 int, ck2 int, value int, PRIMARY KEY ((pk0, pk1), ck0, ck1, ck2))");
-
-        createIndex("CREATE INDEX ON %s (ck1)");
-        createIndex("CREATE INDEX ON %s (ck2)");
-        createIndex("CREATE INDEX ON %s (pk0)");
-        createIndex("CREATE INDEX ON %s (ck0)");
-
-        execute("INSERT INTO %s (pk0, pk1, ck0, ck1, ck2, value) VALUES (?, ?, ?, ?, ?, ?)", 0, 1, 2, 3, 4, 5);
-        execute("INSERT INTO %s (pk0, pk1, ck0, ck1, ck2, value) VALUES (?, ?, ?, ?, ?, ?)", 1, 2, 3, 4, 5, 0);
-        execute("INSERT INTO %s (pk0, pk1, ck0, ck1, ck2, value) VALUES (?, ?, ?, ?, ?, ?)", 2, 3, 4, 5, 0, 1);
-        execute("INSERT INTO %s (pk0, pk1, ck0, ck1, ck2, value) VALUES (?, ?, ?, ?, ?, ?)", 3, 4, 5, 0, 1, 2);
-        execute("INSERT INTO %s (pk0, pk1, ck0, ck1, ck2, value) VALUES (?, ?, ?, ?, ?, ?)", 4, 5, 0, 1, 2, 3);
-        execute("INSERT INTO %s (pk0, pk1, ck0, ck1, ck2, value) VALUES (?, ?, ?, ?, ?, ?)", 5, 0, 1, 2, 3, 4);
-
-        assertRows(execute("SELECT value FROM %s WHERE pk0 = 2"), row(1));
-        assertRows(execute("SELECT value FROM %s WHERE ck0 = 0"), row(3));
-        assertRows(execute("SELECT value FROM %s WHERE pk0 = 3 AND pk1 = 4 AND ck1 = 0"), row(2));
-        assertRows(execute("SELECT value FROM %s WHERE pk0 = 5 AND pk1 = 0 AND ck0 = 1 AND ck2 = 3 ALLOW FILTERING"), row(4));
-    }
-
-    @Test
-    public void testIndexOnClusteringColumns() throws Throwable
-    {
-        createTable("CREATE TABLE %s (id1 int, id2 int, author text, time bigint, v1 text, v2 text, PRIMARY KEY ((id1, id2), author, time))");
-        createIndex("CREATE INDEX ON %s(time)");
-        createIndex("CREATE INDEX ON %s(id2)");
-
-        execute("INSERT INTO %s(id1, id2, author, time, v1, v2) VALUES(0, 0, 'bob', 0, 'A', 'A')");
-        execute("INSERT INTO %s(id1, id2, author, time, v1, v2) VALUES(0, 0, 'bob', 1, 'B', 'B')");
-        execute("INSERT INTO %s(id1, id2, author, time, v1, v2) VALUES(0, 1, 'bob', 2, 'C', 'C')");
-        execute("INSERT INTO %s(id1, id2, author, time, v1, v2) VALUES(0, 0, 'tom', 0, 'D', 'D')");
-        execute("INSERT INTO %s(id1, id2, author, time, v1, v2) VALUES(0, 1, 'tom', 1, 'E', 'E')");
-
-        assertRows(execute("SELECT v1 FROM %s WHERE time = 1"), row("B"), row("E"));
-
-        assertRows(execute("SELECT v1 FROM %s WHERE id2 = 1"), row("C"), row("E"));
-
-        assertRows(execute("SELECT v1 FROM %s WHERE id1 = 0 AND id2 = 0 AND author = 'bob' AND time = 0"), row("A"));
-
-        // Test for CASSANDRA-8206
-        execute("UPDATE %s SET v2 = null WHERE id1 = 0 AND id2 = 0 AND author = 'bob' AND time = 1");
-
-        assertRows(execute("SELECT v1 FROM %s WHERE id2 = 0"), row("A"), row("B"), row("D"));
-
-        assertRows(execute("SELECT v1 FROM %s WHERE time = 1"), row("B"), row("E"));
-
-        assertInvalidMessage("IN restrictions are not supported on indexed columns",
-                             "SELECT v1 FROM %s WHERE id2 = 0 and time IN (1, 2) ALLOW FILTERING");
-    }
-
-    @Test
-    public void testCompositeIndexWithPrimaryKey() throws Throwable
-    {
-        createTable("CREATE TABLE %s (blog_id int, time1 int, time2 int, author text, content text, PRIMARY KEY (blog_id, time1, time2))");
-
-        createIndex("CREATE INDEX ON %s(author)");
-
-        String req = "INSERT INTO %s (blog_id, time1, time2, author, content) VALUES (?, ?, ?, ?, ?)";
-        execute(req, 1, 0, 0, "foo", "bar1");
-        execute(req, 1, 0, 1, "foo", "bar2");
-        execute(req, 2, 1, 0, "foo", "baz");
-        execute(req, 3, 0, 1, "gux", "qux");
-
-        assertRows(execute("SELECT blog_id, content FROM %s WHERE author='foo'"),
-                   row(1, "bar1"),
-                   row(1, "bar2"),
-                   row(2, "baz"));
-        assertRows(execute("SELECT blog_id, content FROM %s WHERE time1 > 0 AND author='foo' ALLOW FILTERING"), row(2, "baz"));
-        assertRows(execute("SELECT blog_id, content FROM %s WHERE time1 = 1 AND author='foo' ALLOW FILTERING"), row(2, "baz"));
-        assertRows(execute("SELECT blog_id, content FROM %s WHERE time1 = 1 AND time2 = 0 AND author='foo' ALLOW FILTERING"),
-                   row(2, "baz"));
-        assertEmpty(execute("SELECT content FROM %s WHERE time1 = 1 AND time2 = 1 AND author='foo' ALLOW FILTERING"));
-        assertEmpty(execute("SELECT content FROM %s WHERE time1 = 1 AND time2 > 0 AND author='foo' ALLOW FILTERING"));
-
-        assertInvalidMessage("Cannot execute this query as it might involve data filtering",
-                             "SELECT content FROM %s WHERE time2 >= 0 AND author='foo'");
-    }
-
-    @Test
-    public void testRangeQueryOnIndex() throws Throwable
-    {
-        createTable("CREATE TABLE %s (id int primary key, row int, setid int);");
-        createIndex("CREATE INDEX ON %s (setid)");
-
-        String q = "INSERT INTO %s (id, row, setid) VALUES (?, ?, ?);";
-        execute(q, 0, 0, 0);
-        execute(q, 1, 1, 0);
-        execute(q, 2, 2, 0);
-        execute(q, 3, 3, 0);
-
-        assertInvalidMessage("Cannot execute this query as it might involve data filtering",
-                             "SELECT * FROM %s WHERE setid = 0 AND row < 1;");
-        assertRows(execute("SELECT * FROM %s WHERE setid = 0 AND row < 1 ALLOW FILTERING;"), row(0, 0, 0));
-    }
-
-    @Test
-    public void testEmptyIN() throws Throwable
-    {
-        for (String compactOption : new String[] { "", " WITH COMPACT STORAGE" })
-        {
-            createTable("CREATE TABLE %s (k1 int, k2 int, v int, PRIMARY KEY (k1, k2))" + compactOption);
-
-            for (int i = 0; i <= 2; i++)
-                for (int j = 0; j <= 2; j++)
-                    execute("INSERT INTO %s (k1, k2, v) VALUES (?, ?, ?)", i, j, i + j);
-
-            assertEmpty(execute("SELECT v FROM %s WHERE k1 IN ()"));
-            assertEmpty(execute("SELECT v FROM %s WHERE k1 = 0 AND k2 IN ()"));
-        }
-    }
-
-    @Test
-    public void testINWithDuplicateValue() throws Throwable
-    {
-        for (String compactOption : new String[] { "", " WITH COMPACT STORAGE" })
-        {
-            createTable("CREATE TABLE %s (k1 int, k2 int, v int, PRIMARY KEY (k1, k2))" + compactOption);
-            execute("INSERT INTO %s (k1,  k2, v) VALUES (?, ?, ?)", 1, 1, 1);
-
-            assertRows(execute("SELECT * FROM %s WHERE k1 IN (?, ?)", 1, 1),
-                       row(1, 1, 1));
-
-            assertRows(execute("SELECT * FROM %s WHERE k1 IN (?, ?) AND k2 IN (?, ?)", 1, 1, 1, 1),
-                       row(1, 1, 1));
-
-            assertRows(execute("SELECT * FROM %s WHERE k1 = ? AND k2 IN (?, ?)", 1, 1, 1),
-                       row(1, 1, 1));
-        }
-    }
-
-    @Test
-    public void testLargeClusteringINValues() throws Throwable
-    {
-        createTable("CREATE TABLE %s (k int, c int, v int, PRIMARY KEY (k, c))");
-        execute("INSERT INTO %s (k, c, v) VALUES (0, 0, 0)");
-        List<Integer> inValues = new ArrayList<>(10000);
-        for (int i = 0; i < 10000; i++)
-            inValues.add(i);
-        assertRows(execute("SELECT * FROM %s WHERE k=? AND c IN ?", 0, inValues),
-                row(0, 0, 0));
-    }
-
-    @Test
-    public void testMultiplePartitionKeyWithIndex() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a int, b int, c int, d int, e int, f int, PRIMARY KEY ((a, b), c, d, e))");
-        createIndex("CREATE INDEX ON %s (c)");
-        createIndex("CREATE INDEX ON %s (f)");
-
-        execute("INSERT INTO %s (a, b, c, d, e, f) VALUES (?, ?, ?, ?, ?, ?)", 0, 0, 0, 0, 0, 0);
-        execute("INSERT INTO %s (a, b, c, d, e, f) VALUES (?, ?, ?, ?, ?, ?)", 0, 0, 0, 1, 0, 1);
-        execute("INSERT INTO %s (a, b, c, d, e, f) VALUES (?, ?, ?, ?, ?, ?)", 0, 0, 0, 1, 1, 2);
-
-        execute("INSERT INTO %s (a, b, c, d, e, f) VALUES (?, ?, ?, ?, ?, ?)", 0, 0, 1, 0, 0, 3);
-        execute("INSERT INTO %s (a, b, c, d, e, f) VALUES (?, ?, ?, ?, ?, ?)", 0, 0, 1, 1, 0, 4);
-        execute("INSERT INTO %s (a, b, c, d, e, f) VALUES (?, ?, ?, ?, ?, ?)", 0, 0, 1, 1, 1, 5);
-
-        execute("INSERT INTO %s (a, b, c, d, e, f) VALUES (?, ?, ?, ?, ?, ?)", 0, 0, 2, 0, 0, 5);
-
-        assertInvalidMessage("Cannot execute this query as it might involve data filtering",
-                             "SELECT * FROM %s WHERE a = ? AND c = ?", 0, 1);
-        assertRows(execute("SELECT * FROM %s WHERE a = ? AND c = ? ALLOW FILTERING", 0, 1),
-                   row(0, 0, 1, 0, 0, 3),
-                   row(0, 0, 1, 1, 0, 4),
-                   row(0, 0, 1, 1, 1, 5));
-
-        assertInvalidMessage("Cannot execute this query as it might involve data filtering",
-                             "SELECT * FROM %s WHERE a = ? AND c = ? AND d = ?", 0, 1, 1);
-        assertRows(execute("SELECT * FROM %s WHERE a = ? AND c = ? AND d = ? ALLOW FILTERING", 0, 1, 1),
-                   row(0, 0, 1, 1, 0, 4),
-                   row(0, 0, 1, 1, 1, 5));
-
-        assertInvalidMessage("Partition key parts: b must be restricted as other parts are",
-                             "SELECT * FROM %s WHERE a = ? AND c IN (?) AND  d IN (?) ALLOW FILTERING", 0, 1, 1);
-
-        assertInvalidMessage("Partition key parts: b must be restricted as other parts are",
-                             "SELECT * FROM %s WHERE a = ? AND (c, d) >= (?, ?) ALLOW FILTERING", 0, 1, 1);
-
-        assertInvalidMessage("Cannot execute this query as it might involve data filtering",
-                             "SELECT * FROM %s WHERE a = ? AND c IN (?) AND f = ?", 0, 1, 5);
-        assertRows(execute("SELECT * FROM %s WHERE a = ? AND c IN (?) AND f = ? ALLOW FILTERING", 0, 1, 5),
-                   row(0, 0, 1, 1, 1, 5));
-
-        assertInvalidMessage("Cannot execute this query as it might involve data filtering",
-                             "SELECT * FROM %s WHERE a = ? AND c IN (?, ?) AND f = ?", 0, 1, 2, 5);
-        assertRows(execute("SELECT * FROM %s WHERE a = ? AND c IN (?, ?) AND f = ? ALLOW FILTERING", 0, 1, 2, 5),
-                   row(0, 0, 1, 1, 1, 5),
-                   row(0, 0, 2, 0, 0, 5));
-
-        assertInvalidMessage("Cannot execute this query as it might involve data filtering",
-                             "SELECT * FROM %s WHERE a = ? AND c IN (?) AND d IN (?) AND f = ?", 0, 1, 0, 3);
-        assertRows(execute("SELECT * FROM %s WHERE a = ? AND c IN (?) AND d IN (?) AND f = ? ALLOW FILTERING", 0, 1, 0, 3),
-                   row(0, 0, 1, 0, 0, 3));
-
-        assertInvalidMessage("Partition key parts: b must be restricted as other parts are",
-                             "SELECT * FROM %s WHERE a = ? AND c >= ? ALLOW FILTERING", 0, 1);
-
-        assertInvalidMessage("Cannot execute this query as it might involve data filtering",
-                             "SELECT * FROM %s WHERE a = ? AND c >= ? AND f = ?", 0, 1, 5);
-        assertRows(execute("SELECT * FROM %s WHERE a = ? AND c >= ? AND f = ? ALLOW FILTERING", 0, 1, 5),
-                   row(0, 0, 1, 1, 1, 5),
-                   row(0, 0, 2, 0, 0, 5));
-
-        assertInvalidMessage("Cannot execute this query as it might involve data filtering",
-                             "SELECT * FROM %s WHERE a = ? AND c = ? AND d >= ? AND f = ?", 0, 1, 1, 5);
-        assertRows(execute("SELECT * FROM %s WHERE a = ? AND c = ? AND d >= ? AND f = ? ALLOW FILTERING", 0, 1, 1, 5),
-                   row(0, 0, 1, 1, 1, 5));
-    }
-
-    @Test
-    public void testFunctionCallWithUnset() throws Throwable
-    {
-        createTable("CREATE TABLE %s (k int PRIMARY KEY, s text, i int)");
-
-        assertInvalidMessage("Invalid unset value for argument in call to function token",
-                             "SELECT * FROM %s WHERE token(k) >= token(?)", unset());
-        assertInvalidMessage("Invalid unset value for argument in call to function blobasint",
-                             "SELECT * FROM %s WHERE k = blobAsInt(?)", unset());
-    }
-
-    @Test
-    public void testLimitWithUnset() throws Throwable
-    {
-        createTable("CREATE TABLE %s (k int PRIMARY KEY, i int)");
-        execute("INSERT INTO %s (k, i) VALUES (1, 1)");
-        execute("INSERT INTO %s (k, i) VALUES (2, 1)");
-        assertRows(execute("SELECT k FROM %s LIMIT ?", unset()), // treat as 'unlimited'
-                row(1),
-                row(2)
-        );
-    }
-
-    @Test
-    public void testWithUnsetValues() throws Throwable
-    {
-        createTable("CREATE TABLE %s (k int, i int, j int, s text, PRIMARY KEY(k,i,j))");
-        createIndex("CREATE INDEX s_index ON %s (s)");
-        // partition key
-        assertInvalidMessage("Invalid unset value for column k", "SELECT * from %s WHERE k = ?", unset());
-        assertInvalidMessage("Invalid unset value for column k", "SELECT * from %s WHERE k IN ?", unset());
-        assertInvalidMessage("Invalid unset value for column k", "SELECT * from %s WHERE k IN(?)", unset());
-        assertInvalidMessage("Invalid unset value for column k", "SELECT * from %s WHERE k IN(?,?)", 1, unset());
-        // clustering column
-        assertInvalidMessage("Invalid unset value for column i", "SELECT * from %s WHERE k = 1 AND i = ?", unset());
-        assertInvalidMessage("Invalid unset value for column i", "SELECT * from %s WHERE k = 1 AND i IN ?", unset());
-        assertInvalidMessage("Invalid unset value for column i", "SELECT * from %s WHERE k = 1 AND i IN(?)", unset());
-        assertInvalidMessage("Invalid unset value for column i", "SELECT * from %s WHERE k = 1 AND i IN(?,?)", 1, unset());
-        assertInvalidMessage("Invalid unset value for column i", "SELECT * from %s WHERE i = ? ALLOW FILTERING", unset());
-        // indexed column
-        assertInvalidMessage("Unsupported unset value for indexed column s", "SELECT * from %s WHERE s = ?", unset());
-        // range
-        assertInvalidMessage("Invalid unset value for column i", "SELECT * from %s WHERE k = 1 AND i > ?", unset());
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/01115f72/test/unit/org/apache/cassandra/cql3/SliceQueryFilterWithTombstonesTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/SliceQueryFilterWithTombstonesTest.java b/test/unit/org/apache/cassandra/cql3/SliceQueryFilterWithTombstonesTest.java
deleted file mode 100644
index a3f7197..0000000
--- a/test/unit/org/apache/cassandra/cql3/SliceQueryFilterWithTombstonesTest.java
+++ /dev/null
@@ -1,170 +0,0 @@
-/*
- * 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;
-
-import java.util.concurrent.TimeUnit;
-
-import com.google.common.base.Throwables;
-
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.db.filter.TombstoneOverwhelmingException;
-
-import static junit.framework.Assert.assertTrue;
-import static junit.framework.Assert.fail;
-
-/**
- * Test that TombstoneOverwhelmingException gets thrown when it should be and doesn't when it shouldn't be.
- */
-public class SliceQueryFilterWithTombstonesTest extends CQLTester
-{
-    static final int ORIGINAL_THRESHOLD = DatabaseDescriptor.getTombstoneFailureThreshold();
-    static final int THRESHOLD = 100;
-
-    @BeforeClass
-    public static void setUp() throws Throwable
-    {
-        DatabaseDescriptor.setTombstoneFailureThreshold(THRESHOLD);
-    }
-
-    @AfterClass
-    public static void tearDown()
-    {
-        DatabaseDescriptor.setTombstoneFailureThreshold(ORIGINAL_THRESHOLD);
-    }
-
-    @Test
-    public void testBelowThresholdSelect() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a text, b text, c text, PRIMARY KEY (a, b));");
-
-        // insert exactly the amount of tombstones that shouldn't trigger an exception
-        for (int i = 0; i < THRESHOLD; i++)
-            execute("INSERT INTO %s (a, b, c) VALUES ('key', 'column" + i + "', null);");
-
-        try
-        {
-            execute("SELECT * FROM %s WHERE a = 'key';");
-        }
-        catch (Throwable e)
-        {
-            fail("SELECT with tombstones below the threshold should not have failed, but has: " + e);
-        }
-    }
-
-    @Test
-    public void testBeyondThresholdSelect() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a text, b text, c text, PRIMARY KEY (a, b));");
-
-        // insert exactly the amount of tombstones that *SHOULD* trigger an exception
-        for (int i = 0; i < THRESHOLD + 1; i++)
-            execute("INSERT INTO %s (a, b, c) VALUES ('key', 'column" + i + "', null);");
-
-        try
-        {
-            execute("SELECT * FROM %s WHERE a = 'key';");
-            fail("SELECT with tombstones beyond the threshold should have failed, but hasn't");
-        }
-        catch (Throwable e)
-        {
-            String error = "Expected exception instanceof TombstoneOverwhelmingException instead got "
-                          + System.lineSeparator()
-                          + Throwables.getStackTraceAsString(e);
-            assertTrue(error, e instanceof TombstoneOverwhelmingException);
-        }
-    }
-
-    @Test
-    public void testAllShadowedSelect() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a text, b text, c text, PRIMARY KEY (a, b));");
-
-        // insert exactly the amount of tombstones that *SHOULD* normally trigger an exception
-        for (int i = 0; i < THRESHOLD + 1; i++)
-            execute("INSERT INTO %s (a, b, c) VALUES ('key', 'column" + i + "', null);");
-
-        // delete all with a partition level tombstone
-        execute("DELETE FROM %s WHERE a = 'key'");
-
-        try
-        {
-            execute("SELECT * FROM %s WHERE a = 'key';");
-        }
-        catch (Throwable e)
-        {
-            fail("SELECT with tombstones shadowed by a partition tombstone should not have failed, but has: " + e);
-        }
-    }
-
-    @Test
-    public void testLiveShadowedCellsSelect() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a text, b text, c text, PRIMARY KEY (a, b));");
-
-        for (int i = 0; i < THRESHOLD + 1; i++)
-            execute("INSERT INTO %s (a, b, c) VALUES ('key', 'column" + i + "', 'column');");
-
-        // delete all with a partition level tombstone
-        execute("DELETE FROM %s WHERE a = 'key'");
-
-        try
-        {
-            execute("SELECT * FROM %s WHERE a = 'key';");
-        }
-        catch (Throwable e)
-        {
-            fail("SELECT with regular cells shadowed by a partition tombstone should not have failed, but has: " + e);
-        }
-    }
-
-    @Test
-    public void testExpiredTombstones() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a text, b text, c text, PRIMARY KEY (a, b)) WITH gc_grace_seconds = 1;");
-
-        for (int i = 0; i < THRESHOLD + 1; i++)
-            execute("INSERT INTO %s (a, b, c) VALUES ('key', 'column" + i + "', null);");
-
-        // not yet past gc grace - must throw a TOE
-        try
-        {
-            execute("SELECT * FROM %s WHERE a = 'key';");
-            fail("SELECT with tombstones beyond the threshold should have failed, but hasn't");
-        }
-        catch (Throwable e)
-        {
-            assertTrue(e instanceof TombstoneOverwhelmingException);
-        }
-
-        // sleep past gc grace
-        TimeUnit.SECONDS.sleep(2);
-
-        // past gc grace - must not throw a TOE now
-        try
-        {
-            execute("SELECT * FROM %s WHERE a = 'key';");
-        }
-        catch (Throwable e)
-        {
-            fail("SELECT with expired tombstones beyond the threshold should not have failed, but has: " + e);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/01115f72/test/unit/org/apache/cassandra/cql3/StaticColumnsQueryTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/StaticColumnsQueryTest.java b/test/unit/org/apache/cassandra/cql3/StaticColumnsQueryTest.java
deleted file mode 100644
index e27f968..0000000
--- a/test/unit/org/apache/cassandra/cql3/StaticColumnsQueryTest.java
+++ /dev/null
@@ -1,280 +0,0 @@
-/*
- * 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;
-
-import org.junit.Test;
-
-/**
- * Test column ranges and ordering with static column in table
- */
-public class StaticColumnsQueryTest extends CQLTester
-{
-    @Test
-    public void testSingleClustering() throws Throwable
-    {
-        createTable("CREATE TABLE %s (p text, c text, v text, s text static, PRIMARY KEY (p, c))");
-
-        execute("INSERT INTO %s(p, c, v, s) values (?, ?, ?, ?)", "p1", "k1", "v1", "sv1");
-        execute("INSERT INTO %s(p, c, v) values (?, ?, ?)", "p1", "k2", "v2");
-        execute("INSERT INTO %s(p, s) values (?, ?)", "p2", "sv2");
-
-        assertRows(execute("SELECT * FROM %s WHERE p=?", "p1"),
-            row("p1", "k1", "sv1", "v1"),
-            row("p1", "k2", "sv1", "v2")
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE p=?", "p2"),
-            row("p2", null, "sv2", null)
-        );
-
-        // Ascending order
-
-        assertRows(execute("SELECT * FROM %s WHERE p=? ORDER BY c ASC", "p1"),
-            row("p1", "k1", "sv1", "v1"),
-            row("p1", "k2", "sv1", "v2")
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE p=? ORDER BY c ASC", "p2"),
-            row("p2", null, "sv2", null)
-        );
-
-        // Descending order
-
-        assertRows(execute("SELECT * FROM %s WHERE p=? ORDER BY c DESC", "p1"),
-            row("p1", "k2", "sv1", "v2"),
-            row("p1", "k1", "sv1", "v1")
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE p=? ORDER BY c DESC", "p2"),
-            row("p2", null, "sv2", null)
-        );
-
-        // No order with one relation
-
-        assertRows(execute("SELECT * FROM %s WHERE p=? AND c>=?", "p1", "k1"),
-            row("p1", "k1", "sv1", "v1"),
-            row("p1", "k2", "sv1", "v2")
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE p=? AND c>=?", "p1", "k2"),
-            row("p1", "k2", "sv1", "v2")
-        );
-
-        assertEmpty(execute("SELECT * FROM %s WHERE p=? AND c>=?", "p1", "k3"));
-
-        assertRows(execute("SELECT * FROM %s WHERE p=? AND c =?", "p1", "k1"),
-            row("p1", "k1", "sv1", "v1")
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE p=? AND c<=?", "p1", "k1"),
-            row("p1", "k1", "sv1", "v1")
-        );
-
-        assertEmpty(execute("SELECT * FROM %s WHERE p=? AND c<=?", "p1", "k0"));
-
-        // Ascending with one relation
-
-        assertRows(execute("SELECT * FROM %s WHERE p=? AND c>=? ORDER BY c ASC", "p1", "k1"),
-            row("p1", "k1", "sv1", "v1"),
-            row("p1", "k2", "sv1", "v2")
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE p=? AND c>=? ORDER BY c ASC", "p1", "k2"),
-            row("p1", "k2", "sv1", "v2")
-        );
-
-        assertEmpty(execute("SELECT * FROM %s WHERE p=? AND c>=? ORDER BY c ASC", "p1", "k3"));
-
-        assertRows(execute("SELECT * FROM %s WHERE p=? AND c =? ORDER BY c ASC", "p1", "k1"),
-            row("p1", "k1", "sv1", "v1")
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE p=? AND c<=? ORDER BY c ASC", "p1", "k1"),
-            row("p1", "k1", "sv1", "v1")
-        );
-
-        assertEmpty(execute("SELECT * FROM %s WHERE p=? AND c<=? ORDER BY c ASC", "p1", "k0"));
-
-        // Descending with one relation
-
-        assertRows(execute("SELECT * FROM %s WHERE p=? AND c>=? ORDER BY c DESC", "p1", "k1"),
-            row("p1", "k2", "sv1", "v2"),
-            row("p1", "k1", "sv1", "v1")
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE p=? AND c>=? ORDER BY c DESC", "p1", "k2"),
-            row("p1", "k2", "sv1", "v2")
-        );
-
-        assertEmpty(execute("SELECT * FROM %s WHERE p=? AND c>=? ORDER BY c DESC", "p1", "k3"));
-
-        assertRows(execute("SELECT * FROM %s WHERE p=? AND c =? ORDER BY c DESC", "p1", "k1"),
-            row("p1", "k1", "sv1", "v1")
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE p=? AND c<=? ORDER BY c DESC", "p1", "k1"),
-            row("p1", "k1", "sv1", "v1")
-        );
-
-        assertEmpty(execute("SELECT * FROM %s WHERE p=? AND c<=? ORDER BY c DESC", "p1", "k0"));
-
-        // IN
-
-        assertRows(execute("SELECT * FROM %s WHERE p=? AND c IN (?, ?)", "p1", "k1", "k2"),
-            row("p1", "k1", "sv1", "v1"),
-            row("p1", "k2", "sv1", "v2")
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE p=? AND c IN (?, ?) ORDER BY c ASC", "p1", "k1", "k2"),
-            row("p1", "k1", "sv1", "v1"),
-            row("p1", "k2", "sv1", "v2")
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE p=? AND c IN (?, ?) ORDER BY c DESC", "p1", "k1", "k2"),
-            row("p1", "k2", "sv1", "v2"),
-            row("p1", "k1", "sv1", "v1")
-        );
-    }
-
-    @Test
-    public void testSingleClusteringReversed() throws Throwable
-    {
-        createTable("CREATE TABLE %s (p text, c text, v text, s text static, PRIMARY KEY (p, c)) WITH CLUSTERING ORDER BY (c DESC)");
-
-        execute("INSERT INTO %s(p, c, v, s) values (?, ?, ?, ?)", "p1", "k1", "v1", "sv1");
-        execute("INSERT INTO %s(p, c, v) values (?, ?, ?)", "p1", "k2", "v2");
-        execute("INSERT INTO %s(p, s) values (?, ?)", "p2", "sv2");
-
-        assertRows(execute("SELECT * FROM %s WHERE p=?", "p1"),
-            row("p1", "k2", "sv1", "v2"),
-            row("p1", "k1", "sv1", "v1")
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE p=?", "p2"),
-            row("p2", null, "sv2", null)
-        );
-
-        // Ascending order
-
-        assertRows(execute("SELECT * FROM %s WHERE p=? ORDER BY c ASC", "p1"),
-            row("p1", "k1", "sv1", "v1"),
-            row("p1", "k2", "sv1", "v2")
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE p=? ORDER BY c ASC", "p2"),
-            row("p2", null, "sv2", null)
-        );
-
-        // Descending order
-
-        assertRows(execute("SELECT * FROM %s WHERE p=? ORDER BY c DESC", "p1"),
-            row("p1", "k2", "sv1", "v2"),
-            row("p1", "k1", "sv1", "v1")
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE p=? ORDER BY c DESC", "p2"),
-            row("p2", null, "sv2", null)
-        );
-
-        // No order with one relation
-
-        assertRows(execute("SELECT * FROM %s WHERE p=? AND c>=?", "p1", "k1"),
-            row("p1", "k2", "sv1", "v2"),
-            row("p1", "k1", "sv1", "v1")
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE p=? AND c>=?", "p1", "k2"),
-            row("p1", "k2", "sv1", "v2")
-        );
-
-        assertEmpty(execute("SELECT * FROM %s WHERE p=? AND c>=?", "p1", "k3"));
-
-        assertRows(execute("SELECT * FROM %s WHERE p=? AND c=?", "p1", "k1"),
-            row("p1", "k1", "sv1", "v1")
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE p=? AND c<=?", "p1", "k1"),
-            row("p1", "k1", "sv1", "v1")
-        );
-
-        assertEmpty(execute("SELECT * FROM %s WHERE p=? AND c<=?", "p1", "k0"));
-
-        // Ascending with one relation
-
-        assertRows(execute("SELECT * FROM %s WHERE p=? AND c>=? ORDER BY c ASC", "p1", "k1"),
-            row("p1", "k1", "sv1", "v1"),
-            row("p1", "k2", "sv1", "v2")
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE p=? AND c>=? ORDER BY c ASC", "p1", "k2"),
-            row("p1", "k2", "sv1", "v2")
-        );
-
-        assertEmpty(execute("SELECT * FROM %s WHERE p=? AND c>=? ORDER BY c ASC", "p1", "k3"));
-
-        assertRows(execute("SELECT * FROM %s WHERE p=? AND c=? ORDER BY c ASC", "p1", "k1"),
-            row("p1", "k1", "sv1", "v1")
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE p=? AND c<=? ORDER BY c ASC", "p1", "k1"),
-            row("p1", "k1", "sv1", "v1")
-        );
-
-        assertEmpty(execute("SELECT * FROM %s WHERE p=? AND c<=? ORDER BY c ASC", "p1", "k0"));
-
-        // Descending with one relation
-
-        assertRows(execute("SELECT * FROM %s WHERE p=? AND c>=? ORDER BY c DESC", "p1", "k1"),
-            row("p1", "k2", "sv1", "v2"),
-            row("p1", "k1", "sv1", "v1")
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE p=? AND c>=? ORDER BY c DESC", "p1", "k2"),
-            row("p1", "k2", "sv1", "v2")
-        );
-
-        assertEmpty(execute("SELECT * FROM %s WHERE p=? AND c>=? ORDER BY c DESC", "p1", "k3"));
-
-        assertRows(execute("SELECT * FROM %s WHERE p=? AND c=? ORDER BY c DESC", "p1", "k1"),
-            row("p1", "k1", "sv1", "v1")
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE p=? AND c<=? ORDER BY c DESC", "p1", "k1"),
-            row("p1", "k1", "sv1", "v1")
-        );
-
-        assertEmpty(execute("SELECT * FROM %s WHERE p=? AND c<=? ORDER BY c DESC", "p1", "k0"));
-
-        // IN
-
-        assertRows(execute("SELECT * FROM %s WHERE p=? AND c IN (?, ?)", "p1", "k1", "k2"),
-            row("p1", "k2", "sv1", "v2"),
-            row("p1", "k1", "sv1", "v1")
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE p=? AND c IN (?, ?) ORDER BY c ASC", "p1", "k1", "k2"),
-            row("p1", "k1", "sv1", "v1"),
-            row("p1", "k2", "sv1", "v2")
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE p=? AND c IN (?, ?) ORDER BY c DESC", "p1", "k1", "k2"),
-            row("p1", "k2", "sv1", "v2"),
-            row("p1", "k1", "sv1", "v1")
-        );
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/01115f72/test/unit/org/apache/cassandra/cql3/ThriftCompatibilityTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/ThriftCompatibilityTest.java b/test/unit/org/apache/cassandra/cql3/ThriftCompatibilityTest.java
index 3125b28..7b72ef8 100644
--- a/test/unit/org/apache/cassandra/cql3/ThriftCompatibilityTest.java
+++ b/test/unit/org/apache/cassandra/cql3/ThriftCompatibilityTest.java
@@ -27,6 +27,7 @@ import org.junit.Test;
 import org.apache.cassandra.SchemaLoader;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
 
 public class ThriftCompatibilityTest extends SchemaLoader
 {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/01115f72/test/unit/org/apache/cassandra/cql3/TimestampTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/TimestampTest.java b/test/unit/org/apache/cassandra/cql3/TimestampTest.java
deleted file mode 100644
index 6673904..0000000
--- a/test/unit/org/apache/cassandra/cql3/TimestampTest.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/*
- * 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;
-
-import org.junit.Test;
-
-public class TimestampTest extends CQLTester
-{
-    @Test
-    public void testNegativeTimestamps() throws Throwable
-    {
-        createTable("CREATE TABLE %s (k int PRIMARY KEY, v int)");
-
-        execute("INSERT INTO %s (k, v) VALUES (?, ?) USING TIMESTAMP ?", 1, 1, -42L);
-        assertRows(execute("SELECT writetime(v) FROM %s WHERE k = ?", 1),
-            row(-42L)
-        );
-
-        assertInvalid("INSERT INTO %s (k, v) VALUES (?, ?) USING TIMESTAMP ?", 2, 2, Long.MIN_VALUE);
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/01115f72/test/unit/org/apache/cassandra/cql3/TupleTypeTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/TupleTypeTest.java b/test/unit/org/apache/cassandra/cql3/TupleTypeTest.java
deleted file mode 100644
index 48f0caf..0000000
--- a/test/unit/org/apache/cassandra/cql3/TupleTypeTest.java
+++ /dev/null
@@ -1,114 +0,0 @@
-/*
- * 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;
-
-import org.junit.Test;
-
-public class TupleTypeTest extends CQLTester
-{
-    @Test
-    public void testTuplePutAndGet() throws Throwable
-    {
-        String[] valueTypes = {"frozen<tuple<int, text, double>>", "tuple<int, text, double>"};
-        for (String valueType : valueTypes)
-        {
-            createTable("CREATE TABLE %s (k int PRIMARY KEY, t " + valueType + ")");
-
-            execute("INSERT INTO %s (k, t) VALUES (?, ?)", 0, tuple(3, "foo", 3.4));
-            execute("INSERT INTO %s (k, t) VALUES (?, ?)", 1, tuple(8, "bar", 0.2));
-            assertAllRows(
-                row(0, tuple(3, "foo", 3.4)),
-                row(1, tuple(8, "bar", 0.2))
-            );
-
-            // nulls
-            execute("INSERT INTO %s (k, t) VALUES (?, ?)", 2, tuple(5, null, 3.4));
-            assertRows(execute("SELECT * FROM %s WHERE k=?", 2),
-                row(2, tuple(5, null, 3.4))
-            );
-
-            // incomplete tuple
-            execute("INSERT INTO %s (k, t) VALUES (?, ?)", 3, tuple(5, "bar"));
-            assertRows(execute("SELECT * FROM %s WHERE k=?", 3),
-                row(3, tuple(5, "bar"))
-            );
-        }
-    }
-
-    @Test
-    public void testNestedTuple() throws Throwable
-    {
-        createTable("CREATE TABLE %s (k int PRIMARY KEY, t frozen<tuple<int, tuple<text, double>>>)");
-
-        execute("INSERT INTO %s (k, t) VALUES (?, ?)", 0, tuple(3, tuple("foo", 3.4)));
-        execute("INSERT INTO %s (k, t) VALUES (?, ?)", 1, tuple(8, tuple("bar", 0.2)));
-        assertAllRows(
-            row(0, tuple(3, tuple("foo", 3.4))),
-            row(1, tuple(8, tuple("bar", 0.2)))
-        );
-    }
-
-    @Test
-    public void testTupleInPartitionKey() throws Throwable
-    {
-        createTable("CREATE TABLE %s (t frozen<tuple<int, text>> PRIMARY KEY)");
-
-        execute("INSERT INTO %s (t) VALUES (?)", tuple(3, "foo"));
-        assertAllRows(row(tuple(3, "foo")));
-    }
-
-    @Test
-    public void testTupleInClusteringKey() throws Throwable
-    {
-        createTable("CREATE TABLE %s (k int, t frozen<tuple<int, text>>, PRIMARY KEY (k, t))");
-
-        execute("INSERT INTO %s (k, t) VALUES (?, ?)", 0, tuple(5, "bar"));
-        execute("INSERT INTO %s (k, t) VALUES (?, ?)", 0, tuple(3, "foo"));
-        execute("INSERT INTO %s (k, t) VALUES (?, ?)", 0, tuple(6, "bar"));
-        execute("INSERT INTO %s (k, t) VALUES (?, ?)", 0, tuple(5, "foo"));
-
-        assertAllRows(
-            row(0, tuple(3, "foo")),
-            row(0, tuple(5, "bar")),
-            row(0, tuple(5, "foo")),
-            row(0, tuple(6, "bar"))
-        );
-    }
-
-    @Test
-    public void testInvalidQueries() throws Throwable
-    {
-        createTable("CREATE TABLE %s (k int PRIMARY KEY, t frozen<tuple<int, text, double>>)");
-
-        assertInvalidSyntax("INSERT INTO %s (k, t) VALUES (0, ())");
-        assertInvalid("INSERT INTO %s (k, t) VALUES (0, (2, 'foo', 3.1, 'bar'))");
-    }
-
-    @Test
-    public void testTupleWithUnsetValues() throws Throwable
-    {
-        createTable("CREATE TABLE %s (k int PRIMARY KEY, t tuple<int, text, double>)");
-        // invalid positional field substitution
-        assertInvalidMessage("Invalid unset value for tuple field number 1",
-                "INSERT INTO %s (k, t) VALUES(0, (3, ?, 2.1))", unset());
-
-        createIndex("CREATE INDEX tuple_index ON %s (t)");
-        // select using unset
-        assertInvalidMessage("Invalid unset value for tuple field number 0", "SELECT * FROM %s WHERE k = ? and t = (?,?,?)", unset(), unset(), unset(), unset());
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/01115f72/test/unit/org/apache/cassandra/cql3/TypeCastTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/TypeCastTest.java b/test/unit/org/apache/cassandra/cql3/TypeCastTest.java
deleted file mode 100644
index 7b9c9a2..0000000
--- a/test/unit/org/apache/cassandra/cql3/TypeCastTest.java
+++ /dev/null
@@ -1,54 +0,0 @@
-/*
- * 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;
-
-import org.junit.Test;
-
-/**
- * Type-casting is mostly using for functions and their use with functions is
- * tested in UFTest. This is a few additional "sanity" tests.
- */
-public class TypeCastTest extends CQLTester
-{
-    @Test
-    public void testTypeCasts() throws Throwable
-    {
-        createTable("CREATE TABLE %s (k int PRIMARY KEY, t text, a ascii, d double, i int)");
-
-        // The followings is fine
-        execute("UPDATE %s SET t = 'foo' WHERE k = ?", 0);
-        execute("UPDATE %s SET t = (ascii)'foo' WHERE k = ?", 0);
-        execute("UPDATE %s SET t = (text)(ascii)'foo' WHERE k = ?", 0);
-        execute("UPDATE %s SET a = 'foo' WHERE k = ?", 0);
-        execute("UPDATE %s SET a = (ascii)'foo' WHERE k = ?", 0);
-
-        // But trying to put some explicitely type-casted text into an ascii
-        // column should be rejected (even though the text is actually ascci)
-        assertInvalid("UPDATE %s SET a = (text)'foo' WHERE k = ?", 0);
-
-        // This is also fine because integer constants works for both integer and float types
-        execute("UPDATE %s SET i = 3 WHERE k = ?", 0);
-        execute("UPDATE %s SET i = (int)3 WHERE k = ?", 0);
-        execute("UPDATE %s SET d = 3 WHERE k = ?", 0);
-        execute("UPDATE %s SET d = (double)3 WHERE k = ?", 0);
-
-        // But values for ints and doubles are not truly compatible (their binary representation differs)
-        assertInvalid("UPDATE %s SET d = (int)3 WHERE k = ?", 0);
-        assertInvalid("UPDATE %s SET i = (double)3 WHERE k = ?", 0);
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/01115f72/test/unit/org/apache/cassandra/cql3/TypeTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/TypeTest.java b/test/unit/org/apache/cassandra/cql3/TypeTest.java
deleted file mode 100644
index 0605554..0000000
--- a/test/unit/org/apache/cassandra/cql3/TypeTest.java
+++ /dev/null
@@ -1,89 +0,0 @@
-/*
- * 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;
-
-import org.junit.Test;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
-
-public class TypeTest extends CQLTester
-{
-    @Test
-    public void testNowToUUIDCompatibility() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a int, b uuid, PRIMARY KEY (a, b))");
-        execute("INSERT INTO %s (a, b) VALUES (0, now())");
-        UntypedResultSet results = execute("SELECT * FROM %s WHERE a=0 AND b < now()");
-        assertEquals(1, results.size());
-    }
-
-    @Test
-    public void testDateCompatibility() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a int, b timestamp, c bigint, d varint, PRIMARY KEY (a, b, c, d))");
-
-        execute("INSERT INTO %s (a, b, c, d) VALUES (0, toUnixTimestamp(now()), toTimestamp(now()), toTimestamp(now()))");
-        UntypedResultSet results = execute("SELECT * FROM %s WHERE a=0 AND b < toUnixTimestamp(now())");
-        assertEquals(1, results.size());
-
-        execute("INSERT INTO %s (a, b, c, d) VALUES (1, unixTimestampOf(now()), dateOf(now()), dateOf(now()))");
-        results = execute("SELECT * FROM %s WHERE a=1 AND b < toUnixTimestamp(now())");
-        assertEquals(1, results.size());
-    }
-
-    @Test
-    public void testReversedTypeCompatibility() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a int, b timeuuid, PRIMARY KEY (a, b)) WITH CLUSTERING ORDER BY (b DESC)");
-        execute("INSERT INTO %s (a, b) VALUES (0, now())");
-        UntypedResultSet results = execute("SELECT * FROM %s WHERE a=0 AND b < now()");
-        assertEquals(1, results.size());
-    }
-
-    @Test
-    // tests CASSANDRA-7797
-    public void testAlterReversedColumn() throws Throwable
-    {
-        createTable("CREATE TABLE IF NOT EXISTS %s (a int, b 'org.apache.cassandra.db.marshal.DateType', PRIMARY KEY (a, b)) WITH CLUSTERING ORDER BY (b DESC)");
-        alterTable("ALTER TABLE %s ALTER b TYPE 'org.apache.cassandra.db.marshal.ReversedType(org.apache.cassandra.db.marshal.TimestampType)'");
-    }
-
-    @Test
-    public void testIncompatibleReversedTypes() throws Throwable
-    {
-        createTable("CREATE TABLE IF NOT EXISTS %s (a int, b 'org.apache.cassandra.db.marshal.DateType', PRIMARY KEY (a, b)) WITH CLUSTERING ORDER BY (b DESC)");
-        try
-        {
-            alterTable("ALTER TABLE %s ALTER b TYPE 'org.apache.cassandra.db.marshal.ReversedType(org.apache.cassandra.db.marshal.TimeUUIDType)'");
-            fail("Expected error for ALTER statement");
-        }
-        catch (RuntimeException e) { }
-    }
-
-    @Test
-    public void testReversedAndNonReversed() throws Throwable
-    {
-        createTable("CREATE TABLE IF NOT EXISTS %s (a int, b 'org.apache.cassandra.db.marshal.DateType', PRIMARY KEY (a, b))");
-        try
-        {
-            alterTable("ALTER TABLE %s ALTER b TYPE 'org.apache.cassandra.db.marshal.ReversedType(org.apache.cassandra.db.marshal.DateType)'");
-            fail("Expected error for ALTER statement");
-        }
-        catch (RuntimeException e) { }
-    }
-}


[02/32] cassandra git commit: Migrate CQL tests from dtest to unit tests

Posted by jm...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/f797bfa4/test/unit/org/apache/cassandra/cql3/validation/operations/SelectLimitTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/SelectLimitTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/SelectLimitTest.java
new file mode 100644
index 0000000..730da84
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/SelectLimitTest.java
@@ -0,0 +1,111 @@
+package org.apache.cassandra.cql3.validation.operations;
+
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.dht.ByteOrderedPartitioner;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+
+public class SelectLimitTest extends CQLTester
+{
+    @BeforeClass
+    public static void setUp()
+    {
+        DatabaseDescriptor.setPartitioner(new ByteOrderedPartitioner());
+    }
+
+    /**
+     * Test limit across a partition range, requires byte ordered partitioner,
+     * migrated from cql_tests.py:TestCQL.limit_range_test()
+     */
+    @Test
+    public void testPartitionRange() throws Throwable
+    {
+        createTable("CREATE TABLE %s (userid int, url text, time bigint, PRIMARY KEY (userid, url)) WITH COMPACT STORAGE");
+
+        for (int i = 0; i < 100; i++)
+            for (String tld : new String[] { "com", "org", "net" })
+                execute("INSERT INTO %s (userid, url, time) VALUES (?, ?, ?)", i, String.format("http://foo.%s", tld), 42L);
+
+        assertRows(execute("SELECT * FROM %s WHERE token(userid) >= token(2) LIMIT 1"),
+                   row(2, "http://foo.com", 42L));
+
+        assertRows(execute("SELECT * FROM %s WHERE token(userid) > token(2) LIMIT 1"),
+                   row(3, "http://foo.com", 42L));
+    }
+
+    /**
+     * Test limit across a column range,
+     * migrated from cql_tests.py:TestCQL.limit_multiget_test()
+     */
+    @Test
+    public void testColumnRange() throws Throwable
+    {
+        createTable("CREATE TABLE %s (userid int, url text, time bigint, PRIMARY KEY (userid, url)) WITH COMPACT STORAGE");
+
+        for (int i = 0; i < 100; i++)
+            for (String tld : new String[] { "com", "org", "net" })
+                execute("INSERT INTO %s (userid, url, time) VALUES (?, ?, ?)", i, String.format("http://foo.%s", tld), 42L);
+
+        // Check that we do limit the output to 1 *and* that we respect query
+        // order of keys (even though 48 is after 2)
+        assertRows(execute("SELECT * FROM %s WHERE userid IN (48, 2) LIMIT 1"),
+                   row(48, "http://foo.com", 42L));
+    }
+
+    /**
+     * Test limit queries on a sparse table,
+     * migrated from cql_tests.py:TestCQL.limit_sparse_test()
+     */
+    @Test
+    public void testSparseTable() throws Throwable
+    {
+        createTable("CREATE TABLE %s (userid int, url text, day int, month text, year int, PRIMARY KEY (userid, url))");
+
+        for (int i = 0; i < 100; i++)
+            for (String tld : new String[] { "com", "org", "net" })
+                execute("INSERT INTO %s (userid, url, day, month, year) VALUES (?, ?, 1, 'jan', 2012)", i, String.format("http://foo.%s", tld));
+
+        assertRowCount(execute("SELECT * FROM %s LIMIT 4"), 4);
+
+    }
+
+    /**
+     * Check for #7052 bug,
+     * migrated from cql_tests.py:TestCQL.limit_compact_table()
+     */
+    @Test
+    public void testLimitInCompactTable() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int, v int, PRIMARY KEY (k, v) ) WITH COMPACT STORAGE ");
+
+        for (int i = 0; i < 4; i++)
+            for (int j = 0; j < 4; j++)
+                execute("INSERT INTO %s(k, v) VALUES (?, ?)", i, j);
+
+        assertRows(execute("SELECT v FROM %s WHERE k=0 AND v > 0 AND v <= 4 LIMIT 2"),
+                   row(1),
+                   row(2));
+        assertRows(execute("SELECT v FROM %s WHERE k=0 AND v > -1 AND v <= 4 LIMIT 2"),
+                   row(0),
+                   row(1));
+        assertRows(execute("SELECT * FROM %s WHERE k IN (0, 1, 2) AND v > 0 AND v <= 4 LIMIT 2"),
+                   row(0, 1),
+                   row(0, 2));
+        assertRows(execute("SELECT * FROM %s WHERE k IN (0, 1, 2) AND v > -1 AND v <= 4 LIMIT 2"),
+                   row(0, 0),
+                   row(0, 1));
+        assertRows(execute("SELECT * FROM %s WHERE k IN (0, 1, 2) AND v > 0 AND v <= 4 LIMIT 6"),
+                   row(0, 1),
+                   row(0, 2),
+                   row(0, 3),
+                   row(1, 1),
+                   row(1, 2),
+                   row(1, 3));
+
+        // strict bound (v > 1) over a range of partitions is not supported for compact storage if limit is provided
+        assertInvalidThrow(InvalidRequestException.class, "SELECT * FROM %s WHERE v > 1 AND v <= 3 LIMIT 6 ALLOW FILTERING");
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f797bfa4/test/unit/org/apache/cassandra/cql3/validation/operations/SelectMultiColumnRelationTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/SelectMultiColumnRelationTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/SelectMultiColumnRelationTest.java
new file mode 100644
index 0000000..b56ab8c
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/SelectMultiColumnRelationTest.java
@@ -0,0 +1,982 @@
+/*
+ * 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.validation.operations;
+
+import org.junit.Ignore;
+import org.junit.Test;
+
+import org.apache.cassandra.cql3.CQLTester;
+
+import static org.junit.Assert.assertEquals;
+
+public class SelectMultiColumnRelationTest extends CQLTester
+{
+    @Test
+    public void testSingleClusteringInvalidQueries() throws Throwable
+    {
+        for (String compactOption : new String[]{"", " WITH COMPACT STORAGE"})
+        {
+            createTable("CREATE TABLE %s (a int, b int, c int, PRIMARY KEY (a, b))" + compactOption);
+
+            assertInvalidSyntax("SELECT * FROM %s WHERE () = (?, ?)", 1, 2);
+            assertInvalidMessage("Column \"b\" cannot be restricted by an equality relation and an inequality relation",
+                                 "SELECT * FROM %s WHERE a = 0 AND (b) = (?) AND (b) > (?)", 0, 0);
+            assertInvalidMessage("More than one restriction was found for the start bound on b",
+                                 "SELECT * FROM %s WHERE a = 0 AND (b) > (?) AND (b) > (?)", 0, 1);
+            assertInvalidMessage("Multi-column relations can only be applied to clustering columns: a",
+                                 "SELECT * FROM %s WHERE (a, b) = (?, ?)", 0, 0);
+        }
+    }
+
+    @Test
+    public void testMultiClusteringInvalidQueries() throws Throwable
+    {
+        for (String compactOption : new String[]{"", " WITH COMPACT STORAGE"})
+        {
+            createTable("CREATE TABLE %s (a int, b int, c int, d int, PRIMARY KEY (a, b, c, d))" + compactOption);
+
+            assertInvalidSyntax("SELECT * FROM %s WHERE a = 0 AND (b, c) > ()");
+            assertInvalidMessage("Expected 2 elements in value tuple, but got 3: (?, ?, ?)",
+                                 "SELECT * FROM %s WHERE a = 0 AND (b, c) > (?, ?, ?)", 1, 2, 3);
+            assertInvalidMessage("Invalid null value in condition for column c",
+                                 "SELECT * FROM %s WHERE a = 0 AND (b, c) > (?, ?)", 1, null);
+
+            // Wrong order of columns
+            assertInvalidMessage("Clustering columns must appear in the PRIMARY KEY order in multi-column relations: (d, c, b) = (?, ?, ?)",
+                                 "SELECT * FROM %s WHERE a = 0 AND (d, c, b) = (?, ?, ?)", 0, 0, 0);
+            assertInvalidMessage("Clustering columns must appear in the PRIMARY KEY order in multi-column relations: (d, c, b) > (?, ?, ?)",
+                                 "SELECT * FROM %s WHERE a = 0 AND (d, c, b) > (?, ?, ?)", 0, 0, 0);
+
+            // Wrong number of values
+            assertInvalidMessage("Expected 3 elements in value tuple, but got 2: (?, ?)",
+                                 "SELECT * FROM %s WHERE a=0 AND (b, c, d) IN ((?, ?))", 0, 1);
+            assertInvalidMessage("Expected 3 elements in value tuple, but got 5: (?, ?, ?, ?, ?)",
+                                 "SELECT * FROM %s WHERE a=0 AND (b, c, d) IN ((?, ?, ?, ?, ?))", 0, 1, 2, 3, 4);
+
+            // Missing first clustering column
+            assertInvalidMessage("PRIMARY KEY column \"c\" cannot be restricted (preceding column \"b\" is not restricted)",
+                                 "SELECT * FROM %s WHERE a = 0 AND (c, d) = (?, ?)", 0, 0);
+            assertInvalidMessage("PRIMARY KEY column \"c\" cannot be restricted (preceding column \"b\" is not restricted)",
+                                 "SELECT * FROM %s WHERE a = 0 AND (c, d) > (?, ?)", 0, 0);
+
+            // Nulls
+            assertInvalidMessage("Invalid null value in condition for column d",
+                                 "SELECT * FROM %s WHERE a = 0 AND (b, c, d) IN ((?, ?, ?))", 1, 2, null);
+
+            // Wrong type for 'd'
+            assertInvalidMessage("Expected 4 or 0 byte int (6)",
+                                 "SELECT * FROM %s WHERE a = 0 AND (b, c, d) = (?, ?, ?)", 1, 2, "foobar");
+
+            assertInvalidMessage("Invalid tuple type literal for b of type int",
+                                 "SELECT * FROM %s WHERE a = 0 AND b = (?, ?, ?)", 1, 2, 3);
+
+            // Mix single and tuple inequalities
+            assertInvalidMessage("Column \"b\" cannot be restricted by both a tuple notation inequality and a single column inequality (b < ?)",
+                                 "SELECT * FROM %s WHERE a = 0 AND (b, c, d) > (?, ?, ?) AND b < ?", 0, 1, 0, 1);
+            assertInvalidMessage("Column \"c\" cannot be restricted by both a tuple notation inequality and a single column inequality (c < ?)",
+                                 "SELECT * FROM %s WHERE a = 0 AND (b, c, d) > (?, ?, ?) AND c < ?", 0, 1, 0, 1);
+            assertInvalidMessage("Column \"b\" cannot have both tuple-notation inequalities and single-column inequalities: (b, c, d) < (?, ?, ?)",
+                                 "SELECT * FROM %s WHERE a = 0 AND b > ? AND (b, c, d) < (?, ?, ?)", 1, 1, 1, 0);
+            assertInvalidMessage("Column \"c\" cannot have both tuple-notation inequalities and single-column inequalities: (b, c, d) < (?, ?, ?)",
+                                 "SELECT * FROM %s WHERE a = 0 AND c > ? AND (b, c, d) < (?, ?, ?)", 1, 1, 1, 0);
+
+            assertInvalidMessage("Multi-column relations can only be applied to clustering columns: a",
+                                 "SELECT * FROM %s WHERE (a, b, c, d) IN ((?, ?, ?, ?))", 0, 1, 2, 3);
+            assertInvalidMessage("PRIMARY KEY column \"c\" cannot be restricted (preceding column \"b\" is not restricted)",
+                                 "SELECT * FROM %s WHERE (c, d) IN ((?, ?))", 0, 1);
+            assertInvalidMessage("PRIMARY KEY column \"c\" cannot be restricted (preceding column \"b\" is restricted by a non-EQ relation)",
+                                 "SELECT * FROM %s WHERE a = ? AND b > ?  AND (c, d) IN ((?, ?))", 0, 0, 0, 0);
+
+            assertInvalidMessage("PRIMARY KEY column \"c\" cannot be restricted (preceding column \"b\" is restricted by a non-EQ relation)",
+                                 "SELECT * FROM %s WHERE a = ? AND b > ?  AND (c, d) > (?, ?)", 0, 0, 0, 0);
+            assertInvalidMessage("PRIMARY KEY column \"c\" cannot be restricted (preceding column \"b\" is restricted by a non-EQ relation)",
+                                 "SELECT * FROM %s WHERE a = ? AND (c, d) > (?, ?) AND b > ?  ", 0, 0, 0, 0);
+            assertInvalidMessage("Column \"c\" cannot be restricted by two tuple-notation inequalities not starting with the same column: (c) < (?)",
+                                 "SELECT * FROM %s WHERE a = ? AND (b, c) > (?, ?) AND (b) < (?) AND (c) < (?)", 0, 0, 0, 0, 0);
+            assertInvalidMessage("Column \"c\" cannot be restricted by two tuple-notation inequalities not starting with the same column: (b, c) > (?, ?)",
+                                 "SELECT * FROM %s WHERE a = ? AND (c) < (?) AND (b, c) > (?, ?) AND (b) < (?)", 0, 0, 0, 0, 0);
+            assertInvalidMessage("Column \"c\" cannot be restricted by two tuple-notation inequalities not starting with the same column: (b, c) > (?, ?)",
+                                 "SELECT * FROM %s WHERE a = ? AND (b) < (?) AND (c) < (?) AND (b, c) > (?, ?)", 0, 0, 0, 0, 0);
+
+            assertInvalidMessage("Column \"c\" cannot be restricted by two tuple-notation inequalities not starting with the same column: (c) < (?)",
+                                 "SELECT * FROM %s WHERE a = ? AND (b, c) > (?, ?) AND (c) < (?)", 0, 0, 0, 0);
+
+            assertInvalidMessage("PRIMARY KEY column \"d\" cannot be restricted (preceding column \"c\" is restricted by an IN tuple notation)",
+                                 "SELECT * FROM %s WHERE a = ? AND (b, c) in ((?, ?), (?, ?)) AND d > ?", 0, 0, 0, 0, 0, 0);
+        }
+    }
+
+    @Test
+    public void testMultiAndSingleColumnRelationMix() throws Throwable
+    {
+        for (String compactOption : new String[]{"", " WITH COMPACT STORAGE"})
+        {
+            createTable("CREATE TABLE %s (a int, b int, c int, d int, PRIMARY KEY (a, b, c, d))" + compactOption);
+
+            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 0, 0);
+            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 1, 0);
+            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 1, 1);
+
+            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 0, 0);
+            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 1, 0);
+            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 1, 1);
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? and b = ? and (c, d) = (?, ?)", 0, 1, 0, 0),
+                       row(0, 1, 0, 0));
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? and b = ? and (c) IN ((?))", 0, 1, 0),
+                       row(0, 1, 0, 0));
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? and b = ? and (c) IN ((?), (?))", 0, 1, 0, 1),
+                       row(0, 1, 0, 0),
+                       row(0, 1, 1, 0),
+                       row(0, 1, 1, 1));
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? and b = ? and (c, d) IN ((?, ?))", 0, 1, 0, 0),
+                       row(0, 1, 0, 0));
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? and b = ? and (c, d) IN ((?, ?), (?, ?))", 0, 1, 0, 0, 1, 1),
+                       row(0, 1, 0, 0),
+                       row(0, 1, 1, 1));
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? and b = ? and (c, d) > (?, ?)", 0, 1, 0, 0),
+                       row(0, 1, 1, 0),
+                       row(0, 1, 1, 1));
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? and b = ? and (c, d) > (?, ?) and (c) <= (?) ", 0, 1, 0, 0, 1),
+                       row(0, 1, 1, 0),
+                       row(0, 1, 1, 1));
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? and b = ? and (c, d) >= (?, ?) and (c, d) < (?, ?)", 0, 1, 0, 0, 1, 1),
+                       row(0, 1, 0, 0),
+                       row(0, 1, 1, 0));
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b, c) = (?, ?) and d = ?", 0, 0, 1, 0),
+                       row(0, 0, 1, 0));
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? and b = ? and (c) = (?) and d = ?", 0, 0, 1, 0),
+                       row(0, 0, 1, 0));
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b, c) = (?, ?) and d IN (?, ?)", 0, 0, 1, 0, 2),
+                       row(0, 0, 1, 0));
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? and b = ? and (c) = (?) and d IN (?, ?)", 0, 0, 1, 0, 2),
+                       row(0, 0, 1, 0));
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b, c) = (?, ?) and d >= ?", 0, 0, 1, 0),
+                       row(0, 0, 1, 0),
+                       row(0, 0, 1, 1));
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? and d < 1 and (b, c) = (?, ?) and d >= ?", 0, 0, 1, 0),
+                       row(0, 0, 1, 0));
+        }
+    }
+
+    @Test
+    public void testMultipleMultiColumnRelation() throws Throwable
+    {
+        for (String compactOption : new String[]{"", " WITH COMPACT STORAGE"})
+        {
+            createTable("CREATE TABLE %s (a int, b int, c int, d int, PRIMARY KEY (a, b, c, d))" + compactOption);
+
+            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 0, 0);
+            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 1, 0);
+            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 1, 1);
+
+            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 0, 0);
+            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 1, 0);
+            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 1, 1);
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b) = (?) and (c, d) = (?, ?)", 0, 1, 0, 0),
+                       row(0, 1, 0, 0));
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b) = (?) and (c) = (?) and (d) = (?)", 0, 1, 0, 0),
+                       row(0, 1, 0, 0));
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b) = (?) and (c) IN ((?))", 0, 1, 0),
+                       row(0, 1, 0, 0));
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b) = (?) and (c) IN ((?), (?))", 0, 1, 0, 1),
+                       row(0, 1, 0, 0),
+                       row(0, 1, 1, 0),
+                       row(0, 1, 1, 1));
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b) = (?) and (c, d) IN ((?, ?))", 0, 1, 0, 0),
+                       row(0, 1, 0, 0));
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b) = (?) and (c, d) IN ((?, ?), (?, ?))", 0, 1, 0, 0, 1, 1),
+                       row(0, 1, 0, 0),
+                       row(0, 1, 1, 1));
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b) = (?) and (c, d) > (?, ?)", 0, 1, 0, 0),
+                       row(0, 1, 1, 0),
+                       row(0, 1, 1, 1));
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b) = (?) and (c, d) > (?, ?) and (c) <= (?) ", 0, 1, 0, 0, 1),
+                       row(0, 1, 1, 0),
+                       row(0, 1, 1, 1));
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b) = (?) and (c, d) >= (?, ?) and (c, d) < (?, ?)", 0, 1, 0, 0, 1, 1),
+                       row(0, 1, 0, 0),
+                       row(0, 1, 1, 0));
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b, c) = (?, ?) and (d) = (?)", 0, 0, 1, 0),
+                       row(0, 0, 1, 0));
+        }
+    }
+
+    @Test
+    public void testSinglePartitionInvalidQueries() throws Throwable
+    {
+        for (String compactOption : new String[]{"", " WITH COMPACT STORAGE"})
+        {
+            createTable("CREATE TABLE %s (a int PRIMARY KEY, b int)" + compactOption);
+
+            assertInvalid("SELECT * FROM %s WHERE (a) > (?)", 0);
+            assertInvalid("SELECT * FROM %s WHERE (a) = (?)", 0);
+            assertInvalid("SELECT * FROM %s WHERE (b) = (?)", 0);
+        }
+    }
+
+    @Test
+    public void testSingleClustering() throws Throwable
+    {
+        for (String compactOption : new String[]{"", " WITH COMPACT STORAGE"})
+        {
+            createTable("CREATE TABLE %s (a int, b int, c int, PRIMARY KEY (a, b))" + compactOption);
+
+            execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, 0, 0);
+            execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, 1, 0);
+            execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, 2, 0);
+
+            // Equalities
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) = (?)", 0, 1),
+                    row(0, 1, 0)
+            );
+
+            // Same but check the whole tuple can be prepared
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) = ?", 0, tuple(1)),
+                    row(0, 1, 0)
+            );
+
+            assertEmpty(execute("SELECT * FROM %s WHERE a = ? AND (b) = (?)", 0, 3));
+
+            // Inequalities
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) > (?)", 0, 0),
+                    row(0, 1, 0),
+                    row(0, 2, 0)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) >= (?)", 0, 1),
+                    row(0, 1, 0),
+                    row(0, 2, 0)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) < (?)", 0, 2),
+                    row(0, 0, 0),
+                    row(0, 1, 0)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) <= (?)", 0, 1),
+                    row(0, 0, 0),
+                    row(0, 1, 0)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) > (?) AND (b) < (?)", 0, 0, 2),
+                    row(0, 1, 0)
+            );
+        }
+    }
+
+    @Test
+    public void testNonEqualsRelation() throws Throwable
+    {
+        for (String compactOption : new String[]{"", " WITH COMPACT STORAGE"})
+        {
+            createTable("CREATE TABLE %s (a int PRIMARY KEY, b int)" + compactOption);
+            assertInvalid("SELECT * FROM %s WHERE a = 0 AND (b) != (0)");
+        }
+    }
+
+    @Test
+    public void testMultipleClustering() throws Throwable
+    {
+        for (String compactOption : new String[]{"", " WITH COMPACT STORAGE"})
+        {
+            createTable("CREATE TABLE %s (a int, b int, c int, d int, PRIMARY KEY (a, b, c, d))" + compactOption);
+
+            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 0, 0);
+            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 1, 0);
+            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 1, 1);
+
+            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 0, 0);
+            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 1, 0);
+            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 1, 1);
+
+            // Empty query
+            assertEmpty(execute("SELECT * FROM %s WHERE a = 0 AND (b, c, d) IN ()"));
+
+            // Equalities
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) = (?)", 0, 1),
+                    row(0, 1, 0, 0),
+                    row(0, 1, 1, 0),
+                    row(0, 1, 1, 1)
+            );
+
+            // Same with whole tuple prepared
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) = ?", 0, tuple(1)),
+                    row(0, 1, 0, 0),
+                    row(0, 1, 1, 0),
+                    row(0, 1, 1, 1)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c) = (?, ?)", 0, 1, 1),
+                    row(0, 1, 1, 0),
+                    row(0, 1, 1, 1)
+            );
+
+            // Same with whole tuple prepared
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c) = ?", 0, tuple(1, 1)),
+                    row(0, 1, 1, 0),
+                    row(0, 1, 1, 1)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) = (?, ?, ?)", 0, 1, 1, 1),
+                    row(0, 1, 1, 1)
+            );
+
+            // Same with whole tuple prepared
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) = ?", 0, tuple(1, 1, 1)),
+                    row(0, 1, 1, 1)
+            );
+
+            // Inequalities
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) > (?)", 0, 0),
+                    row(0, 1, 0, 0),
+                    row(0, 1, 1, 0),
+                    row(0, 1, 1, 1)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) >= (?)", 0, 0),
+                    row(0, 0, 0, 0),
+                    row(0, 0, 1, 0),
+                    row(0, 0, 1, 1),
+                    row(0, 1, 0, 0),
+                    row(0, 1, 1, 0),
+                    row(0, 1, 1, 1)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c) > (?, ?)", 0, 1, 0),
+                    row(0, 1, 1, 0),
+                    row(0, 1, 1, 1)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c) >= (?, ?)", 0, 1, 0),
+                    row(0, 1, 0, 0),
+                    row(0, 1, 1, 0),
+                    row(0, 1, 1, 1)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) > (?, ?, ?)", 0, 1, 1, 0),
+                    row(0, 1, 1, 1)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) >= (?, ?, ?)", 0, 1, 1, 0),
+                    row(0, 1, 1, 0),
+                    row(0, 1, 1, 1)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) < (?)", 0, 1),
+                    row(0, 0, 0, 0),
+                    row(0, 0, 1, 0),
+                    row(0, 0, 1, 1)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) <= (?)", 0, 1),
+                    row(0, 0, 0, 0),
+                    row(0, 0, 1, 0),
+                    row(0, 0, 1, 1),
+                    row(0, 1, 0, 0),
+                    row(0, 1, 1, 0),
+                    row(0, 1, 1, 1)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c) < (?, ?)", 0, 0, 1),
+                    row(0, 0, 0, 0)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c) <= (?, ?)", 0, 0, 1),
+                    row(0, 0, 0, 0),
+                    row(0, 0, 1, 0),
+                    row(0, 0, 1, 1)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) < (?, ?, ?)", 0, 0, 1, 1),
+                    row(0, 0, 0, 0),
+                    row(0, 0, 1, 0)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) <= (?, ?, ?)", 0, 0, 1, 1),
+                    row(0, 0, 0, 0),
+                    row(0, 0, 1, 0),
+                    row(0, 0, 1, 1)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) > (?, ?, ?) AND (b) < (?)", 0, 0, 1, 0, 1),
+                    row(0, 0, 1, 1)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) > (?, ?, ?) AND (b, c) < (?, ?)", 0, 0, 1, 1, 1, 1),
+                    row(0, 1, 0, 0)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) > (?, ?, ?) AND (b, c, d) < (?, ?, ?)", 0, 0, 1, 1, 1, 1, 0),
+                    row(0, 1, 0, 0)
+            );
+
+            // Same with whole tuple prepared
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) > ? AND (b, c, d) < ?", 0, tuple(0, 1, 1), tuple(1, 1, 0)),
+                    row(0, 1, 0, 0)
+            );
+
+            // reversed
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) > (?) ORDER BY b DESC, c DESC, d DESC", 0, 0),
+                    row(0, 1, 1, 1),
+                    row(0, 1, 1, 0),
+                    row(0, 1, 0, 0)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) >= (?) ORDER BY b DESC, c DESC, d DESC", 0, 0),
+                    row(0, 1, 1, 1),
+                    row(0, 1, 1, 0),
+                    row(0, 1, 0, 0),
+                    row(0, 0, 1, 1),
+                    row(0, 0, 1, 0),
+                    row(0, 0, 0, 0)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c) > (?, ?) ORDER BY b DESC, c DESC, d DESC", 0, 1, 0),
+                    row(0, 1, 1, 1),
+                    row(0, 1, 1, 0)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c) >= (?, ?) ORDER BY b DESC, c DESC, d DESC", 0, 1, 0),
+                    row(0, 1, 1, 1),
+                    row(0, 1, 1, 0),
+                    row(0, 1, 0, 0)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) > (?, ?, ?) ORDER BY b DESC, c DESC, d DESC", 0, 1, 1, 0),
+                    row(0, 1, 1, 1)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) >= (?, ?, ?) ORDER BY b DESC, c DESC, d DESC", 0, 1, 1, 0),
+                    row(0, 1, 1, 1),
+                    row(0, 1, 1, 0)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) < (?) ORDER BY b DESC, c DESC, d DESC", 0, 1),
+                    row(0, 0, 1, 1),
+                    row(0, 0, 1, 0),
+                    row(0, 0, 0, 0)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) <= (?) ORDER BY b DESC, c DESC, d DESC", 0, 1),
+                    row(0, 1, 1, 1),
+                    row(0, 1, 1, 0),
+                    row(0, 1, 0, 0),
+                    row(0, 0, 1, 1),
+                    row(0, 0, 1, 0),
+                    row(0, 0, 0, 0)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c) < (?, ?) ORDER BY b DESC, c DESC, d DESC", 0, 0, 1),
+                    row(0, 0, 0, 0)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c) <= (?, ?) ORDER BY b DESC, c DESC, d DESC", 0, 0, 1),
+                    row(0, 0, 1, 1),
+                    row(0, 0, 1, 0),
+                    row(0, 0, 0, 0)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) < (?, ?, ?) ORDER BY b DESC, c DESC, d DESC", 0, 0, 1, 1),
+                    row(0, 0, 1, 0),
+                    row(0, 0, 0, 0)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) <= (?, ?, ?) ORDER BY b DESC, c DESC, d DESC", 0, 0, 1, 1),
+                    row(0, 0, 1, 1),
+                    row(0, 0, 1, 0),
+                    row(0, 0, 0, 0)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) > (?, ?, ?) AND (b) < (?) ORDER BY b DESC, c DESC, d DESC", 0, 0, 1, 0, 1),
+                    row(0, 0, 1, 1)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) > (?, ?, ?) AND (b, c) < (?, ?) ORDER BY b DESC, c DESC, d DESC", 0, 0, 1, 1, 1, 1),
+                    row(0, 1, 0, 0)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) > (?, ?, ?) AND (b, c, d) < (?, ?, ?) ORDER BY b DESC, c DESC, d DESC", 0, 0, 1, 1, 1, 1, 0),
+                    row(0, 1, 0, 0)
+            );
+
+            // IN
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) IN ((?, ?, ?), (?, ?, ?))", 0, 0, 1, 0, 0, 1, 1),
+                    row(0, 0, 1, 0),
+                    row(0, 0, 1, 1)
+            );
+
+            // same query but with whole tuple prepared
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) IN (?, ?)", 0, tuple(0, 1, 0), tuple(0, 1, 1)),
+                    row(0, 0, 1, 0),
+                    row(0, 0, 1, 1)
+            );
+
+            // same query but with whole IN list prepared
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) IN ?", 0, list(tuple(0, 1, 0), tuple(0, 1, 1))),
+                    row(0, 0, 1, 0),
+                    row(0, 0, 1, 1)
+            );
+
+            // same query, but reversed order for the IN values
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) IN (?, ?)", 0, tuple(0, 1, 1), tuple(0, 1, 0)),
+                    row(0, 0, 1, 0),
+                    row(0, 0, 1, 1)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b, c) IN ((?, ?))", 0, 0, 1),
+                    row(0, 0, 1, 0),
+                    row(0, 0, 1, 1)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b) IN ((?))", 0, 0),
+                    row(0, 0, 0, 0),
+                    row(0, 0, 1, 0),
+                    row(0, 0, 1, 1)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c) IN ((?, ?)) ORDER BY b DESC, c DESC, d DESC", 0, 0, 1),
+                    row(0, 0, 1, 1),
+                    row(0, 0, 1, 0)
+            );
+
+            // IN on both partition key and clustering key
+            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 1, 0, 0, 0);
+            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 1, 0, 1, 0);
+            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 1, 0, 1, 1);
+
+            assertRows(execute("SELECT * FROM %s WHERE a IN (?, ?) AND (b, c, d) IN (?, ?)", 0, 1, tuple(0, 1, 0), tuple(0, 1, 1)),
+                    row(0, 0, 1, 0),
+                    row(0, 0, 1, 1),
+                    row(1, 0, 1, 0),
+                    row(1, 0, 1, 1)
+            );
+
+            // same but with whole IN lists prepared
+            assertRows(execute("SELECT * FROM %s WHERE a IN ? AND (b, c, d) IN ?", list(0, 1), list(tuple(0, 1, 0), tuple(0, 1, 1))),
+                    row(0, 0, 1, 0),
+                    row(0, 0, 1, 1),
+                    row(1, 0, 1, 0),
+                    row(1, 0, 1, 1)
+            );
+
+            // same query, but reversed order for the IN values
+            assertRows(execute("SELECT * FROM %s WHERE a IN (?, ?) AND (b, c, d) IN (?, ?)", 1, 0, tuple(0, 1, 1), tuple(0, 1, 0)),
+                    row(1, 0, 1, 0),
+                    row(1, 0, 1, 1),
+                    row(0, 0, 1, 0),
+                    row(0, 0, 1, 1)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a IN (?, ?) and (b, c) IN ((?, ?))", 0, 1, 0, 1),
+                    row(0, 0, 1, 0),
+                    row(0, 0, 1, 1),
+                    row(1, 0, 1, 0),
+                    row(1, 0, 1, 1)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a IN (?, ?) and (b) IN ((?))", 0, 1, 0),
+                    row(0, 0, 0, 0),
+                    row(0, 0, 1, 0),
+                    row(0, 0, 1, 1),
+                    row(1, 0, 0, 0),
+                    row(1, 0, 1, 0),
+                    row(1, 0, 1, 1)
+            );
+        }
+    }
+
+    @Test
+    public void testMultipleClusteringReversedComponents() throws Throwable
+    {
+        for (String compactOption : new String[]{"", " COMPACT STORAGE AND"})
+        {
+            createTable("CREATE TABLE %s (a int, b int, c int, d int, PRIMARY KEY (a, b, c, d)) WITH" + compactOption + " CLUSTERING ORDER BY (b DESC, c ASC, d DESC)");
+
+            // b and d are reversed in the clustering order
+            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 0, 0);
+            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 1, 1);
+            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 1, 0);
+
+            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 0, 0);
+            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 1, 1);
+            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 1, 0);
+
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) > (?)", 0, 0),
+                    row(0, 1, 0, 0),
+                    row(0, 1, 1, 1),
+                    row(0, 1, 1, 0)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) >= (?)", 0, 0),
+                    row(0, 1, 0, 0),
+                    row(0, 1, 1, 1),
+                    row(0, 1, 1, 0),
+                    row(0, 0, 0, 0),
+                    row(0, 0, 1, 1),
+                    row(0, 0, 1, 0)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) < (?)", 0, 1),
+                    row(0, 0, 0, 0),
+                    row(0, 0, 1, 1),
+                    row(0, 0, 1, 0)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) <= (?)", 0, 1),
+                    row(0, 1, 0, 0),
+                    row(0, 1, 1, 1),
+                    row(0, 1, 1, 0),
+                    row(0, 0, 0, 0),
+                    row(0, 0, 1, 1),
+                    row(0, 0, 1, 0)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a=? AND (b, c, d) IN ((?, ?, ?), (?, ?, ?))", 0, 1, 1, 1, 0, 1, 1),
+                    row(0, 1, 1, 1),
+                    row(0, 0, 1, 1)
+            );
+
+            // same query, but reversed order for the IN values
+            assertRows(execute("SELECT * FROM %s WHERE a=? AND (b, c, d) IN ((?, ?, ?), (?, ?, ?))", 0, 0, 1, 1, 1, 1, 1),
+                    row(0, 1, 1, 1),
+                    row(0, 0, 1, 1)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) IN (?, ?, ?, ?, ?, ?)",
+                            0, tuple(1, 0, 0), tuple(1, 1, 1), tuple(1, 1, 0), tuple(0, 0, 0), tuple(0, 1, 1), tuple(0, 1, 0)),
+                    row(0, 1, 0, 0),
+                    row(0, 1, 1, 1),
+                    row(0, 1, 1, 0),
+                    row(0, 0, 0, 0),
+                    row(0, 0, 1, 1),
+                    row(0, 0, 1, 0)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c) IN (?)", 0, tuple(0, 1)),
+                    row(0, 0, 1, 1),
+                    row(0, 0, 1, 0)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c) IN (?)", 0, tuple(0, 0)),
+                    row(0, 0, 0, 0)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) IN ((?))", 0, 0),
+                    row(0, 0, 0, 0),
+                    row(0, 0, 1, 1),
+                    row(0, 0, 1, 0)
+            );
+
+            // preserve pre-6875 behavior (even though the query result is technically incorrect)
+            assertEmpty(execute("SELECT * FROM %s WHERE a = ? AND (b, c) > (?, ?)", 0, 1, 0));
+        }
+    }
+
+    @Test
+    public void testMultipleClusteringWithIndex() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int, b int, c int, d int, e int, PRIMARY KEY (a, b, c, d))");
+        createIndex("CREATE INDEX ON %s (b)");
+        createIndex("CREATE INDEX ON %s (e)");
+        execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 0, 0, 0, 0);
+        execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 0, 1, 0, 1);
+        execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 0, 1, 1, 2);
+        execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 1, 0, 0, 0);
+        execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 1, 1, 0, 1);
+        execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 1, 1, 1, 2);
+        execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 2, 0, 0, 0);
+        assertRows(execute("SELECT * FROM %s WHERE (b) = (?)", 1),
+                   row(0, 1, 0, 0, 0),
+                   row(0, 1, 1, 0, 1),
+                   row(0, 1, 1, 1, 2));
+        assertRows(execute("SELECT * FROM %s WHERE (b, c) = (?, ?) ALLOW FILTERING", 1, 1),
+                   row(0, 1, 1, 0, 1),
+                   row(0, 1, 1, 1, 2));
+        assertRows(execute("SELECT * FROM %s WHERE (b, c) = (?, ?) AND e = ? ALLOW FILTERING", 1, 1, 2),
+                   row(0, 1, 1, 1, 2));
+        assertRows(execute("SELECT * FROM %s WHERE (b) IN ((?)) AND e = ?", 1, 2),
+                   row(0, 1, 1, 1, 2));
+
+        assertRows(execute("SELECT * FROM %s WHERE (b) IN ((?), (?)) AND e = ?", 0, 1, 2),
+                   row(0, 0, 1, 1, 2),
+                   row(0, 1, 1, 1, 2));
+
+        assertRows(execute("SELECT * FROM %s WHERE (b, c) IN ((?, ?)) AND e = ?", 0, 1, 2),
+                   row(0, 0, 1, 1, 2));
+
+        assertRows(execute("SELECT * FROM %s WHERE (b, c) IN ((?, ?), (?, ?)) AND e = ?", 0, 1, 1, 1, 2),
+                   row(0, 0, 1, 1, 2),
+                   row(0, 1, 1, 1, 2));
+
+        assertRows(execute("SELECT * FROM %s WHERE (b) >= (?) AND e = ?", 1, 2),
+                   row(0, 1, 1, 1, 2));
+
+        assertRows(execute("SELECT * FROM %s WHERE (b, c) >= (?, ?) AND e = ?", 1, 1, 2),
+                   row(0, 1, 1, 1, 2));
+    }
+
+    @Test
+    public void testMultiplePartitionKeyAndMultiClusteringWithIndex() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int, b int, c int, d int, e int, f int, PRIMARY KEY ((a, b), c, d, e))");
+        createIndex("CREATE INDEX ON %s (c)");
+        createIndex("CREATE INDEX ON %s (f)");
+
+        execute("INSERT INTO %s (a, b, c, d, e, f) VALUES (?, ?, ?, ?, ?, ?)", 0, 0, 0, 0, 0, 0);
+        execute("INSERT INTO %s (a, b, c, d, e, f) VALUES (?, ?, ?, ?, ?, ?)", 0, 0, 0, 1, 0, 1);
+        execute("INSERT INTO %s (a, b, c, d, e, f) VALUES (?, ?, ?, ?, ?, ?)", 0, 0, 0, 1, 1, 2);
+
+        execute("INSERT INTO %s (a, b, c, d, e, f) VALUES (?, ?, ?, ?, ?, ?)", 0, 0, 1, 0, 0, 3);
+        execute("INSERT INTO %s (a, b, c, d, e, f) VALUES (?, ?, ?, ?, ?, ?)", 0, 0, 1, 1, 0, 4);
+        execute("INSERT INTO %s (a, b, c, d, e, f) VALUES (?, ?, ?, ?, ?, ?)", 0, 0, 1, 1, 1, 5);
+
+        execute("INSERT INTO %s (a, b, c, d, e, f) VALUES (?, ?, ?, ?, ?, ?)", 0, 0, 2, 0, 0, 5);
+
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND (c) = (?) ALLOW FILTERING", 0, 1),
+                   row(0, 0, 1, 0, 0, 3),
+                   row(0, 0, 1, 1, 0, 4),
+                   row(0, 0, 1, 1, 1, 5));
+
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND (c, d) = (?, ?) ALLOW FILTERING", 0, 1, 1),
+                   row(0, 0, 1, 1, 0, 4),
+                   row(0, 0, 1, 1, 1, 5));
+
+        assertInvalidMessage("Partition key part b must be restricted since preceding part is",
+                             "SELECT * FROM %s WHERE a = ? AND (c, d) IN ((?, ?)) ALLOW FILTERING", 0, 1, 1);
+
+        assertInvalidMessage("Partition key part b must be restricted since preceding part is",
+                             "SELECT * FROM %s WHERE a = ? AND (c, d) >= (?, ?) ALLOW FILTERING", 0, 1, 1);
+
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND (c) IN ((?)) AND f = ? ALLOW FILTERING", 0, 1, 5),
+                   row(0, 0, 1, 1, 1, 5));
+
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND (c) IN ((?), (?)) AND f = ? ALLOW FILTERING", 0, 1, 2, 5),
+                   row(0, 0, 1, 1, 1, 5),
+                   row(0, 0, 2, 0, 0, 5));
+
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND (c, d) IN ((?, ?)) AND f = ? ALLOW FILTERING", 0, 1, 0, 3),
+                   row(0, 0, 1, 0, 0, 3));
+
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND (c) >= (?) AND f = ? ALLOW FILTERING", 0, 1, 5),
+                   row(0, 0, 1, 1, 1, 5),
+                   row(0, 0, 2, 0, 0, 5));
+
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND (c, d) >= (?, ?) AND f = ? ALLOW FILTERING", 0, 1, 1, 5),
+                   row(0, 0, 1, 1, 1, 5),
+                   row(0, 0, 2, 0, 0, 5));
+    }
+
+    /**
+     * Check select on tuple relations with mixed ASC | DESC clustering, see CASSANDRA-7281
+     * migrated from cql_tests.py:TestCQL.tuple_query_mixed_order_columns_test to tuple_query_mixed_order_columns_test9
+     */
+    @Ignore // CASSANDRA-7281 not yet delivered
+    public void testMixedOrderClustering1() throws Throwable
+    {
+        createTableForMixedOrderClusteringTest("DESC", "ASC", "DESC", "ASC");
+
+        assertRows(execute("SELECT * FROM %s WHERE a=0 AND (b, c, d, e) > (0, 1, 1, 0)"),
+                   row(0, 2, 0, 0, 0),
+                   row(0, 1, 0, 0, 0),
+                   row(0, 0, 1, 2, -1),
+                   row(0, 0, 1, 1, 1),
+                   row(0, 0, 2, 1, -3),
+                   row(0, 0, 2, 0, 3));
+    }
+
+    @Ignore // CASSANDRA-7281 not yet delivered
+    public void testMixedOrderClustering2() throws Throwable
+    {
+        createTableForMixedOrderClusteringTest("DESC", "DESC", "DESC", "ASC");
+
+        assertRows(execute("SELECT * FROM %s WHERE a=0 AND (b, c, d, e) > (0, 1, 1, 0)"),
+                   row(0, 2, 0, 0, 0),
+                   row(0, 1, 0, 0, 0),
+                   row(0, 0, 2, 1, -3),
+                   row(0, 0, 2, 0, 3),
+                   row(0, 0, 1, 2, -1),
+                   row(0, 0, 1, 1, 1));
+    }
+
+    @Ignore // CASSANDRA-7281 not yet delivered
+    public void testMixedOrderClustering3() throws Throwable
+    {
+        createTableForMixedOrderClusteringTest("ASC", "DESC", "DESC", "ASC");
+
+        assertRows(execute("SELECT * FROM %s WHERE a=0 AND (b, c, d, e) > (0, 1, 1, 0)"),
+                   row(0, 0, 2, 1, -3),
+                   row(0, 0, 2, 0, 3),
+                   row(0, 0, 1, 2, -1),
+                   row(0, 0, 1, 1, 1),
+                   row(0, 1, 0, 0, 0),
+                   row(0, 2, 0, 0, 0));
+    }
+
+    @Ignore // CASSANDRA-7281 not yet delivered
+    public void testMixedOrderClustering4() throws Throwable
+    {
+        createTableForMixedOrderClusteringTest("DESC", "ASC", "ASC", "DESC");
+
+        assertRows(execute("SELECT * FROM %s WHERE a=0 AND (b, c, d, e) > (0, 1, 1, 0)"),
+                   row(0, 2, 0, 0, 0),
+                   row(0, 1, 0, 0, 0),
+                   row(0, 0, 1, 1, 1),
+                   row(0, 0, 1, 2, -1),
+                   row(0, 0, 2, 0, 3),
+                   row(0, 0, 2, 1, -3));
+    }
+
+    @Ignore // CASSANDRA-7281 not yet delivered
+    public void testMixedOrderClustering5() throws Throwable
+    {
+        createTableForMixedOrderClusteringTest("DESC", "DESC", "DESC", "DESC");
+
+        assertRows(execute("SELECT * FROM %s WHERE a=0 AND (b, c, d, e) > (0, 1, 1, 0)"),
+                   row(0, 2, 0, 0, 0),
+                   row(0, 1, 0, 0, 0),
+                   row(0, 0, 2, 1, -3),
+                   row(0, 0, 2, 0, 3),
+                   row(0, 0, 1, 2, -1),
+                   row(0, 0, 1, 1, 1));
+    }
+
+    @Ignore // CASSANDRA-7281 not yet delivered
+    public void testMixedOrderClustering6() throws Throwable
+    {
+        createTableForMixedOrderClusteringTest("ASC", "ASC", "ASC", "ASC");
+
+        assertRows(execute("SELECT * FROM %s WHERE a=0 AND (b, c, d, e) > (0, 1, 1, 0)"),
+                   row(0, 0, 1, 1, 1),
+                   row(0, 0, 1, 2, -1),
+                   row(0, 0, 2, 0, 3),
+                   row(0, 0, 2, 1, -3),
+                   row(0, 1, 0, 0, 0),
+                   row(0, 2, 0, 0, 0));
+    }
+
+    @Ignore // CASSANDRA-7281 not yet delivered
+    public void testMixedOrderClustering7() throws Throwable
+    {
+        createTableForMixedOrderClusteringTest("DESC", "ASC", "DESC", "ASC");
+
+        assertRows(execute("SELECT * FROM %s WHERE a=0 AND (b, c, d, e) <= (0, 1, 1, 0)"),
+                   row(0, 0, 0, 0, 0),
+                   row(0, 0, 1, 1, -1),
+                   row(0, 0, 1, 1, 0),
+                   row(0, 0, 1, 0, 2),
+                   row(0, -1, 2, 2, 2));
+    }
+
+    @Ignore // CASSANDRA-7281 not yet delivered
+    public void testMixedOrderClustering8() throws Throwable
+    {
+        createTableForMixedOrderClusteringTest("ASC", "DESC", "DESC", "ASC");
+
+        assertRows(execute("SELECT * FROM %s WHERE a=0 AND (b, c, d, e) <= (0, 1, 1, 0)"),
+                   row(0, -1, 2, 2, 2),
+                   row(0, 0, 1, 1, -1),
+                   row(0, 0, 1, 1, 0),
+                   row(0, 0, 1, 0, 2),
+                   row(0, 0, 0, 0, 0));
+    }
+
+    @Ignore // CASSANDRA-7281 not yet delivered
+    public void testMixedOrderClustering9() throws Throwable
+    {
+        createTableForMixedOrderClusteringTest("DESC", "ASC", "DESC", "DESC");
+
+        assertRows(execute("SELECT * FROM %s WHERE a=0 AND (b, c, d, e) <= (0, 1, 1, 0)"),
+                   row(0, 0, 0, 0, 0),
+                   row(0, 0, 1, 1, 0),
+                   row(0, 0, 1, 1, -1),
+                   row(0, 0, 1, 0, 2),
+                   row(0, -1, 2, 2, 2));
+    }
+
+    private void createTableForMixedOrderClusteringTest(String ... formats) throws Throwable
+    {
+        assertEquals(4, formats.length);
+
+        String clustering = String.format("WITH CLUSTERING ORDER BY (b %s, c %s, d %s, e %s)", (Object[])formats);
+        createTable("CREATE TABLE %s (a int, b int, c int, d int , e int, PRIMARY KEY (a, b, c, d, e) ) " + clustering);
+
+        execute("INSERT INTO %s (a, b, c, d, e) VALUES (0, 2, 0, 0, 0)");
+        execute("INSERT INTO %s (a, b, c, d, e) VALUES (0, 1, 0, 0, 0)");
+        execute("INSERT INTO %s (a, b, c, d, e) VALUES (0, 0, 0, 0, 0)");
+        execute("INSERT INTO %s (a, b, c, d, e) VALUES (0, 0, 1, 2, -1)");
+        execute("INSERT INTO %s (a, b, c, d, e) VALUES (0, 0, 1, 1, -1)");
+        execute("INSERT INTO %s (a, b, c, d, e) VALUES (0, 0, 1, 1, 0)");
+        execute("INSERT INTO %s (a, b, c, d, e) VALUES (0, 0, 1, 1, 1)");
+        execute("INSERT INTO %s (a, b, c, d, e) VALUES (0, 0, 1, 0, 2)");
+        execute("INSERT INTO %s (a, b, c, d, e) VALUES (0, 0, 2, 1, -3)");
+        execute("INSERT INTO %s (a, b, c, d, e) VALUES (0, 0, 2, 0, 3)");
+        execute("INSERT INTO %s (a, b, c, d, e) VALUES (0, -1, 2, 2, 2)");
+    }
+
+    /**
+     * Check select on tuple relations, see CASSANDRA-8613
+     * migrated from cql_tests.py:TestCQL.simple_tuple_query_test()
+     */
+    @Test
+    public void testSimpleTupleQuery() throws Throwable
+    {
+        createTable("create table %s (a int, b int, c int, d int , e int, PRIMARY KEY (a, b, c, d, e))");
+
+        execute("INSERT INTO %s (a, b, c, d, e) VALUES (0, 2, 0, 0, 0)");
+        execute("INSERT INTO %s (a, b, c, d, e) VALUES (0, 1, 0, 0, 0)");
+        execute("INSERT INTO %s (a, b, c, d, e) VALUES (0, 0, 0, 0, 0)");
+        execute("INSERT INTO %s (a, b, c, d, e) VALUES (0, 0, 1, 1, 1)");
+        execute("INSERT INTO %s (a, b, c, d, e) VALUES (0, 0, 2, 2, 2)");
+        execute("INSERT INTO %s (a, b, c, d, e) VALUES (0, 0, 3, 3, 3)");
+        execute("INSERT INTO %s (a, b, c, d, e) VALUES (0, 0, 1, 1, 1)");
+
+        assertRows(execute("SELECT * FROM %s WHERE b=0 AND (c, d, e) > (1, 1, 1) ALLOW FILTERING"),
+                   row(0, 0, 2, 2, 2),
+                   row(0, 0, 3, 3, 3));
+    }
+ }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f797bfa4/test/unit/org/apache/cassandra/cql3/validation/operations/SelectOrderByTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/SelectOrderByTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/SelectOrderByTest.java
new file mode 100644
index 0000000..9d2e594
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/SelectOrderByTest.java
@@ -0,0 +1,503 @@
+/*
+ * 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.validation.operations;
+
+import org.junit.Test;
+
+import org.apache.cassandra.cql3.CQLTester;
+
+import static org.junit.Assert.assertTrue;
+
+public class SelectOrderByTest extends CQLTester
+{
+    @Test
+    public void testNormalSelectionOrderSingleClustering() throws Throwable
+    {
+        for (String descOption : new String[]{"", " WITH CLUSTERING ORDER BY (b DESC)"})
+        {
+            createTable("CREATE TABLE %s (a int, b int, c int, PRIMARY KEY (a, b))" + descOption);
+            execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, 0, 0);
+            execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, 1, 1);
+            execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, 2, 2);
+
+            assertRows(execute("SELECT * FROM %s WHERE a=? ORDER BY b ASC", 0),
+                    row(0, 0, 0),
+                    row(0, 1, 1),
+                    row(0, 2, 2)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a=? ORDER BY b DESC", 0),
+                    row(0, 2, 2),
+                    row(0, 1, 1),
+                    row(0, 0, 0)
+            );
+
+            // order by the only column in the selection
+            assertRows(execute("SELECT b FROM %s WHERE a=? ORDER BY b ASC", 0),
+                    row(0), row(1), row(2));
+
+            assertRows(execute("SELECT b FROM %s WHERE a=? ORDER BY b DESC", 0),
+                    row(2), row(1), row(0));
+
+            // order by a column not in the selection
+            assertRows(execute("SELECT c FROM %s WHERE a=? ORDER BY b ASC", 0),
+                    row(0), row(1), row(2));
+
+            assertRows(execute("SELECT c FROM %s WHERE a=? ORDER BY b DESC", 0),
+                    row(2), row(1), row(0));
+        }
+    }
+
+    @Test
+    public void testFunctionSelectionOrderSingleClustering() throws Throwable
+    {
+        for (String descOption : new String[]{"", " WITH CLUSTERING ORDER BY (b DESC)"})
+        {
+            createTable("CREATE TABLE %s (a int, b int, c int, PRIMARY KEY (a, b))" + descOption);
+            execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, 0, 0);
+            execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, 1, 1);
+            execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, 2, 2);
+
+            // order by the only column in the selection
+            assertRows(execute("SELECT blobAsInt(intAsBlob(b)) FROM %s WHERE a=? ORDER BY b ASC", 0),
+                    row(0), row(1), row(2));
+
+            assertRows(execute("SELECT blobAsInt(intAsBlob(b)) FROM %s WHERE a=? ORDER BY b DESC", 0),
+                    row(2), row(1), row(0));
+
+            // order by a column not in the selection
+            assertRows(execute("SELECT blobAsInt(intAsBlob(c)) FROM %s WHERE a=? ORDER BY b ASC", 0),
+                    row(0), row(1), row(2));
+
+            assertRows(execute("SELECT blobAsInt(intAsBlob(c)) FROM %s WHERE a=? ORDER BY b DESC", 0),
+                    row(2), row(1), row(0));
+
+            assertInvalid("SELECT * FROM %s WHERE a=? ORDER BY c ASC", 0);
+            assertInvalid("SELECT * FROM %s WHERE a=? ORDER BY c DESC", 0);
+        }
+    }
+
+    @Test
+    public void testFieldSelectionOrderSingleClustering() throws Throwable
+    {
+        String type = createType("CREATE TYPE %s (a int)");
+
+        for (String descOption : new String[]{"", " WITH CLUSTERING ORDER BY (b DESC)"})
+        {
+            createTable("CREATE TABLE %s (a int, b int, c frozen<" + type + "   >, PRIMARY KEY (a, b))" + descOption);
+            execute("INSERT INTO %s (a, b, c) VALUES (?, ?, {a: ?})", 0, 0, 0);
+            execute("INSERT INTO %s (a, b, c) VALUES (?, ?, {a: ?})", 0, 1, 1);
+            execute("INSERT INTO %s (a, b, c) VALUES (?, ?, {a: ?})", 0, 2, 2);
+
+            // order by a column not in the selection
+            assertRows(execute("SELECT c.a FROM %s WHERE a=? ORDER BY b ASC", 0),
+                    row(0), row(1), row(2));
+
+            assertRows(execute("SELECT c.a FROM %s WHERE a=? ORDER BY b DESC", 0),
+                    row(2), row(1), row(0));
+
+            assertRows(execute("SELECT blobAsInt(intAsBlob(c.a)) FROM %s WHERE a=? ORDER BY b DESC", 0),
+                       row(2), row(1), row(0));
+            dropTable("DROP TABLE %s");
+        }
+    }
+
+    @Test
+    public void testNormalSelectionOrderMultipleClustering() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int, b int, c int, d int, PRIMARY KEY (a, b, c))");
+        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 0, 0);
+        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 1, 1);
+        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 2, 2);
+        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 0, 3);
+        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 1, 4);
+        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 2, 5);
+
+        assertRows(execute("SELECT * FROM %s WHERE a=? ORDER BY b ASC", 0),
+                row(0, 0, 0, 0),
+                row(0, 0, 1, 1),
+                row(0, 0, 2, 2),
+                row(0, 1, 0, 3),
+                row(0, 1, 1, 4),
+                row(0, 1, 2, 5)
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE a=? ORDER BY b DESC", 0),
+                row(0, 1, 2, 5),
+                row(0, 1, 1, 4),
+                row(0, 1, 0, 3),
+                row(0, 0, 2, 2),
+                row(0, 0, 1, 1),
+                row(0, 0, 0, 0)
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE a=? ORDER BY b DESC, c DESC", 0),
+                row(0, 1, 2, 5),
+                row(0, 1, 1, 4),
+                row(0, 1, 0, 3),
+                row(0, 0, 2, 2),
+                row(0, 0, 1, 1),
+                row(0, 0, 0, 0)
+        );
+
+        assertInvalid("SELECT * FROM %s WHERE a=? ORDER BY c ASC", 0);
+        assertInvalid("SELECT * FROM %s WHERE a=? ORDER BY c DESC", 0);
+        assertInvalid("SELECT * FROM %s WHERE a=? ORDER BY b ASC, c DESC", 0);
+        assertInvalid("SELECT * FROM %s WHERE a=? ORDER BY b DESC, c ASC", 0);
+        assertInvalid("SELECT * FROM %s WHERE a=? ORDER BY d ASC", 0);
+
+        // select and order by b
+        assertRows(execute("SELECT b FROM %s WHERE a=? ORDER BY b ASC", 0),
+                row(0), row(0), row(0), row(1), row(1), row(1));
+        assertRows(execute("SELECT b FROM %s WHERE a=? ORDER BY b DESC", 0),
+                row(1), row(1), row(1), row(0), row(0), row(0));
+
+        // select c, order by b
+        assertRows(execute("SELECT c FROM %s WHERE a=? ORDER BY b ASC", 0),
+                row(0), row(1), row(2), row(0), row(1), row(2));
+        assertRows(execute("SELECT c FROM %s WHERE a=? ORDER BY b DESC", 0),
+                row(2), row(1), row(0), row(2), row(1), row(0));
+
+        // select c, order by b, c
+        assertRows(execute("SELECT c FROM %s WHERE a=? ORDER BY b ASC, c ASC", 0),
+                row(0), row(1), row(2), row(0), row(1), row(2));
+        assertRows(execute("SELECT c FROM %s WHERE a=? ORDER BY b DESC, c DESC", 0),
+                row(2), row(1), row(0), row(2), row(1), row(0));
+
+        // select d, order by b, c
+        assertRows(execute("SELECT d FROM %s WHERE a=? ORDER BY b ASC, c ASC", 0),
+                row(0), row(1), row(2), row(3), row(4), row(5));
+        assertRows(execute("SELECT d FROM %s WHERE a=? ORDER BY b DESC, c DESC", 0),
+                row(5), row(4), row(3), row(2), row(1), row(0));
+    }
+
+    @Test
+    public void testFunctionSelectionOrderMultipleClustering() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int, b int, c int, d int, PRIMARY KEY (a, b, c))");
+        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 0, 0);
+        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 1, 1);
+        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 2, 2);
+        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 0, 3);
+        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 1, 4);
+        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 2, 5);
+
+        assertInvalid("SELECT blobAsInt(intAsBlob(b)) FROM %s WHERE a=? ORDER BY c ASC", 0);
+        assertInvalid("SELECT blobAsInt(intAsBlob(b)) FROM %s WHERE a=? ORDER BY c DESC", 0);
+        assertInvalid("SELECT blobAsInt(intAsBlob(b)) FROM %s WHERE a=? ORDER BY b ASC, c DESC", 0);
+        assertInvalid("SELECT blobAsInt(intAsBlob(b)) FROM %s WHERE a=? ORDER BY b DESC, c ASC", 0);
+        assertInvalid("SELECT blobAsInt(intAsBlob(b)) FROM %s WHERE a=? ORDER BY d ASC", 0);
+
+        // select and order by b
+        assertRows(execute("SELECT blobAsInt(intAsBlob(b)) FROM %s WHERE a=? ORDER BY b ASC", 0),
+                   row(0), row(0), row(0), row(1), row(1), row(1));
+        assertRows(execute("SELECT blobAsInt(intAsBlob(b)) FROM %s WHERE a=? ORDER BY b DESC", 0),
+                   row(1), row(1), row(1), row(0), row(0), row(0));
+
+        assertRows(execute("SELECT b, blobAsInt(intAsBlob(b)) FROM %s WHERE a=? ORDER BY b ASC", 0),
+                row(0, 0), row(0, 0), row(0, 0), row(1, 1), row(1, 1), row(1, 1));
+        assertRows(execute("SELECT b, blobAsInt(intAsBlob(b)) FROM %s WHERE a=? ORDER BY b DESC", 0),
+                row(1, 1), row(1, 1), row(1, 1), row(0, 0), row(0, 0), row(0, 0));
+
+        // select c, order by b
+        assertRows(execute("SELECT blobAsInt(intAsBlob(c)) FROM %s WHERE a=? ORDER BY b ASC", 0),
+                row(0), row(1), row(2), row(0), row(1), row(2));
+        assertRows(execute("SELECT blobAsInt(intAsBlob(c)) FROM %s WHERE a=? ORDER BY b DESC", 0),
+                row(2), row(1), row(0), row(2), row(1), row(0));
+
+        // select c, order by b, c
+        assertRows(execute("SELECT blobAsInt(intAsBlob(c)) FROM %s WHERE a=? ORDER BY b ASC, c ASC", 0),
+                row(0), row(1), row(2), row(0), row(1), row(2));
+        assertRows(execute("SELECT blobAsInt(intAsBlob(c)) FROM %s WHERE a=? ORDER BY b DESC, c DESC", 0),
+                row(2), row(1), row(0), row(2), row(1), row(0));
+
+        // select d, order by b, c
+        assertRows(execute("SELECT blobAsInt(intAsBlob(d)) FROM %s WHERE a=? ORDER BY b ASC, c ASC", 0),
+                row(0), row(1), row(2), row(3), row(4), row(5));
+        assertRows(execute("SELECT blobAsInt(intAsBlob(d)) FROM %s WHERE a=? ORDER BY b DESC, c DESC", 0),
+                row(5), row(4), row(3), row(2), row(1), row(0));
+
+    }
+
+    /**
+     * Check ORDER BY support in SELECT statement
+     * migrated from cql_tests.py:TestCQL.order_by_test()
+     */
+    @Test
+    public void testSimpleOrderBy() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int, c int, v int, PRIMARY KEY (k, c)) WITH COMPACT STORAGE");
+
+        for (int i = 0; i < 10; i++)
+            execute("INSERT INTO %s (k, c, v) VALUES (0, ?, ?)", i, i);
+
+        assertRows(execute("SELECT v FROM %s WHERE k = 0 ORDER BY c DESC"),
+                   row(9), row(8), row(7), row(6), row(5), row(4), row(3), row(2), row(1), row(0));
+
+        createTable("CREATE TABLE %s (k int, c1 int, c2 int, v int, PRIMARY KEY (k, c1, c2)) WITH COMPACT STORAGE");
+
+        for (int i = 0; i < 4; i++)
+            for (int j = 0; j < 2; j++)
+                execute("INSERT INTO %s (k, c1, c2, v) VALUES (0, ?, ?, ?)", i, j, i * 2 + j);
+
+        assertInvalid("SELECT v FROM %s WHERE k = 0 ORDER BY c DESC");
+        assertInvalid("SELECT v FROM %s WHERE k = 0 ORDER BY c2 DESC");
+        assertInvalid("SELECT v FROM %s WHERE k = 0 ORDER BY k DESC");
+
+        assertRows(execute("SELECT v FROM %s WHERE k = 0 ORDER BY c1 DESC"),
+                   row(7), row(6), row(5), row(4), row(3), row(2), row(1), row(0));
+
+        assertRows(execute("SELECT v FROM %s WHERE k = 0 ORDER BY c1"),
+                   row(0), row(1), row(2), row(3), row(4), row(5), row(6), row(7));
+    }
+
+    /**
+     * More ORDER BY checks (#4160)
+     * migrated from cql_tests.py:TestCQL.more_order_by_test()
+     */
+    @Test
+    public void testMoreOrderBy() throws Throwable
+    {
+        createTable("CREATE TABLE %s (row text, number int, string text, PRIMARY KEY(row, number)) WITH COMPACT STORAGE ");
+
+        execute("INSERT INTO %s (row, number, string) VALUES ('row', 1, 'one')");
+        execute("INSERT INTO %s (row, number, string) VALUES ('row', 2, 'two')");
+        execute("INSERT INTO %s (row, number, string) VALUES ('row', 3, 'three')");
+        execute("INSERT INTO %s (row, number, string) VALUES ('row', 4, 'four')");
+
+        assertRows(execute("SELECT number FROM %s WHERE row='row' AND number < 3 ORDER BY number ASC"),
+                   row(1), row(2));
+
+        assertRows(execute("SELECT number FROM %s WHERE row='row' AND number >= 3 ORDER BY number ASC"),
+                   row(3), row(4));
+
+        assertRows(execute("SELECT number FROM %s WHERE row='row' AND number < 3 ORDER BY number DESC"),
+                   row(2), row(1));
+
+        assertRows(execute("SELECT number FROM %s WHERE row='row' AND number >= 3 ORDER BY number DESC"),
+                   row(4), row(3));
+
+        assertRows(execute("SELECT number FROM %s WHERE row='row' AND number > 3 ORDER BY number DESC"),
+                   row(4));
+
+        assertRows(execute("SELECT number FROM %s WHERE row='row' AND number <= 3 ORDER BY number DESC"),
+                   row(3), row(2), row(1));
+    }
+
+    /**
+     * Check we don't allow order by on row key (#4246)
+     * migrated from cql_tests.py:TestCQL.order_by_validation_test()
+     */
+    @Test
+    public void testInvalidOrderBy() throws Throwable
+    {
+        createTable("CREATE TABLE %s( k1 int, k2 int, v int, PRIMARY KEY (k1, k2))");
+
+        execute("INSERT INTO %s (k1, k2, v) VALUES (?, ?, ?)", 0, 0, 0);
+        execute("INSERT INTO %s (k1, k2, v) VALUES (?, ?, ?)", 1, 1, 1);
+        execute("INSERT INTO %s (k1, k2, v) VALUES (?, ?, ?)", 2, 2, 2);
+
+        assertInvalid("SELECT * FROM %s ORDER BY k2");
+    }
+
+    /**
+     * Check that order-by works with IN (#4327)
+     * migrated from cql_tests.py:TestCQL.order_by_with_in_test()
+     */
+    @Test
+    public void testOrderByForInClause() throws Throwable
+    {
+        createTable("CREATE TABLE %s (my_id varchar, col1 int, value varchar, PRIMARY KEY (my_id, col1))");
+
+        execute("INSERT INTO %s (my_id, col1, value) VALUES ( 'key1', 1, 'a')");
+        execute("INSERT INTO %s (my_id, col1, value) VALUES ( 'key2', 3, 'c')");
+        execute("INSERT INTO %s (my_id, col1, value) VALUES ( 'key3', 2, 'b')");
+        execute("INSERT INTO %s (my_id, col1, value) VALUES ( 'key4', 4, 'd')");
+
+        assertRows(execute("SELECT col1 FROM %s WHERE my_id in('key1', 'key2', 'key3') ORDER BY col1"),
+                   row(1), row(2), row(3));
+
+        assertRows(execute("SELECT col1, my_id FROM %s WHERE my_id in('key1', 'key2', 'key3') ORDER BY col1"),
+                   row(1, "key1"), row(2, "key3"), row(3, "key2"));
+
+        assertRows(execute("SELECT my_id, col1 FROM %s WHERE my_id in('key1', 'key2', 'key3') ORDER BY col1"),
+                   row("key1", 1), row("key3", 2), row("key2", 3));
+    }
+
+    /**
+     * Test reversed comparators
+     * migrated from cql_tests.py:TestCQL.reversed_comparator_test()
+     */
+    @Test
+    public void testReversedComparator() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int, c int, v int, PRIMARY KEY (k, c)) WITH CLUSTERING ORDER BY (c DESC);");
+
+        for(int i =0; i < 10; i++)
+            execute("INSERT INTO %s (k, c, v) VALUES (0, ?, ?)", i, i);
+
+        assertRows(execute("SELECT c, v FROM %s WHERE k = 0 ORDER BY c ASC"),
+                   row(0, 0), row(1, 1), row(2, 2), row(3, 3), row(4, 4),
+                   row(5, 5), row(6, 6), row(7, 7), row(8, 8), row(9, 9));
+
+        assertRows(execute("SELECT c, v FROM %s WHERE k = 0 ORDER BY c DESC"),
+                   row(9, 9), row(8, 8), row(7, 7), row(6, 6), row(5, 5),
+                   row(4, 4), row(3, 3), row(2, 2), row(1, 1), row(0, 0));
+
+        createTable("CREATE TABLE %s (k int, c1 int, c2 int, v text, PRIMARY KEY (k, c1, c2)) WITH CLUSTERING ORDER BY (c1 ASC, c2 DESC)");
+
+        for(int i = 0; i < 10; i++)
+            for(int j = 0; j < 10; j++)
+                execute("INSERT INTO %s (k, c1, c2, v) VALUES (0, ?, ?, ?)", i, j, String.format("%d%d", i, j));
+
+        assertInvalid("SELECT c1, c2, v FROM %s WHERE k = 0 ORDER BY c1 ASC, c2 ASC");
+        assertInvalid("SELECT c1, c2, v FROM %s WHERE k = 0 ORDER BY c1 DESC, c2 DESC");
+
+        Object[][] expectedRows = new Object[100][];
+        for(int i = 0; i < 10; i++)
+            for(int j = 9; j >= 0; j--)
+                expectedRows[i * 10 + (9 - j)] = row(i, j, String.format("%d%d", i, j));
+
+        assertRows(execute("SELECT c1, c2, v FROM %s WHERE k = 0 ORDER BY c1 ASC"),
+                   expectedRows);
+
+        assertRows(execute("SELECT c1, c2, v FROM %s WHERE k = 0 ORDER BY c1 ASC, c2 DESC"),
+                   expectedRows);
+
+        for(int i = 9; i >= 0; i--)
+            for(int j = 0; j < 10; j++)
+                expectedRows[(9 - i) * 10 + j] = row(i, j, String.format("%d%d", i, j));
+
+        assertRows(execute("SELECT c1, c2, v FROM %s WHERE k = 0 ORDER BY c1 DESC, c2 ASC"),
+                   expectedRows);
+
+        assertInvalid("SELECT c1, c2, v FROM %s WHERE k = 0 ORDER BY c2 DESC, c1 ASC");
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.multiordering_test()
+     */
+    @Test
+    public void testMultiordering() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k text, c1 int, c2 int, PRIMARY KEY (k, c1, c2) ) WITH CLUSTERING ORDER BY (c1 ASC, c2 DESC)");
+
+        for (int i = 0; i < 2; i++)
+            for (int j = 0; j < 2; j++)
+                execute("INSERT INTO %s (k, c1, c2) VALUES ('foo', ?, ?)", i, j);
+
+        assertRows(execute("SELECT c1, c2 FROM %s WHERE k = 'foo'"),
+                   row(0, 1), row(0, 0), row(1, 1), row(1, 0));
+
+        assertRows(execute("SELECT c1, c2 FROM %s WHERE k = 'foo' ORDER BY c1 ASC, c2 DESC"),
+                   row(0, 1), row(0, 0), row(1, 1), row(1, 0));
+
+        assertRows(execute("SELECT c1, c2 FROM %s WHERE k = 'foo' ORDER BY c1 DESC, c2 ASC"),
+                   row(1, 0), row(1, 1), row(0, 0), row(0, 1));
+
+        assertInvalid("SELECT c1, c2 FROM %s WHERE k = 'foo' ORDER BY c2 DESC");
+        assertInvalid("SELECT c1, c2 FROM %s WHERE k = 'foo' ORDER BY c2 ASC");
+        assertInvalid("SELECT c1, c2 FROM %s WHERE k = 'foo' ORDER BY c1 ASC, c2 ASC");
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.in_with_desc_order_test()
+     */
+    @Test
+    public void testSelectInStatementWithDesc() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int, c1 int, c2 int, PRIMARY KEY (k, c1, c2))");
+        execute("INSERT INTO %s(k, c1, c2) VALUES (0, 0, 0)");
+        execute("INSERT INTO %s(k, c1, c2) VALUES (0, 0, 1)");
+        execute("INSERT INTO %s(k, c1, c2) VALUES (0, 0, 2)");
+
+        assertRows(execute("SELECT * FROM %s WHERE k=0 AND c1 = 0 AND c2 IN (2, 0) ORDER BY c1 DESC"),
+                   row(0, 0, 2),
+                   row(0, 0, 0));
+    }
+
+    /**
+     * Test that columns don't need to be selected for ORDER BY when there is a IN (#4911),
+     * migrated from cql_tests.py:TestCQL.in_order_by_without_selecting_test()
+     */
+    @Test
+    public void testInOrderByWithoutSelecting() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int, c1 int, c2 int, v int, PRIMARY KEY (k, c1, c2))");
+
+        execute("INSERT INTO %s (k, c1, c2, v) VALUES (0, 0, 0, 0)");
+        execute("INSERT INTO %s (k, c1, c2, v) VALUES (0, 0, 1, 1)");
+        execute("INSERT INTO %s (k, c1, c2, v) VALUES (0, 0, 2, 2)");
+        execute("INSERT INTO %s (k, c1, c2, v) VALUES (1, 1, 0, 3)");
+        execute("INSERT INTO %s (k, c1, c2, v) VALUES (1, 1, 1, 4)");
+        execute("INSERT INTO %s (k, c1, c2, v) VALUES (1, 1, 2, 5)");
+
+        assertRows(execute("SELECT * FROM %s WHERE k=0 AND c1 = 0 AND c2 IN (2, 0)"),
+                   row(0, 0, 0, 0),
+                   row(0, 0, 2, 2));
+        assertRows(execute("SELECT * FROM %s WHERE k=0 AND c1 = 0 AND c2 IN (2, 0) ORDER BY c1 ASC, c2 ASC"),
+                   row(0, 0, 0, 0),
+                   row(0, 0, 2, 2));
+
+        // check that we don 't need to select the column on which we order
+        assertRows(execute("SELECT v FROM %s WHERE k=0 AND c1 = 0 AND c2 IN (2, 0)"),
+                   row(0),
+                   row(2));
+        assertRows(execute("SELECT v FROM %s WHERE k=0 AND c1 = 0 AND c2 IN (2, 0) ORDER BY c1 ASC"),
+                   row(0),
+                   row(2));
+        assertRows(execute("SELECT v FROM %s WHERE k=0 AND c1 = 0 AND c2 IN (2, 0) ORDER BY c1 DESC"),
+                   row(2),
+                   row(0));
+        assertRows(execute("SELECT v FROM %s WHERE k IN (1, 0)"),
+                   row(3),
+                   row(4),
+                   row(5),
+                   row(0),
+                   row(1),
+                   row(2));
+
+        assertRows(execute("SELECT v FROM %s WHERE k IN (1, 0) ORDER BY c1 ASC"),
+                   row(0),
+                   row(1),
+                   row(2),
+                   row(3),
+                   row(4),
+                   row(5));
+
+        // we should also be able to use functions in the select clause (additional test for CASSANDRA - 8286)
+        Object[][] results = getRows(execute("SELECT writetime(v) FROM %s WHERE k IN (1, 0) ORDER BY c1 ASC"));
+
+        // since we don 't know the write times, just assert that the order matches the order we expect
+        assertTrue(isFirstIntSorted(results));
+    }
+
+    private boolean isFirstIntSorted(Object[][] rows)
+    {
+        for (int i = 1; i < rows.length; i++)
+        {
+            Long prev = (Long)rows[i-1][0];
+            Long curr = (Long)rows[i][0];
+
+            if (prev > curr)
+                return false;
+        }
+
+        return true;
+    }
+}


[31/32] cassandra git commit: 2.2 commit for CASSANDRA-9160

Posted by jm...@apache.org.
2.2 commit for CASSANDRA-9160


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

Branch: refs/heads/trunk
Commit: 01115f72fc50b603ece0a00431308abec24706b7
Parents: 20364f4
Author: Stefania Alborghetti <st...@datastax.com>
Authored: Wed Jun 24 12:11:46 2015 -0400
Committer: Josh McKenzie <jo...@datastax.com>
Committed: Wed Jun 24 12:11:46 2015 -0400

----------------------------------------------------------------------
 .../cassandra/config/DatabaseDescriptor.java    |    6 +
 .../org/apache/cassandra/cql3/ResultSet.java    |    9 +
 .../apache/cassandra/cql3/UntypedResultSet.java |    2 +-
 .../cql3/statements/BatchStatement.java         |   69 +-
 .../cql3/statements/CQL3CasRequest.java         |    8 +-
 .../cql3/statements/ModificationStatement.java  |   70 +-
 .../cql3/statements/SelectStatement.java        |   84 +-
 .../cql3/statements/TruncateStatement.java      |   13 +-
 .../apache/cassandra/service/StorageProxy.java  |    4 +-
 .../org/apache/cassandra/utils/UUIDGen.java     |   16 +-
 .../org/apache/cassandra/cql3/ManyRowsTest.java |   92 +
 .../apache/cassandra/cql3/AggregationTest.java  | 1479 ----------
 .../org/apache/cassandra/cql3/AliasTest.java    |   40 -
 .../apache/cassandra/cql3/AlterTableTest.java   |  113 -
 .../org/apache/cassandra/cql3/CQLTester.java    |  172 +-
 .../apache/cassandra/cql3/CollectionsTest.java  |  340 ---
 .../cassandra/cql3/ContainsRelationTest.java    |  283 --
 .../cassandra/cql3/CrcCheckChanceTest.java      |  159 --
 .../cql3/CreateAndAlterKeyspaceTest.java        |   37 -
 .../cql3/CreateIndexStatementTest.java          |  101 -
 .../apache/cassandra/cql3/CreateTableTest.java  |   69 -
 .../cql3/CreateTriggerStatementTest.java        |  121 -
 .../cassandra/cql3/FrozenCollectionsTest.java   | 1101 --------
 .../cql3/IndexedValuesValidationTest.java       |  149 -
 .../org/apache/cassandra/cql3/JsonTest.java     |  947 -------
 .../apache/cassandra/cql3/ModificationTest.java |  112 -
 .../cassandra/cql3/MultiColumnRelationTest.java |  936 -------
 .../org/apache/cassandra/cql3/PgStringTest.java |   76 -
 .../cassandra/cql3/RangeDeletionTest.java       |   35 -
 .../apache/cassandra/cql3/RoleSyntaxTest.java   |   51 -
 .../cql3/SSTableMetadataTrackingTest.java       |  160 --
 .../cql3/SecondaryIndexOnMapEntriesTest.java    |  337 ---
 .../cql3/SelectWithTokenFunctionTest.java       |  233 --
 .../cassandra/cql3/SelectionOrderingTest.java   |  233 --
 .../cql3/SingleColumnRelationTest.java          |  553 ----
 .../SliceQueryFilterWithTombstonesTest.java     |  170 --
 .../cassandra/cql3/StaticColumnsQueryTest.java  |  280 --
 .../cassandra/cql3/ThriftCompatibilityTest.java |    1 +
 .../apache/cassandra/cql3/TimestampTest.java    |   36 -
 .../apache/cassandra/cql3/TupleTypeTest.java    |  114 -
 .../org/apache/cassandra/cql3/TypeCastTest.java |   54 -
 .../org/apache/cassandra/cql3/TypeTest.java     |   89 -
 .../org/apache/cassandra/cql3/UFAuthTest.java   |  724 -----
 .../cassandra/cql3/UFIdentificationTest.java    |  376 ---
 test/unit/org/apache/cassandra/cql3/UFTest.java | 2585 -----------------
 .../apache/cassandra/cql3/UseStatementTest.java |   29 -
 .../apache/cassandra/cql3/UserTypesTest.java    |  334 ---
 .../selection/SelectionColumnMappingTest.java   |    9 +
 .../validation/entities/CollectionsTest.java    |  588 ++++
 .../cql3/validation/entities/CountersTest.java  |  115 +
 .../cql3/validation/entities/DateTypeTest.java  |   39 +
 .../entities/FrozenCollectionsTest.java         | 1111 ++++++++
 .../cql3/validation/entities/JsonTest.java      |  958 +++++++
 .../SecondaryIndexOnMapEntriesTest.java         |  348 +++
 .../validation/entities/SecondaryIndexTest.java |  645 +++++
 .../validation/entities/StaticColumnsTest.java  |  271 ++
 .../cql3/validation/entities/TimestampTest.java |  155 ++
 .../cql3/validation/entities/TimeuuidTest.java  |   81 +
 .../cql3/validation/entities/TupleTypeTest.java |  171 ++
 .../cql3/validation/entities/TypeTest.java      |   92 +
 .../cql3/validation/entities/UFAuthTest.java    |  728 +++++
 .../entities/UFIdentificationTest.java          |  380 +++
 .../cql3/validation/entities/UFTest.java        | 2596 ++++++++++++++++++
 .../cql3/validation/entities/UserTypesTest.java |  404 +++
 .../miscellaneous/CrcCheckChanceTest.java       |  160 ++
 .../validation/miscellaneous/OverflowTest.java  |  331 +++
 .../validation/miscellaneous/PgStringTest.java  |   77 +
 .../miscellaneous/RoleSyntaxTest.java           |   53 +
 .../SSTableMetadataTrackingTest.java            |  161 ++
 .../miscellaneous/TombstonesTest.java           |  171 ++
 .../validation/operations/AggregationTest.java  | 1481 ++++++++++
 .../cql3/validation/operations/AlterTest.java   |  203 ++
 .../cql3/validation/operations/BatchTest.java   |  106 +
 .../cql3/validation/operations/CreateTest.java  |  498 ++++
 .../cql3/validation/operations/DeleteTest.java  |  329 +++
 .../cql3/validation/operations/InsertTest.java  |   59 +
 .../operations/InsertUpdateIfCondition.java     |  861 ++++++
 .../validation/operations/SelectLimitTest.java  |  112 +
 .../SelectMultiColumnRelationTest.java          |  962 +++++++
 .../operations/SelectOrderByTest.java           |  504 ++++
 .../SelectOrderedPartitionerTest.java           |  481 ++++
 .../SelectSingleColumnRelationTest.java         |  555 ++++
 .../cql3/validation/operations/SelectTest.java  | 1336 +++++++++
 .../cql3/validation/operations/UpdateTest.java  |   86 +
 .../cql3/validation/operations/UseTest.java     |   31 +
 .../cassandra/service/ClientWarningsTest.java   |    5 +-
 .../cassandra/transport/MessagePayloadTest.java |    6 +-
 .../stress/generate/values/TimeUUIDs.java       |    2 +-
 88 files changed, 17720 insertions(+), 12543 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/01115f72/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
index 443246e..39a06cb 100644
--- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
+++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
@@ -1544,6 +1544,12 @@ public class DatabaseDescriptor
         return conf.row_cache_size_in_mb;
     }
 
+    @VisibleForTesting
+    public static void setRowCacheSizeInMB(long val)
+    {
+        conf.row_cache_size_in_mb = val;
+    }
+
     public static int getRowCacheSavePeriod()
     {
         return conf.row_cache_save_period;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/01115f72/src/java/org/apache/cassandra/cql3/ResultSet.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/ResultSet.java b/src/java/org/apache/cassandra/cql3/ResultSet.java
index 281923c..ea26f34 100644
--- a/src/java/org/apache/cassandra/cql3/ResultSet.java
+++ b/src/java/org/apache/cassandra/cql3/ResultSet.java
@@ -254,6 +254,15 @@ public class ResultSet
             return new ResultMetadata(EnumSet.copyOf(flags), names, columnCount, pagingState);
         }
 
+        /**
+         * Return only the column names requested by the user, excluding those added for post-query re-orderings,
+         * see definition of names and columnCount.
+         **/
+        public List<ColumnSpecification> requestNames()
+        {
+            return names.subList(0, columnCount);
+        }
+
         // The maximum number of values that the ResultSet can hold. This can be bigger than columnCount due to CASSANDRA-4911
         public int valueCount()
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/01115f72/src/java/org/apache/cassandra/cql3/UntypedResultSet.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/UntypedResultSet.java b/src/java/org/apache/cassandra/cql3/UntypedResultSet.java
index bf3cbb5..49e0d86 100644
--- a/src/java/org/apache/cassandra/cql3/UntypedResultSet.java
+++ b/src/java/org/apache/cassandra/cql3/UntypedResultSet.java
@@ -95,7 +95,7 @@ public abstract class UntypedResultSet implements Iterable<UntypedResultSet.Row>
 
         public List<ColumnSpecification> metadata()
         {
-            return cqlRows.metadata.names;
+            return cqlRows.metadata.requestNames();
         }
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/01115f72/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java b/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
index b1751a2..0661b56 100644
--- a/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
@@ -38,9 +38,11 @@ import org.apache.cassandra.service.ClientState;
 import org.apache.cassandra.service.ClientWarn;
 import org.apache.cassandra.service.QueryState;
 import org.apache.cassandra.service.StorageProxy;
+import org.apache.cassandra.thrift.Column;
 import org.apache.cassandra.tracing.Tracing;
 import org.apache.cassandra.transport.messages.ResultMessage;
 import org.apache.cassandra.utils.NoSpamLogger;
+import org.apache.cassandra.utils.Pair;
 
 /**
  * A <code>BATCH</code> statement parsed from a CQL query.
@@ -343,10 +345,31 @@ public class BatchStatement implements CQLStatement
     private ResultMessage executeWithConditions(BatchQueryOptions options, QueryState state)
     throws RequestExecutionException, RequestValidationException
     {
+        Pair<CQL3CasRequest, Set<ColumnDefinition>> p = makeCasRequest(options, state);
+        CQL3CasRequest casRequest = p.left;
+        Set<ColumnDefinition> columnsWithConditions = p.right;
+
+        ColumnFamily result = StorageProxy.cas(casRequest.cfm.ksName,
+                                               casRequest.cfm.cfName,
+                                               casRequest.key,
+                                               casRequest,
+                                               options.getSerialConsistency(),
+                                               options.getConsistency(),
+                                               state.getClientState());
+
+        return new ResultMessage.Rows(ModificationStatement.buildCasResultSet(casRequest.cfm.ksName,
+                                                                              casRequest.key,
+                                                                              casRequest.cfm.cfName,
+                                                                              result,
+                                                                              columnsWithConditions,
+                                                                              true,
+                                                                              options.forStatement(0)));
+    }
+
+    private Pair<CQL3CasRequest,Set<ColumnDefinition>> makeCasRequest(BatchQueryOptions options, QueryState state)
+    {
         long now = state.getTimestamp();
         ByteBuffer key = null;
-        String ksName = null;
-        String cfName = null;
         CQL3CasRequest casRequest = null;
         Set<ColumnDefinition> columnsWithConditions = new LinkedHashSet<>();
 
@@ -361,8 +384,6 @@ public class BatchStatement implements CQLStatement
             if (key == null)
             {
                 key = pks.get(0);
-                ksName = statement.cfm.ksName;
-                cfName = statement.cfm.cfName;
                 casRequest = new CQL3CasRequest(statement.cfm, key, true);
             }
             else if (!key.equals(pks.get(0)))
@@ -383,23 +404,49 @@ public class BatchStatement implements CQLStatement
             casRequest.addRowUpdate(clusteringPrefix, statement, statementOptions, timestamp);
         }
 
-        ColumnFamily result = StorageProxy.cas(ksName, cfName, key, casRequest, options.getSerialConsistency(), options.getConsistency(), state.getClientState());
-
-        return new ResultMessage.Rows(ModificationStatement.buildCasResultSet(ksName, key, cfName, result, columnsWithConditions, true, options.forStatement(0)));
+        return Pair.create(casRequest, columnsWithConditions);
     }
 
     public ResultMessage executeInternal(QueryState queryState, QueryOptions options) throws RequestValidationException, RequestExecutionException
     {
-        assert !hasConditions;
+        if (hasConditions)
+            return executeInternalWithConditions(BatchQueryOptions.withoutPerStatementVariables(options), queryState);
+
+        executeInternalWithoutCondition(queryState, options);
+        return new ResultMessage.Void();
+    }
+
+    private ResultMessage executeInternalWithoutCondition(QueryState queryState, QueryOptions options) throws RequestValidationException, RequestExecutionException
+    {
         for (IMutation mutation : getMutations(BatchQueryOptions.withoutPerStatementVariables(options), true, queryState.getTimestamp()))
         {
-            // We don't use counters internally.
-            assert mutation instanceof Mutation;
-            ((Mutation) mutation).apply();
+            assert mutation instanceof Mutation || mutation instanceof CounterMutation;
+
+            if (mutation instanceof Mutation)
+                ((Mutation) mutation).apply();
+            else if (mutation instanceof CounterMutation)
+                ((CounterMutation) mutation).apply();
         }
         return null;
     }
 
+    private ResultMessage executeInternalWithConditions(BatchQueryOptions options, QueryState state) throws RequestExecutionException, RequestValidationException
+    {
+        Pair<CQL3CasRequest, Set<ColumnDefinition>> p = makeCasRequest(options, state);
+        CQL3CasRequest request = p.left;
+        Set<ColumnDefinition> columnsWithConditions = p.right;
+
+        ColumnFamily result = ModificationStatement.casInternal(request, state);
+
+        return new ResultMessage.Rows(ModificationStatement.buildCasResultSet(request.cfm.ksName,
+                                                                              request.key,
+                                                                              request.cfm.cfName,
+                                                                              result,
+                                                                              columnsWithConditions,
+                                                                              true,
+                                                                              options.forStatement(0)));
+    }
+
     public interface BatchVariables
     {
         public List<ByteBuffer> getVariablesForStatement(int statementInBatch);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/01115f72/src/java/org/apache/cassandra/cql3/statements/CQL3CasRequest.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/CQL3CasRequest.java b/src/java/org/apache/cassandra/cql3/statements/CQL3CasRequest.java
index 4ff9c27..081a14e 100644
--- a/src/java/org/apache/cassandra/cql3/statements/CQL3CasRequest.java
+++ b/src/java/org/apache/cassandra/cql3/statements/CQL3CasRequest.java
@@ -36,10 +36,10 @@ import org.apache.cassandra.utils.Pair;
  */
 public class CQL3CasRequest implements CASRequest
 {
-    private final CFMetaData cfm;
-    private final ByteBuffer key;
-    private final long now;
-    private final boolean isBatch;
+    final CFMetaData cfm;
+    final ByteBuffer key;
+    final long now;
+    final boolean isBatch;
 
     // We index RowCondition by the prefix of the row they applied to for 2 reasons:
     //   1) this allows to keep things sorted to build the ColumnSlice array below

http://git-wip-us.apache.org/repos/asf/cassandra/blob/01115f72/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java b/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
index 0862a9f..aac94be 100644
--- a/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
@@ -26,6 +26,7 @@ import com.google.common.collect.Lists;
 import org.apache.cassandra.auth.Permission;
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.cql3.*;
 import org.apache.cassandra.cql3.functions.Function;
 import org.apache.cassandra.cql3.restrictions.Restriction;
@@ -41,9 +42,12 @@ import org.apache.cassandra.exceptions.*;
 import org.apache.cassandra.service.ClientState;
 import org.apache.cassandra.service.QueryState;
 import org.apache.cassandra.service.StorageProxy;
+import org.apache.cassandra.service.paxos.Commit;
 import org.apache.cassandra.thrift.ThriftValidation;
 import org.apache.cassandra.transport.messages.ResultMessage;
+import org.apache.cassandra.triggers.TriggerExecutor;
 import org.apache.cassandra.utils.Pair;
+import org.apache.cassandra.utils.UUIDGen;
 
 import static org.apache.cassandra.cql3.statements.RequestValidations.checkFalse;
 import static org.apache.cassandra.cql3.statements.RequestValidations.checkNotNull;
@@ -486,6 +490,20 @@ public abstract class ModificationStatement implements CQLStatement
     public ResultMessage executeWithCondition(QueryState queryState, QueryOptions options)
     throws RequestExecutionException, RequestValidationException
     {
+        CQL3CasRequest request = makeCasRequest(queryState, options);
+
+        ColumnFamily result = StorageProxy.cas(keyspace(),
+                                               columnFamily(),
+                                               request.key,
+                                               request,
+                                               options.getSerialConsistency(),
+                                               options.getConsistency(),
+                                               queryState.getClientState());
+        return new ResultMessage.Rows(buildCasResultSet(request.key, result, options));
+    }
+
+    private CQL3CasRequest makeCasRequest(QueryState queryState, QueryOptions options)
+    {
         List<ByteBuffer> keys = buildPartitionKeyNames(options);
         // We don't support IN for CAS operation so far
         if (keys.size() > 1)
@@ -498,15 +516,7 @@ public abstract class ModificationStatement implements CQLStatement
         CQL3CasRequest request = new CQL3CasRequest(cfm, key, false);
         addConditions(prefix, request, options);
         request.addRowUpdate(prefix, this, options, now);
-
-        ColumnFamily result = StorageProxy.cas(keyspace(),
-                                               columnFamily(),
-                                               key,
-                                               request,
-                                               options.getSerialConsistency(),
-                                               options.getConsistency(),
-                                               queryState.getClientState());
-        return new ResultMessage.Rows(buildCasResultSet(key, result, options));
+        return request;
     }
 
     public void addConditions(Composite clusteringPrefix, CQL3CasRequest request, QueryOptions options) throws InvalidRequestException
@@ -608,9 +618,13 @@ public abstract class ModificationStatement implements CQLStatement
 
     public ResultMessage executeInternal(QueryState queryState, QueryOptions options) throws RequestValidationException, RequestExecutionException
     {
-        if (hasConditions())
-            throw new UnsupportedOperationException();
+        return hasConditions()
+               ? executeInternalWithCondition(queryState, options)
+               : executeInternalWithoutCondition(queryState, options);
+    }
 
+    public ResultMessage executeInternalWithoutCondition(QueryState queryState, QueryOptions options) throws RequestValidationException, RequestExecutionException
+    {
         for (IMutation mutation : getMutations(options, true, queryState.getTimestamp()))
         {
             assert mutation instanceof Mutation || mutation instanceof CounterMutation;
@@ -623,6 +637,40 @@ public abstract class ModificationStatement implements CQLStatement
         return null;
     }
 
+    public ResultMessage executeInternalWithCondition(QueryState state, QueryOptions options) throws RequestValidationException, RequestExecutionException
+    {
+        CQL3CasRequest request = makeCasRequest(state, options);
+        ColumnFamily result = casInternal(request, state);
+        return new ResultMessage.Rows(buildCasResultSet(request.key, result, options));
+    }
+
+    static ColumnFamily casInternal(CQL3CasRequest request, QueryState state)
+    {
+        long millis = state.getTimestamp() / 1000;
+        long nanos = ((state.getTimestamp() - (millis * 1000)) + 1) * 10;
+        UUID ballot = UUIDGen.getTimeUUID(millis, nanos);
+        CFMetaData metadata = Schema.instance.getCFMetaData(request.cfm.ksName, request.cfm.cfName);
+
+        ReadCommand readCommand = ReadCommand.create(request.cfm.ksName, request.key, request.cfm.cfName, request.now, request.readFilter());
+        Keyspace keyspace = Keyspace.open(request.cfm.ksName);
+
+        Row row = readCommand.getRow(keyspace);
+        ColumnFamily current = row.cf;
+        if (!request.appliesTo(current))
+        {
+            if (current == null)
+                current = ArrayBackedSortedColumns.factory.create(metadata);
+            return current;
+        }
+
+        ColumnFamily updates = request.makeUpdates(current);
+        updates = TriggerExecutor.instance.execute(request.key, updates);
+
+        Commit proposal = Commit.newProposal(request.key, ballot, updates);
+        proposal.makeMutation().apply();
+        return null;
+    }
+
     /**
      * Convert statement into a list of mutations to apply on the server
      *

http://git-wip-us.apache.org/repos/asf/cassandra/blob/01115f72/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
index 8ce555f..e2708cd 100644
--- a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
@@ -164,37 +164,13 @@ public class SelectStatement implements CQLStatement
         int limit = getLimit(options);
         long now = System.currentTimeMillis();
         Pageable command = getPageableCommand(options, limit, now);
-
-        int pageSize = options.getPageSize();
-
-        // An aggregation query will never be paged for the user, but we always page it internally to avoid OOM.
-        // If we user provided a pageSize we'll use that to page internally (because why not), otherwise we use our default
-        // Note that if there are some nodes in the cluster with a version less than 2.0, we can't use paging (CASSANDRA-6707).
-        if (selection.isAggregate() && pageSize <= 0)
-            pageSize = DEFAULT_COUNT_PAGE_SIZE;
+        int pageSize = getPageSize(options);
 
         if (pageSize <= 0 || command == null || !QueryPagers.mayNeedPaging(command, pageSize))
-        {
             return execute(command, options, limit, now, state);
-        }
 
         QueryPager pager = QueryPagers.pager(command, cl, state.getClientState(), options.getPagingState());
-
-        if (selection.isAggregate())
-            return pageAggregateQuery(pager, options, pageSize, now);
-
-        // We can't properly do post-query ordering if we page (see #6722)
-        checkFalse(needsPostQueryOrdering(),
-                  "Cannot page queries with both ORDER BY and a IN restriction on the partition key;"
-                  + " you must either remove the ORDER BY or the IN and sort client side, or disable paging for this query");
-
-        List<Row> page = pager.fetchPage(pageSize);
-        ResultMessage.Rows msg = processResults(page, options, limit, now);
-
-        if (!pager.isExhausted())
-            msg.result.metadata.setHasMorePages(pager.state());
-
-        return msg;
+        return execute(pager, options, limit, now, pageSize);
     }
 
     private Pageable getPageableCommand(QueryOptions options, int limit, long now) throws RequestValidationException
@@ -212,7 +188,21 @@ public class SelectStatement implements CQLStatement
         return getPageableCommand(options, getLimit(options), System.currentTimeMillis());
     }
 
-    private ResultMessage.Rows execute(Pageable command, QueryOptions options, int limit, long now, QueryState state) throws RequestValidationException, RequestExecutionException
+    private int getPageSize(QueryOptions options)
+    {
+        int pageSize = options.getPageSize();
+
+        // An aggregation query will never be paged for the user, but we always page it internally to avoid OOM.
+        // If we user provided a pageSize we'll use that to page internally (because why not), otherwise we use our default
+        // Note that if there are some nodes in the cluster with a version less than 2.0, we can't use paging (CASSANDRA-6707).
+        if (selection.isAggregate() && pageSize <= 0)
+            pageSize = DEFAULT_COUNT_PAGE_SIZE;
+
+        return  pageSize;
+    }
+
+    private ResultMessage.Rows execute(Pageable command, QueryOptions options, int limit, long now, QueryState state)
+    throws RequestValidationException, RequestExecutionException
     {
         List<Row> rows;
         if (command == null)
@@ -229,6 +219,26 @@ public class SelectStatement implements CQLStatement
         return processResults(rows, options, limit, now);
     }
 
+    private ResultMessage.Rows execute(QueryPager pager, QueryOptions options, int limit, long now, int pageSize)
+    throws RequestValidationException, RequestExecutionException
+    {
+        if (selection.isAggregate())
+            return pageAggregateQuery(pager, options, pageSize, now);
+
+        // We can't properly do post-query ordering if we page (see #6722)
+        checkFalse(needsPostQueryOrdering(),
+                   "Cannot page queries with both ORDER BY and a IN restriction on the partition key;"
+                   + " you must either remove the ORDER BY or the IN and sort client side, or disable paging for this query");
+
+        List<Row> page = pager.fetchPage(pageSize);
+        ResultMessage.Rows msg = processResults(page, options, limit, now);
+
+        if (!pager.isExhausted())
+            msg.result.metadata.setHasMorePages(pager.state());
+
+        return msg;
+    }
+
     private ResultMessage.Rows pageAggregateQuery(QueryPager pager, QueryOptions options, int pageSize, long now)
             throws RequestValidationException, RequestExecutionException
     {
@@ -267,13 +277,21 @@ public class SelectStatement implements CQLStatement
         int limit = getLimit(options);
         long now = System.currentTimeMillis();
         Pageable command = getPageableCommand(options, limit, now);
-        List<Row> rows = command == null
-                       ? Collections.<Row>emptyList()
-                       : (command instanceof Pageable.ReadCommands
-                          ? readLocally(keyspace(), ((Pageable.ReadCommands)command).commands)
-                          : ((RangeSliceCommand)command).executeLocally());
+        int pageSize = getPageSize(options);
 
-        return processResults(rows, options, limit, now);
+        if (pageSize <= 0 || command == null || !QueryPagers.mayNeedPaging(command, pageSize))
+        {
+            List<Row> rows = command == null
+                             ? Collections.<Row>emptyList()
+                             : (command instanceof Pageable.ReadCommands
+                                ? readLocally(keyspace(), ((Pageable.ReadCommands)command).commands)
+                                : ((RangeSliceCommand)command).executeLocally());
+
+            return processResults(rows, options, limit, now);
+        }
+
+        QueryPager pager = QueryPagers.localPager(command);
+        return execute(pager, options, limit, now, pageSize);
     }
 
     public ResultSet process(List<Row> rows) throws InvalidRequestException

http://git-wip-us.apache.org/repos/asf/cassandra/blob/01115f72/src/java/org/apache/cassandra/cql3/statements/TruncateStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/TruncateStatement.java b/src/java/org/apache/cassandra/cql3/statements/TruncateStatement.java
index 16c531c..9234a79 100644
--- a/src/java/org/apache/cassandra/cql3/statements/TruncateStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/TruncateStatement.java
@@ -22,6 +22,8 @@ import java.util.concurrent.TimeoutException;
 
 import org.apache.cassandra.auth.Permission;
 import org.apache.cassandra.cql3.*;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.exceptions.*;
 import org.apache.cassandra.transport.messages.ResultMessage;
 import org.apache.cassandra.service.ClientState;
@@ -71,6 +73,15 @@ public class TruncateStatement extends CFStatement implements CQLStatement
 
     public ResultMessage executeInternal(QueryState state, QueryOptions options)
     {
-        throw new UnsupportedOperationException();
+        try
+        {
+            ColumnFamilyStore cfs = Keyspace.open(keyspace()).getColumnFamilyStore(columnFamily());
+            cfs.truncateBlocking();
+        }
+        catch (Exception e)
+        {
+            throw new TruncateException(e);
+        }
+        return null;
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/01115f72/src/java/org/apache/cassandra/service/StorageProxy.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/StorageProxy.java b/src/java/org/apache/cassandra/service/StorageProxy.java
index 7801c3e..ac42eb0 100644
--- a/src/java/org/apache/cassandra/service/StorageProxy.java
+++ b/src/java/org/apache/cassandra/service/StorageProxy.java
@@ -228,7 +228,9 @@ public class StorageProxy implements StorageProxyMBean
                 Tracing.trace("Reading existing values for CAS precondition");
                 long timestamp = System.currentTimeMillis();
                 ReadCommand readCommand = ReadCommand.create(keyspaceName, key, cfName, timestamp, request.readFilter());
-                List<Row> rows = read(Arrays.asList(readCommand), consistencyForPaxos == ConsistencyLevel.LOCAL_SERIAL ? ConsistencyLevel.LOCAL_QUORUM : ConsistencyLevel.QUORUM);
+                List<Row> rows = read(Arrays.asList(readCommand), consistencyForPaxos == ConsistencyLevel.LOCAL_SERIAL
+                                                                  ? ConsistencyLevel.LOCAL_QUORUM
+                                                                  : ConsistencyLevel.QUORUM);
                 ColumnFamily current = rows.get(0).cf;
                 if (!request.appliesTo(current))
                 {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/01115f72/src/java/org/apache/cassandra/utils/UUIDGen.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/UUIDGen.java b/src/java/org/apache/cassandra/utils/UUIDGen.java
index 33f14a4..16190e2 100644
--- a/src/java/org/apache/cassandra/utils/UUIDGen.java
+++ b/src/java/org/apache/cassandra/utils/UUIDGen.java
@@ -82,10 +82,15 @@ public class UUIDGen
         return new UUID(createTime(fromUnixTimestamp(when)), clockSeqAndNode);
     }
 
+    public static UUID getTimeUUID(long when, long nanos)
+    {
+        return new UUID(createTime(fromUnixTimestamp(when, nanos)), clockSeqAndNode);
+    }
+
     @VisibleForTesting
-    public static UUID getTimeUUID(long when, long clockSeqAndNode)
+    public static UUID getTimeUUID(long when, long nanos, long clockSeqAndNode)
     {
-        return new UUID(createTime(fromUnixTimestamp(when)), clockSeqAndNode);
+        return new UUID(createTime(fromUnixTimestamp(when, nanos)), clockSeqAndNode);
     }
 
     /** creates a type 1 uuid from raw bytes. */
@@ -169,7 +174,12 @@ public class UUIDGen
      * @return
      */
     private static long fromUnixTimestamp(long timestamp) {
-        return (timestamp - START_EPOCH) * 10000;
+        return fromUnixTimestamp(timestamp, 0L);
+    }
+
+    private static long fromUnixTimestamp(long timestamp, long nanos)
+    {
+        return ((timestamp - START_EPOCH) * 10000) + nanos;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/cassandra/blob/01115f72/test/long/org/apache/cassandra/cql3/ManyRowsTest.java
----------------------------------------------------------------------
diff --git a/test/long/org/apache/cassandra/cql3/ManyRowsTest.java b/test/long/org/apache/cassandra/cql3/ManyRowsTest.java
new file mode 100644
index 0000000..82eeabd
--- /dev/null
+++ b/test/long/org/apache/cassandra/cql3/ManyRowsTest.java
@@ -0,0 +1,92 @@
+/*
+ * 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;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Random;
+import java.util.Set;
+
+import org.junit.Test;
+
+public class ManyRowsTest extends CQLTester
+{
+    /**
+     * Migrated from cql_tests.py:TestCQL.large_count_test()
+     */
+    @Test
+    public void testLargeCount() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int, v int, PRIMARY KEY (k))");
+
+        // We know we page at 10K, so test counting just before, at 10K, just after and
+        // a bit after that.
+        for (int k = 1; k < 10000; k++)
+            execute("INSERT INTO %s (k) VALUES (?)", k);
+
+        assertRows(execute("SELECT COUNT(*) FROM %s"), row(9999L));
+
+        execute("INSERT INTO %s (k) VALUES (?)", 10000);
+
+        assertRows(execute("SELECT COUNT(*) FROM %s"), row(10000L));
+
+        execute("INSERT INTO %s (k) VALUES (?)", 10001);
+
+        assertRows(execute("SELECT COUNT(*) FROM %s"), row(10001L));
+
+        for (int k = 10002; k < 15001; k++)
+            execute("INSERT INTO %s (k) VALUES (?)", k);
+
+        assertRows(execute("SELECT COUNT(*) FROM %s"), row(15000L));
+    }
+
+    /**
+     * Test for CASSANDRA-8410,
+     * migrated from cql_tests.py:TestCQL.large_clustering_in_test()
+     */
+    @Test
+    public void testLargeClustering() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int, c int, v int, PRIMARY KEY (k, c) )");
+
+        execute("INSERT INTO %s (k, c, v) VALUES (?, ?, ?)", 0, 0, 0);
+
+        // try to fetch one existing row and 9999 non-existing rows
+        List<Integer> inValues = new ArrayList(10000);
+        for (int i = 0; i < 10000; i++)
+            inValues.add(i);
+
+        assertRows(execute("SELECT * FROM %s WHERE k=? AND c IN ?", 0, inValues),
+                   row(0, 0, 0));
+
+        // insert approximately 1000 random rows between 0 and 10k
+        Random rnd = new Random();
+        Set<Integer> clusteringValues = new HashSet<>();
+        for (int i = 0; i < 1000; i++)
+            clusteringValues.add(rnd.nextInt(10000));
+
+        clusteringValues.add(0);
+
+        for (int i : clusteringValues) // TODO - this was done in parallel by dtests
+            execute("INSERT INTO %s (k, c, v) VALUES (?, ?, ?)", 0, i, i);
+
+        assertRowCount(execute("SELECT * FROM %s WHERE k=? AND c IN ?", 0, inValues), clusteringValues.size());
+    }
+}


[12/32] cassandra git commit: 2.2 commit for CASSANDRA-9160

Posted by jm...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/01115f72/test/unit/org/apache/cassandra/cql3/validation/operations/SelectOrderedPartitionerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/SelectOrderedPartitionerTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/SelectOrderedPartitionerTest.java
new file mode 100644
index 0000000..5152ba9
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/SelectOrderedPartitionerTest.java
@@ -0,0 +1,481 @@
+package org.apache.cassandra.cql3.validation.operations;
+
+import java.util.Arrays;
+
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import static junit.framework.Assert.assertNull;
+import static org.junit.Assert.assertEquals;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.dht.ByteOrderedPartitioner;
+
+/**
+ * SELECT statement tests that require a ByteOrderedPartitioner
+ */
+public class SelectOrderedPartitionerTest extends CQLTester
+{
+    @BeforeClass
+    public static void setUp()
+    {
+        DatabaseDescriptor.setPartitioner(ByteOrderedPartitioner.instance);
+    }
+
+    @Test
+    public void testTokenFunctionWithSingleColumnPartitionKey() throws Throwable
+    {
+        createTable("CREATE TABLE IF NOT EXISTS %s (a int PRIMARY KEY, b text)");
+        execute("INSERT INTO %s (a, b) VALUES (0, 'a')");
+
+        assertRows(execute("SELECT * FROM %s WHERE token(a) >= token(?)", 0), row(0, "a"));
+        assertRows(execute("SELECT * FROM %s WHERE token(a) >= token(?) and token(a) < token(?)", 0, 1), row(0, "a"));
+        assertInvalid("SELECT * FROM %s WHERE token(a) > token(?)", "a");
+        assertInvalidMessage("The token() function must contains only partition key components",
+                             "SELECT * FROM %s WHERE token(a, b) >= token(?, ?)", "b", 0);
+        assertInvalidMessage("More than one restriction was found for the start bound on a",
+                             "SELECT * FROM %s WHERE token(a) >= token(?) and token(a) >= token(?)", 0, 1);
+        assertInvalidMessage("Columns \"a\" cannot be restricted by both an equality and an inequality relation",
+                             "SELECT * FROM %s WHERE token(a) >= token(?) and token(a) = token(?)", 0, 1);
+        assertInvalidSyntax("SELECT * FROM %s WHERE token(a) = token(?) and token(a) IN (token(?))", 0, 1);
+
+        assertInvalidMessage("More than one restriction was found for the start bound on a",
+                             "SELECT * FROM %s WHERE token(a) > token(?) AND token(a) > token(?)", 1, 2);
+        assertInvalidMessage("More than one restriction was found for the end bound on a",
+                             "SELECT * FROM %s WHERE token(a) <= token(?) AND token(a) < token(?)", 1, 2);
+        assertInvalidMessage("Columns \"a\" cannot be restricted by both an equality and an inequality relation",
+                             "SELECT * FROM %s WHERE token(a) > token(?) AND token(a) = token(?)", 1, 2);
+        assertInvalidMessage("a cannot be restricted by more than one relation if it includes an Equal",
+                             "SELECT * FROM %s WHERE  token(a) = token(?) AND token(a) > token(?)", 1, 2);
+    }
+
+    @Test
+    public void testTokenFunctionWithPartitionKeyAndClusteringKeyArguments() throws Throwable
+    {
+        createTable("CREATE TABLE IF NOT EXISTS %s (a int, b text, PRIMARY KEY (a, b))");
+        assertInvalidMessage("The token() function must contains only partition key components",
+                             "SELECT * FROM %s WHERE token(a, b) > token(0, 'c')");
+    }
+
+    @Test
+    public void testTokenFunctionWithMultiColumnPartitionKey() throws Throwable
+    {
+        createTable("CREATE TABLE IF NOT EXISTS %s (a int, b text, PRIMARY KEY ((a, b)))");
+        execute("INSERT INTO %s (a, b) VALUES (0, 'a')");
+        execute("INSERT INTO %s (a, b) VALUES (0, 'b')");
+        execute("INSERT INTO %s (a, b) VALUES (0, 'c')");
+
+        assertRows(execute("SELECT * FROM %s WHERE token(a, b) > token(?, ?)", 0, "a"),
+                   row(0, "b"),
+                   row(0, "c"));
+        assertRows(execute("SELECT * FROM %s WHERE token(a, b) > token(?, ?) and token(a, b) < token(?, ?)",
+                           0, "a",
+                           0, "d"),
+                   row(0, "b"),
+                   row(0, "c"));
+        assertInvalidMessage("The token() function must be applied to all partition key components or none of them",
+                             "SELECT * FROM %s WHERE token(a) > token(?) and token(b) > token(?)", 0, "a");
+        assertInvalidMessage("The token() function must be applied to all partition key components or none of them",
+                             "SELECT * FROM %s WHERE token(a) > token(?, ?) and token(a) < token(?, ?) and token(b) > token(?, ?) ",
+                             0, "a", 0, "d", 0, "a");
+        assertInvalidMessage("The token function arguments must be in the partition key order: a, b",
+                             "SELECT * FROM %s WHERE token(b, a) > token(0, 'c')");
+        assertInvalidMessage("The token() function must be applied to all partition key components or none of them",
+                             "SELECT * FROM %s WHERE token(a, b) > token(?, ?) and token(b) < token(?, ?)", 0, "a", 0, "a");
+        assertInvalidMessage("The token() function must be applied to all partition key components or none of them",
+                             "SELECT * FROM %s WHERE token(a) > token(?, ?) and token(b) > token(?, ?)", 0, "a", 0, "a");
+    }
+
+    @Test
+    public void testSingleColumnPartitionKeyWithTokenNonTokenRestrictionsMix() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int primary key, b int)");
+
+        execute("INSERT INTO %s (a, b) VALUES (0, 0);");
+        execute("INSERT INTO %s (a, b) VALUES (1, 1);");
+        execute("INSERT INTO %s (a, b) VALUES (2, 2);");
+        execute("INSERT INTO %s (a, b) VALUES (3, 3);");
+        execute("INSERT INTO %s (a, b) VALUES (4, 4);");
+        assertRows(execute("SELECT * FROM %s WHERE a IN (?, ?);", 1, 3),
+                   row(1, 1),
+                   row(3, 3));
+        assertRows(execute("SELECT * FROM %s WHERE token(a)> token(?) and token(a) <= token(?);", 1, 3),
+                   row(2, 2),
+                   row(3, 3));
+        assertRows(execute("SELECT * FROM %s WHERE token(a)= token(2);"),
+                   row(2, 2));
+        assertRows(execute("SELECT * FROM %s WHERE token(a) > token(?) AND token(a) <= token(?) AND a IN (?, ?);",
+                           1, 3, 1, 3),
+                   row(3, 3));
+        assertRows(execute("SELECT * FROM %s WHERE token(a) < token(?) AND token(a) >= token(?) AND a IN (?, ?);",
+                           1, 3, 1, 3),
+                   row(3, 3));
+        assertInvalidMessage("Only EQ and IN relation are supported on the partition key (unless you use the token() function)",
+                             "SELECT * FROM %s WHERE token(a) > token(?) AND token(a) <= token(?) AND a > ?;", 1, 3, 1);
+
+        assertRows(execute("SELECT * FROM %s WHERE token(a) > token(?) AND token(a) <= token(?) AND a IN ?;",
+                           1, 3, Arrays.asList(1, 3)),
+                   row(3, 3));
+        assertRows(execute("SELECT * FROM %s WHERE token(a) > token(?) AND a = ?;", 1, 3),
+                   row(3, 3));
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND token(a) > token(?);", 3, 1),
+                   row(3, 3));
+        assertEmpty(execute("SELECT * FROM %s WHERE token(a) > token(?) AND a = ?;", 3, 1));
+        assertEmpty(execute("SELECT * FROM %s WHERE a = ? AND token(a) > token(?);", 1, 3));
+        assertRows(execute("SELECT * FROM %s WHERE token(a) > token(?) AND a IN (?, ?);", 2, 1, 3),
+                   row(3, 3));
+        assertRows(execute("SELECT * FROM %s WHERE token(a) > token(?) AND token(a) < token(?) AND a IN (?, ?) ;", 2, 5, 1, 3),
+                   row(3, 3));
+        assertRows(execute("SELECT * FROM %s WHERE a IN (?, ?) AND token(a) > token(?) AND token(a) < token(?);", 1, 3, 2, 5),
+                   row(3, 3));
+        assertRows(execute("SELECT * FROM %s WHERE token(a) > token(?) AND a IN (?, ?) AND token(a) < token(?);", 2, 1, 3, 5),
+                   row(3, 3));
+        assertEmpty(execute("SELECT * FROM %s WHERE a IN (?, ?) AND token(a) > token(?);", 1, 3, 3));
+        assertRows(execute("SELECT * FROM %s WHERE token(a) <= token(?) AND a = ?;", 2, 2),
+                   row(2, 2));
+        assertEmpty(execute("SELECT * FROM %s WHERE token(a) <= token(?) AND a = ?;", 2, 3));
+        assertEmpty(execute("SELECT * FROM %s WHERE token(a) = token(?) AND a = ?;", 2, 3));
+        assertRows(execute("SELECT * FROM %s WHERE token(a) >= token(?) AND token(a) <= token(?) AND a = ?;", 2, 2, 2),
+                   row(2, 2));
+        assertEmpty(execute("SELECT * FROM %s WHERE token(a) >= token(?) AND token(a) < token(?) AND a = ?;", 2, 2, 2));
+        assertEmpty(execute("SELECT * FROM %s WHERE token(a) > token(?) AND token(a) <= token(?) AND a = ?;", 2, 2, 2));
+        assertEmpty(execute("SELECT * FROM %s WHERE token(a) > token(?) AND token(a) < token(?) AND a = ?;", 2, 2, 2));
+    }
+
+    @Test
+    public void testMultiColumnPartitionKeyWithTokenNonTokenRestrictionsMix() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int, b int, c int, primary key((a, b)))");
+
+        execute("INSERT INTO %s (a, b, c) VALUES (0, 0, 0);");
+        execute("INSERT INTO %s (a, b, c) VALUES (0, 1, 1);");
+        execute("INSERT INTO %s (a, b, c) VALUES (0, 2, 2);");
+        execute("INSERT INTO %s (a, b, c) VALUES (1, 0, 3);");
+        execute("INSERT INTO %s (a, b, c) VALUES (1, 1, 4);");
+
+        assertRows(execute("SELECT * FROM %s WHERE token(a, b) > token(?, ?);", 0, 0),
+                   row(0, 1, 1),
+                   row(0, 2, 2),
+                   row(1, 0, 3),
+                   row(1, 1, 4));
+
+        assertRows(execute("SELECT * FROM %s WHERE token(a, b) > token(?, ?) AND a = ? AND b IN (?, ?);",
+                           0, 0, 1, 0, 1),
+                   row(1, 0, 3),
+                   row(1, 1, 4));
+
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND token(a, b) > token(?, ?) AND b IN (?, ?);",
+                           1, 0, 0, 0, 1),
+                   row(1, 0, 3),
+                   row(1, 1, 4));
+
+        assertRows(execute("SELECT * FROM %s WHERE b IN (?, ?) AND token(a, b) > token(?, ?) AND a = ?;",
+                           0, 1, 0, 0, 1),
+                   row(1, 0, 3),
+                   row(1, 1, 4));
+
+        assertEmpty(execute("SELECT * FROM %s WHERE b IN (?, ?) AND token(a, b) > token(?, ?) AND token(a, b) < token(?, ?) AND a = ?;",
+                            0, 1, 0, 0, 0, 0, 1));
+
+        assertEmpty(execute("SELECT * FROM %s WHERE b IN (?, ?) AND token(a, b) > token(?, ?) AND token(a, b) <= token(?, ?) AND a = ?;",
+                            0, 1, 0, 0, 0, 0, 1));
+
+        assertEmpty(execute("SELECT * FROM %s WHERE b IN (?, ?) AND token(a, b) >= token(?, ?) AND token(a, b) < token(?, ?) AND a = ?;",
+                            0, 1, 0, 0, 0, 0, 1));
+
+        assertEmpty(execute("SELECT * FROM %s WHERE b IN (?, ?) AND token(a, b) = token(?, ?) AND a = ?;",
+                            0, 1, 0, 0, 1));
+
+        assertInvalidMessage("Partition key parts: b must be restricted as other parts are",
+                             "SELECT * FROM %s WHERE token(a, b) > token(?, ?) AND a = ?;", 0, 0, 1);
+    }
+
+    @Test
+    public void testMultiColumnPartitionKeyWithIndexAndTokenNonTokenRestrictionsMix() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int, b int, c int, primary key((a, b)))");
+        createIndex("CREATE INDEX ON %s(b)");
+        createIndex("CREATE INDEX ON %s(c)");
+
+        execute("INSERT INTO %s (a, b, c) VALUES (0, 0, 0);");
+        execute("INSERT INTO %s (a, b, c) VALUES (0, 1, 1);");
+        execute("INSERT INTO %s (a, b, c) VALUES (0, 2, 2);");
+        execute("INSERT INTO %s (a, b, c) VALUES (1, 0, 3);");
+        execute("INSERT INTO %s (a, b, c) VALUES (1, 1, 4);");
+
+        assertRows(execute("SELECT * FROM %s WHERE b = ?;", 1),
+                   row(0, 1, 1),
+                   row(1, 1, 4));
+
+        assertRows(execute("SELECT * FROM %s WHERE token(a, b) > token(?, ?) AND b = ?;", 0, 0, 1),
+                   row(0, 1, 1),
+                   row(1, 1, 4));
+
+        assertRows(execute("SELECT * FROM %s WHERE b = ? AND token(a, b) > token(?, ?);", 1, 0, 0),
+                   row(0, 1, 1),
+                   row(1, 1, 4));
+
+        assertRows(execute("SELECT * FROM %s WHERE b = ? AND token(a, b) > token(?, ?) and c = ? ALLOW FILTERING;", 1, 0, 0, 4),
+                   row(1, 1, 4));
+    }
+
+    @Test
+    public void testTokenFunctionWithCompoundPartitionAndClusteringCols() throws Throwable
+    {
+        createTable("CREATE TABLE IF NOT EXISTS %s (a int, b int, c int, d int, PRIMARY KEY ((a, b), c, d))");
+        // just test that the queries don't error
+        execute("SELECT * FROM %s WHERE token(a, b) > token(0, 0) AND c > 10 ALLOW FILTERING;");
+        execute("SELECT * FROM %s WHERE c > 10 AND token(a, b) > token(0, 0) ALLOW FILTERING;");
+        execute("SELECT * FROM %s WHERE token(a, b) > token(0, 0) AND (c, d) > (0, 0) ALLOW FILTERING;");
+        execute("SELECT * FROM %s WHERE (c, d) > (0, 0) AND token(a, b) > token(0, 0) ALLOW FILTERING;");
+    }
+
+    /**
+     * Test undefined columns
+     * migrated from cql_tests.py:TestCQL.undefined_column_handling_test()
+     */
+    @Test
+    public void testUndefinedColumns() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, v1 int, v2 int,)");
+
+        execute("INSERT INTO %s (k, v1, v2) VALUES (0, 0, 0)");
+        execute("INSERT INTO %s (k, v1) VALUES (1, 1)");
+        execute("INSERT INTO %s (k, v1, v2) VALUES (2, 2, 2)");
+
+        Object[][] rows = getRows(execute("SELECT v2 FROM %s"));
+        assertEquals(0, rows[0][0]);
+        assertEquals(null, rows[1][0]);
+        assertEquals(2, rows[2][0]);
+
+        rows = getRows(execute("SELECT v2 FROM %s WHERE k = 1"));
+        assertEquals(1, rows.length);
+        assertNull(rows[0][0]);
+    }
+
+    /**
+     * Check table with only a PK (#4361),
+     * migrated from cql_tests.py:TestCQL.only_pk_test()
+     */
+    @Test
+    public void testPrimaryKeyOnly() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int, c int, PRIMARY KEY (k, c))");
+
+        for (int k = 0; k < 2; k++)
+            for (int c = 0; c < 2; c++)
+                execute("INSERT INTO %s (k, c) VALUES (?, ?)", k, c);
+
+        assertRows(execute("SELECT * FROM %s"),
+                   row(0, 0),
+                   row(0, 1),
+                   row(1, 0),
+                   row(1, 1));
+
+        // Check for dense tables too
+        createTable(" CREATE TABLE %s (k int, c int, PRIMARY KEY (k, c)) WITH COMPACT STORAGE");
+
+        for (int k = 0; k < 2; k++)
+            for (int c = 0; c < 2; c++)
+                execute("INSERT INTO %s (k, c) VALUES (?, ?)", k, c);
+
+        assertRows(execute("SELECT * FROM %s"),
+                   row(0, 0),
+                   row(0, 1),
+                   row(1, 0),
+                   row(1, 1));
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.composite_index_with_pk_test()
+     */
+    @Test
+    public void testCompositeIndexWithPK() throws Throwable
+    {
+        createTable("CREATE TABLE %s (blog_id int, time1 int, time2 int, author text, content text, PRIMARY KEY (blog_id, time1, time2))");
+
+        createIndex("CREATE INDEX ON %s(author)");
+
+        execute("INSERT INTO %s (blog_id, time1, time2, author, content) VALUES (?, ?, ?, ?, ?)", 1, 0, 0, "foo", "bar1");
+        execute("INSERT INTO %s (blog_id, time1, time2, author, content) VALUES (?, ?, ?, ?, ?)", 1, 0, 1, "foo", "bar2");
+        execute("INSERT INTO %s (blog_id, time1, time2, author, content) VALUES (?, ?, ?, ?, ?)", 2, 1, 0, "foo", "baz");
+        execute("INSERT INTO %s (blog_id, time1, time2, author, content) VALUES (?, ?, ?, ?, ?)", 3, 0, 1, "gux", "qux");
+
+        assertRows(execute("SELECT blog_id, content FROM %s WHERE author='foo'"),
+                   row(1, "bar1"),
+                   row(1, "bar2"),
+                   row(2, "baz"));
+
+        assertRows(execute("SELECT blog_id, content FROM %s WHERE time1 > 0 AND author='foo' ALLOW FILTERING"),
+                   row(2, "baz"));
+
+        assertRows(execute("SELECT blog_id, content FROM %s WHERE time1 = 1 AND author='foo' ALLOW FILTERING"),
+                   row(2, "baz"));
+
+        assertRows(execute("SELECT blog_id, content FROM %s WHERE time1 = 1 AND time2 = 0 AND author='foo' ALLOW FILTERING"),
+                   row(2, "baz"));
+
+        assertEmpty(execute("SELECT content FROM %s WHERE time1 = 1 AND time2 = 1 AND author='foo' ALLOW FILTERING"));
+
+        assertEmpty(execute("SELECT content FROM %s WHERE time1 = 1 AND time2 > 0 AND author='foo' ALLOW FILTERING"));
+
+        assertInvalid("SELECT content FROM %s WHERE time2 >= 0 AND author='foo'");
+
+        assertInvalid("SELECT blog_id, content FROM %s WHERE time1 > 0 AND author='foo'");
+        assertInvalid("SELECT blog_id, content FROM %s WHERE time1 = 1 AND author='foo'");
+        assertInvalid("SELECT blog_id, content FROM %s WHERE time1 = 1 AND time2 = 0 AND author='foo'");
+        assertInvalid("SELECT content FROM %s WHERE time1 = 1 AND time2 = 1 AND author='foo'");
+        assertInvalid("SELECT content FROM %s WHERE time1 = 1 AND time2 > 0 AND author='foo'");
+    }
+
+    /**
+     * Test for LIMIT bugs from 4579,
+     * migrated from cql_tests.py:TestCQL.limit_bugs_test()
+     */
+    @Test
+    public void testLimitBug() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int, b int, c int, d int, e int, PRIMARY KEY (a, b))");
+
+        execute("INSERT INTO %s (a, b, c, d, e) VALUES (1, 1, 1, 1, 1);");
+        execute("INSERT INTO %s (a, b, c, d, e) VALUES (2, 2, 2, 2, 2);");
+        execute("INSERT INTO %s (a, b, c, d, e) VALUES (3, 3, 3, 3, 3);");
+        execute("INSERT INTO %s (a, b, c, d, e) VALUES (4, 4, 4, 4, 4);");
+
+        assertRows(execute("SELECT * FROM %s"),
+                   row(1, 1, 1, 1, 1),
+                   row(2, 2, 2, 2, 2),
+                   row(3, 3, 3, 3, 3),
+                   row(4, 4, 4, 4, 4));
+
+        assertRows(execute("SELECT * FROM %s LIMIT 1"),
+                   row(1, 1, 1, 1, 1));
+
+        assertRows(execute("SELECT * FROM %s LIMIT 2"),
+                   row(1, 1, 1, 1, 1),
+                   row(2, 2, 2, 2, 2));
+
+        createTable("CREATE TABLE %s (a int primary key, b int, c int,)");
+
+        execute("INSERT INTO %s (a, b, c) VALUES (1, 1, 1)");
+        execute("INSERT INTO %s (a, b, c) VALUES (2, 2, 2)");
+        execute("INSERT INTO %s (a, b, c) VALUES (3, 3, 3)");
+        execute("INSERT INTO %s (a, b, c) VALUES (4, 4, 4)");
+
+        assertRows(execute("SELECT * FROM %s"),
+                   row(1, 1, 1),
+                   row(2, 2, 2),
+                   row(3, 3, 3),
+                   row(4, 4, 4));
+
+        assertRows(execute("SELECT * FROM %s LIMIT 1"),
+                   row(1, 1, 1));
+
+        assertRows(execute("SELECT * FROM %s LIMIT 2"),
+                   row(1, 1, 1),
+                   row(2, 2, 2));
+
+        assertRows(execute("SELECT * FROM %s LIMIT 3"),
+                   row(1, 1, 1),
+                   row(2, 2, 2),
+                   row(3, 3, 3));
+
+        assertRows(execute("SELECT * FROM %s LIMIT 4"),
+                   row(1, 1, 1),
+                   row(2, 2, 2),
+                   row(3, 3, 3),
+                   row(4, 4, 4));
+
+        assertRows(execute("SELECT * FROM %s LIMIT 5"),
+                   row(1, 1, 1),
+                   row(2, 2, 2),
+                   row(3, 3, 3),
+                   row(4, 4, 4));
+    }
+
+    /**
+     * Test for #4612 bug and more generally order by when multiple C* rows are queried
+     * migrated from cql_tests.py:TestCQL.order_by_multikey_test()
+     */
+    @Test
+    public void testOrderByMultikey() throws Throwable
+    {
+        createTable("CREATE TABLE %s (my_id varchar, col1 int, col2 int, value varchar, PRIMARY KEY (my_id, col1, col2))");
+
+        execute("INSERT INTO %s (my_id, col1, col2, value) VALUES ( 'key1', 1, 1, 'a');");
+        execute("INSERT INTO %s (my_id, col1, col2, value) VALUES ( 'key2', 3, 3, 'a');");
+        execute("INSERT INTO %s (my_id, col1, col2, value) VALUES ( 'key3', 2, 2, 'b');");
+        execute("INSERT INTO %s (my_id, col1, col2, value) VALUES ( 'key4', 2, 1, 'b');");
+
+        assertRows(execute("SELECT col1 FROM %s WHERE my_id in('key1', 'key2', 'key3') ORDER BY col1"),
+                   row(1), row(2), row(3));
+
+        assertRows(execute("SELECT col1, value, my_id, col2 FROM %s WHERE my_id in('key3', 'key4') ORDER BY col1, col2"),
+                   row(2, "b", "key4", 1), row(2, "b", "key3", 2));
+
+        assertInvalid("SELECT col1 FROM %s ORDER BY col1");
+        assertInvalid("SELECT col1 FROM %s WHERE my_id > 'key1' ORDER BY col1");
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.composite_index_collections_test()
+     */
+    @Test
+    public void testIndexOnCompositeWithCollections() throws Throwable
+    {
+        createTable("CREATE TABLE %s (blog_id int, time1 int, time2 int, author text, content set<text>, PRIMARY KEY (blog_id, time1, time2))");
+
+        createIndex("CREATE INDEX ON %s (author)");
+
+        execute("INSERT INTO %s (blog_id, time1, time2, author, content) VALUES (?, ?, ?, ?, { 'bar1', 'bar2' })", 1, 0, 0, "foo");
+        execute("INSERT INTO %s (blog_id, time1, time2, author, content) VALUES (?, ?, ?, ?, { 'bar2', 'bar3' })", 1, 0, 1, "foo");
+        execute("INSERT INTO %s (blog_id, time1, time2, author, content) VALUES (?, ?, ?, ?, { 'baz' })", 2, 1, 0, "foo");
+        execute("INSERT INTO %s (blog_id, time1, time2, author, content) VALUES (?, ?, ?, ?, { 'qux' })", 3, 0, 1, "gux");
+
+        assertRows(execute("SELECT blog_id, content FROM %s WHERE author='foo'"),
+                   row(1, set("bar1", "bar2")),
+                   row(1, set("bar2", "bar3")),
+                   row(2, set("baz")));
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.truncate_clean_cache_test()
+     */
+    @Test
+    public void testTruncateWithCaching() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, v1 int, v2 int,) WITH CACHING = ALL;");
+
+        for (int i = 0; i < 3; i++)
+            execute("INSERT INTO %s (k, v1, v2) VALUES (?, ?, ?)", i, i, i * 2);
+
+        assertRows(execute("SELECT v1, v2 FROM %s WHERE k IN (0, 1, 2)"),
+                   row(0, 0),
+                   row(1, 2),
+                   row(2, 4));
+
+        execute("TRUNCATE %s");
+
+        assertEmpty(execute("SELECT v1, v2 FROM %s WHERE k IN (0, 1, 2)"));
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.range_key_ordered_test()
+     */
+    @Test
+    public void testRangeKey() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY)");
+
+        execute("INSERT INTO %s (k) VALUES (-1)");
+        execute("INSERT INTO %s (k) VALUES ( 0)");
+        execute("INSERT INTO %s (k) VALUES ( 1)");
+
+        assertRows(execute("SELECT * FROM %s"),
+                   row(0),
+                   row(1),
+                   row(-1));
+
+        assertInvalid("SELECT * FROM %s WHERE k >= -1 AND k < 1");
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/01115f72/test/unit/org/apache/cassandra/cql3/validation/operations/SelectSingleColumnRelationTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/SelectSingleColumnRelationTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/SelectSingleColumnRelationTest.java
new file mode 100644
index 0000000..fd43a51
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/SelectSingleColumnRelationTest.java
@@ -0,0 +1,555 @@
+/*
+ * 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.validation.operations;
+
+import java.util.Arrays;
+
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.cassandra.cql3.CQLTester;
+
+public class SelectSingleColumnRelationTest extends CQLTester
+{
+    @Test
+    public void testInvalidCollectionEqualityRelation() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int PRIMARY KEY, b set<int>, c list<int>, d map<int, int>)");
+        createIndex("CREATE INDEX ON %s (b)");
+        createIndex("CREATE INDEX ON %s (c)");
+        createIndex("CREATE INDEX ON %s (d)");
+
+        assertInvalidMessage("Collection column 'b' (set<int>) cannot be restricted by a '=' relation",
+                             "SELECT * FROM %s WHERE a = 0 AND b=?", set(0));
+        assertInvalidMessage("Collection column 'c' (list<int>) cannot be restricted by a '=' relation",
+                             "SELECT * FROM %s WHERE a = 0 AND c=?", list(0));
+        assertInvalidMessage("Collection column 'd' (map<int, int>) cannot be restricted by a '=' relation",
+                             "SELECT * FROM %s WHERE a = 0 AND d=?", map(0, 0));
+    }
+
+    @Test
+    public void testInvalidCollectionNonEQRelation() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int PRIMARY KEY, b set<int>, c int)");
+        createIndex("CREATE INDEX ON %s (c)");
+        execute("INSERT INTO %s (a, b, c) VALUES (0, {0}, 0)");
+
+        // non-EQ operators
+        assertInvalidMessage("Collection column 'b' (set<int>) cannot be restricted by a '>' relation",
+                             "SELECT * FROM %s WHERE c = 0 AND b > ?", set(0));
+        assertInvalidMessage("Collection column 'b' (set<int>) cannot be restricted by a '>=' relation",
+                             "SELECT * FROM %s WHERE c = 0 AND b >= ?", set(0));
+        assertInvalidMessage("Collection column 'b' (set<int>) cannot be restricted by a '<' relation",
+                             "SELECT * FROM %s WHERE c = 0 AND b < ?", set(0));
+        assertInvalidMessage("Collection column 'b' (set<int>) cannot be restricted by a '<=' relation",
+                             "SELECT * FROM %s WHERE c = 0 AND b <= ?", set(0));
+        assertInvalidMessage("Collection column 'b' (set<int>) cannot be restricted by a 'IN' relation",
+                             "SELECT * FROM %s WHERE c = 0 AND b IN (?)", set(0));
+    }
+
+    @Test
+    public void testClusteringColumnRelations() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a text, b int, c int, d int, primary key(a, b, c))");
+        execute("insert into %s (a, b, c, d) values (?, ?, ?, ?)", "first", 1, 5, 1);
+        execute("insert into %s (a, b, c, d) values (?, ?, ?, ?)", "first", 2, 6, 2);
+        execute("insert into %s (a, b, c, d) values (?, ?, ?, ?)", "first", 3, 7, 3);
+        execute("insert into %s (a, b, c, d) values (?, ?, ?, ?)", "second", 4, 8, 4);
+
+        testSelectQueriesWithClusteringColumnRelations();
+    }
+
+    @Test
+    public void testClusteringColumnRelationsWithCompactStorage() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a text, b int, c int, d int, primary key(a, b, c)) WITH COMPACT STORAGE;");
+        execute("insert into %s (a, b, c, d) values (?, ?, ?, ?)", "first", 1, 5, 1);
+        execute("insert into %s (a, b, c, d) values (?, ?, ?, ?)", "first", 2, 6, 2);
+        execute("insert into %s (a, b, c, d) values (?, ?, ?, ?)", "first", 3, 7, 3);
+        execute("insert into %s (a, b, c, d) values (?, ?, ?, ?)", "second", 4, 8, 4);
+
+        testSelectQueriesWithClusteringColumnRelations();
+    }
+
+    private void testSelectQueriesWithClusteringColumnRelations() throws Throwable
+    {
+        assertRows(execute("select * from %s where a in (?, ?)", "first", "second"),
+                   row("first", 1, 5, 1),
+                   row("first", 2, 6, 2),
+                   row("first", 3, 7, 3),
+                   row("second", 4, 8, 4));
+
+        assertRows(execute("select * from %s where a = ? and b = ? and c in (?, ?)", "first", 2, 6, 7),
+                   row("first", 2, 6, 2));
+
+        assertRows(execute("select * from %s where a = ? and b in (?, ?) and c in (?, ?)", "first", 2, 3, 6, 7),
+                   row("first", 2, 6, 2),
+                   row("first", 3, 7, 3));
+
+        assertRows(execute("select * from %s where a = ? and b in (?, ?) and c in (?, ?)", "first", 3, 2, 7, 6),
+                   row("first", 2, 6, 2),
+                   row("first", 3, 7, 3));
+
+        assertRows(execute("select * from %s where a = ? and c in (?, ?) and b in (?, ?)", "first", 7, 6, 3, 2),
+                   row("first", 2, 6, 2),
+                   row("first", 3, 7, 3));
+
+        assertRows(execute("select c, d from %s where a = ? and c in (?, ?) and b in (?, ?)", "first", 7, 6, 3, 2),
+                   row(6, 2),
+                   row(7, 3));
+
+        assertRows(execute("select c, d from %s where a = ? and c in (?, ?) and b in (?, ?, ?)", "first", 7, 6, 3, 2, 3),
+                   row(6, 2),
+                   row(7, 3));
+
+        assertRows(execute("select * from %s where a = ? and b in (?, ?) and c = ?", "first", 3, 2, 7),
+                   row("first", 3, 7, 3));
+
+        assertRows(execute("select * from %s where a = ? and b in ? and c in ?",
+                           "first", Arrays.asList(3, 2), Arrays.asList(7, 6)),
+                   row("first", 2, 6, 2),
+                   row("first", 3, 7, 3));
+
+        assertInvalidMessage("Invalid null value for column b",
+                             "select * from %s where a = ? and b in ? and c in ?", "first", null, Arrays.asList(7, 6));
+
+        assertRows(execute("select * from %s where a = ? and c >= ? and b in (?, ?)", "first", 6, 3, 2),
+                   row("first", 2, 6, 2),
+                   row("first", 3, 7, 3));
+
+        assertRows(execute("select * from %s where a = ? and c > ? and b in (?, ?)", "first", 6, 3, 2),
+                   row("first", 3, 7, 3));
+
+        assertRows(execute("select * from %s where a = ? and c <= ? and b in (?, ?)", "first", 6, 3, 2),
+                   row("first", 2, 6, 2));
+
+        assertRows(execute("select * from %s where a = ? and c < ? and b in (?, ?)", "first", 7, 3, 2),
+                   row("first", 2, 6, 2));
+//---
+        assertRows(execute("select * from %s where a = ? and c >= ? and c <= ? and b in (?, ?)", "first", 6, 7, 3, 2),
+                   row("first", 2, 6, 2),
+                   row("first", 3, 7, 3));
+
+        assertRows(execute("select * from %s where a = ? and c > ? and c <= ? and b in (?, ?)", "first", 6, 7, 3, 2),
+                   row("first", 3, 7, 3));
+
+        assertEmpty(execute("select * from %s where a = ? and c > ? and c < ? and b in (?, ?)", "first", 6, 7, 3, 2));
+
+        assertInvalidMessage("Column \"c\" cannot be restricted by both an equality and an inequality relation",
+                             "select * from %s where a = ? and c > ? and c = ? and b in (?, ?)", "first", 6, 7, 3, 2);
+
+        assertInvalidMessage("c cannot be restricted by more than one relation if it includes an Equal",
+                             "select * from %s where a = ? and c = ? and c > ?  and b in (?, ?)", "first", 6, 7, 3, 2);
+
+        assertRows(execute("select * from %s where a = ? and c in (?, ?) and b in (?, ?) order by b DESC",
+                           "first", 7, 6, 3, 2),
+                   row("first", 3, 7, 3),
+                   row("first", 2, 6, 2));
+
+        assertInvalidMessage("More than one restriction was found for the start bound on b",
+                             "select * from %s where a = ? and b > ? and b > ?", "first", 6, 3, 2);
+
+        assertInvalidMessage("More than one restriction was found for the end bound on b",
+                             "select * from %s where a = ? and b < ? and b <= ?", "first", 6, 3, 2);
+    }
+
+    @Test
+    public void testPartitionKeyColumnRelations() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a text, b int, c int, d int, primary key((a, b), c))");
+        execute("insert into %s (a, b, c, d) values (?, ?, ?, ?)", "first", 1, 1, 1);
+        execute("insert into %s (a, b, c, d) values (?, ?, ?, ?)", "first", 2, 2, 2);
+        execute("insert into %s (a, b, c, d) values (?, ?, ?, ?)", "first", 3, 3, 3);
+        execute("insert into %s (a, b, c, d) values (?, ?, ?, ?)", "first", 4, 4, 4);
+        execute("insert into %s (a, b, c, d) values (?, ?, ?, ?)", "second", 1, 1, 1);
+        execute("insert into %s (a, b, c, d) values (?, ?, ?, ?)", "second", 4, 4, 4);
+
+        assertRows(execute("select * from %s where a = ? and b = ?", "first", 2),
+                   row("first", 2, 2, 2));
+
+        assertRows(execute("select * from %s where a in (?, ?) and b in (?, ?)", "first", "second", 2, 3),
+                   row("first", 2, 2, 2),
+                   row("first", 3, 3, 3));
+
+        assertRows(execute("select * from %s where a in (?, ?) and b = ?", "first", "second", 4),
+                   row("first", 4, 4, 4),
+                   row("second", 4, 4, 4));
+
+        assertRows(execute("select * from %s where a = ? and b in (?, ?)", "first", 3, 4),
+                   row("first", 3, 3, 3),
+                   row("first", 4, 4, 4));
+
+        assertRows(execute("select * from %s where a in (?, ?) and b in (?, ?)", "first", "second", 1, 4),
+                   row("first", 1, 1, 1),
+                   row("first", 4, 4, 4),
+                   row("second", 1, 1, 1),
+                   row("second", 4, 4, 4));
+
+        assertInvalidMessage("Partition key parts: b must be restricted as other parts are",
+                             "select * from %s where a in (?, ?)", "first", "second");
+        assertInvalidMessage("Partition key parts: b must be restricted as other parts are",
+                             "select * from %s where a = ?", "first");
+        assertInvalidMessage("b cannot be restricted by more than one relation if it includes a IN",
+                             "select * from %s where a = ? AND b IN (?, ?) AND b = ?", "first", 2, 2, 3);
+        assertInvalidMessage("b cannot be restricted by more than one relation if it includes an Equal",
+                             "select * from %s where a = ? AND b = ? AND b IN (?, ?)", "first", 2, 2, 3);
+        assertInvalidMessage("a cannot be restricted by more than one relation if it includes a IN",
+                             "select * from %s where a IN (?, ?) AND a = ? AND b = ?", "first", "second", "first", 3);
+        assertInvalidMessage("a cannot be restricted by more than one relation if it includes an Equal",
+                             "select * from %s where a = ? AND a IN (?, ?) AND b IN (?, ?)", "first", "second", "first", 2, 3);
+    }
+
+    @Test
+    public void testClusteringColumnRelationsWithClusteringOrder() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a text, b int, c int, d int, primary key(a, b, c)) WITH CLUSTERING ORDER BY (b DESC);");
+        execute("insert into %s (a, b, c, d) values (?, ?, ?, ?)", "first", 1, 5, 1);
+        execute("insert into %s (a, b, c, d) values (?, ?, ?, ?)", "first", 2, 6, 2);
+        execute("insert into %s (a, b, c, d) values (?, ?, ?, ?)", "first", 3, 7, 3);
+        execute("insert into %s (a, b, c, d) values (?, ?, ?, ?)", "second", 4, 8, 4);
+
+        assertRows(execute("select * from %s where a = ? and c in (?, ?) and b in (?, ?) order by b DESC",
+                           "first", 7, 6, 3, 2),
+                   row("first", 3, 7, 3),
+                   row("first", 2, 6, 2));
+
+        assertRows(execute("select * from %s where a = ? and c in (?, ?) and b in (?, ?) order by b ASC",
+                           "first", 7, 6, 3, 2),
+                   row("first", 2, 6, 2),
+                   row("first", 3, 7, 3));
+    }
+
+    @Test
+    public void testAllowFilteringWithClusteringColumn() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int, c int, v int, PRIMARY KEY (k, c))");
+
+        execute("INSERT INTO %s (k, c, v) VALUES(?, ?, ?)", 1, 2, 1);
+        execute("INSERT INTO %s (k, c, v) VALUES(?, ?, ?)", 1, 3, 2);
+        execute("INSERT INTO %s (k, c, v) VALUES(?, ?, ?)", 2, 2, 3);
+
+        // Don't require filtering, always allowed
+        assertRows(execute("SELECT * FROM %s WHERE k = ?", 1),
+                   row(1, 2, 1),
+                   row(1, 3, 2));
+
+        assertRows(execute("SELECT * FROM %s WHERE k = ? AND c > ?", 1, 2), row(1, 3, 2));
+
+        assertRows(execute("SELECT * FROM %s WHERE k = ? AND c = ?", 1, 2), row(1, 2, 1));
+
+        assertRows(execute("SELECT * FROM %s WHERE k = ? ALLOW FILTERING", 1),
+                   row(1, 2, 1),
+                   row(1, 3, 2));
+
+        assertRows(execute("SELECT * FROM %s WHERE k = ? AND c > ? ALLOW FILTERING", 1, 2), row(1, 3, 2));
+
+        assertRows(execute("SELECT * FROM %s WHERE k = ? AND c = ? ALLOW FILTERING", 1, 2), row(1, 2, 1));
+
+        // Require filtering, allowed only with ALLOW FILTERING
+        assertInvalidMessage("Cannot execute this query as it might involve data filtering",
+                             "SELECT * FROM %s WHERE c = ?", 2);
+        assertInvalidMessage("Cannot execute this query as it might involve data filtering",
+                             "SELECT * FROM %s WHERE c > ? AND c <= ?", 2, 4);
+
+        assertRows(execute("SELECT * FROM %s WHERE c = ? ALLOW FILTERING", 2),
+                   row(1, 2, 1),
+                   row(2, 2, 3));
+
+        assertRows(execute("SELECT * FROM %s WHERE c > ? AND c <= ? ALLOW FILTERING", 2, 4), row(1, 3, 2));
+    }
+
+    @Test
+    public void testAllowFilteringWithIndexedColumn() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, a int, b int)");
+        createIndex("CREATE INDEX ON %s(a)");
+
+        execute("INSERT INTO %s(k, a, b) VALUES(?, ?, ?)", 1, 10, 100);
+        execute("INSERT INTO %s(k, a, b) VALUES(?, ?, ?)", 2, 20, 200);
+        execute("INSERT INTO %s(k, a, b) VALUES(?, ?, ?)", 3, 30, 300);
+        execute("INSERT INTO %s(k, a, b) VALUES(?, ?, ?)", 4, 40, 400);
+
+        // Don't require filtering, always allowed
+        assertRows(execute("SELECT * FROM %s WHERE k = ?", 1), row(1, 10, 100));
+        assertRows(execute("SELECT * FROM %s WHERE a = ?", 20), row(2, 20, 200));
+        assertRows(execute("SELECT * FROM %s WHERE k = ? ALLOW FILTERING", 1), row(1, 10, 100));
+        assertRows(execute("SELECT * FROM %s WHERE a = ? ALLOW FILTERING", 20), row(2, 20, 200));
+
+        assertInvalid("SELECT * FROM %s WHERE a = ? AND b = ?");
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND b = ? ALLOW FILTERING", 20, 200), row(2, 20, 200));
+    }
+
+    @Test
+    public void testIndexQueriesOnComplexPrimaryKey() throws Throwable
+    {
+        createTable("CREATE TABLE %s (pk0 int, pk1 int, ck0 int, ck1 int, ck2 int, value int, PRIMARY KEY ((pk0, pk1), ck0, ck1, ck2))");
+
+        createIndex("CREATE INDEX ON %s (ck1)");
+        createIndex("CREATE INDEX ON %s (ck2)");
+        createIndex("CREATE INDEX ON %s (pk0)");
+        createIndex("CREATE INDEX ON %s (ck0)");
+
+        execute("INSERT INTO %s (pk0, pk1, ck0, ck1, ck2, value) VALUES (?, ?, ?, ?, ?, ?)", 0, 1, 2, 3, 4, 5);
+        execute("INSERT INTO %s (pk0, pk1, ck0, ck1, ck2, value) VALUES (?, ?, ?, ?, ?, ?)", 1, 2, 3, 4, 5, 0);
+        execute("INSERT INTO %s (pk0, pk1, ck0, ck1, ck2, value) VALUES (?, ?, ?, ?, ?, ?)", 2, 3, 4, 5, 0, 1);
+        execute("INSERT INTO %s (pk0, pk1, ck0, ck1, ck2, value) VALUES (?, ?, ?, ?, ?, ?)", 3, 4, 5, 0, 1, 2);
+        execute("INSERT INTO %s (pk0, pk1, ck0, ck1, ck2, value) VALUES (?, ?, ?, ?, ?, ?)", 4, 5, 0, 1, 2, 3);
+        execute("INSERT INTO %s (pk0, pk1, ck0, ck1, ck2, value) VALUES (?, ?, ?, ?, ?, ?)", 5, 0, 1, 2, 3, 4);
+
+        assertRows(execute("SELECT value FROM %s WHERE pk0 = 2"), row(1));
+        assertRows(execute("SELECT value FROM %s WHERE ck0 = 0"), row(3));
+        assertRows(execute("SELECT value FROM %s WHERE pk0 = 3 AND pk1 = 4 AND ck1 = 0"), row(2));
+        assertRows(execute("SELECT value FROM %s WHERE pk0 = 5 AND pk1 = 0 AND ck0 = 1 AND ck2 = 3 ALLOW FILTERING"), row(4));
+    }
+
+    @Test
+    public void testIndexOnClusteringColumns() throws Throwable
+    {
+        createTable("CREATE TABLE %s (id1 int, id2 int, author text, time bigint, v1 text, v2 text, PRIMARY KEY ((id1, id2), author, time))");
+        createIndex("CREATE INDEX ON %s(time)");
+        createIndex("CREATE INDEX ON %s(id2)");
+
+        execute("INSERT INTO %s(id1, id2, author, time, v1, v2) VALUES(0, 0, 'bob', 0, 'A', 'A')");
+        execute("INSERT INTO %s(id1, id2, author, time, v1, v2) VALUES(0, 0, 'bob', 1, 'B', 'B')");
+        execute("INSERT INTO %s(id1, id2, author, time, v1, v2) VALUES(0, 1, 'bob', 2, 'C', 'C')");
+        execute("INSERT INTO %s(id1, id2, author, time, v1, v2) VALUES(0, 0, 'tom', 0, 'D', 'D')");
+        execute("INSERT INTO %s(id1, id2, author, time, v1, v2) VALUES(0, 1, 'tom', 1, 'E', 'E')");
+
+        assertRows(execute("SELECT v1 FROM %s WHERE time = 1"), row("B"), row("E"));
+
+        assertRows(execute("SELECT v1 FROM %s WHERE id2 = 1"), row("C"), row("E"));
+
+        assertRows(execute("SELECT v1 FROM %s WHERE id1 = 0 AND id2 = 0 AND author = 'bob' AND time = 0"), row("A"));
+
+        // Test for CASSANDRA-8206
+        execute("UPDATE %s SET v2 = null WHERE id1 = 0 AND id2 = 0 AND author = 'bob' AND time = 1");
+
+        assertRows(execute("SELECT v1 FROM %s WHERE id2 = 0"), row("A"), row("B"), row("D"));
+
+        assertRows(execute("SELECT v1 FROM %s WHERE time = 1"), row("B"), row("E"));
+
+        assertInvalidMessage("IN restrictions are not supported on indexed columns",
+                             "SELECT v1 FROM %s WHERE id2 = 0 and time IN (1, 2) ALLOW FILTERING");
+    }
+
+    @Test
+    public void testCompositeIndexWithPrimaryKey() throws Throwable
+    {
+        createTable("CREATE TABLE %s (blog_id int, time1 int, time2 int, author text, content text, PRIMARY KEY (blog_id, time1, time2))");
+
+        createIndex("CREATE INDEX ON %s(author)");
+
+        String req = "INSERT INTO %s (blog_id, time1, time2, author, content) VALUES (?, ?, ?, ?, ?)";
+        execute(req, 1, 0, 0, "foo", "bar1");
+        execute(req, 1, 0, 1, "foo", "bar2");
+        execute(req, 2, 1, 0, "foo", "baz");
+        execute(req, 3, 0, 1, "gux", "qux");
+
+        assertRows(execute("SELECT blog_id, content FROM %s WHERE author='foo'"),
+                   row(1, "bar1"),
+                   row(1, "bar2"),
+                   row(2, "baz"));
+        assertRows(execute("SELECT blog_id, content FROM %s WHERE time1 > 0 AND author='foo' ALLOW FILTERING"), row(2, "baz"));
+        assertRows(execute("SELECT blog_id, content FROM %s WHERE time1 = 1 AND author='foo' ALLOW FILTERING"), row(2, "baz"));
+        assertRows(execute("SELECT blog_id, content FROM %s WHERE time1 = 1 AND time2 = 0 AND author='foo' ALLOW FILTERING"),
+                   row(2, "baz"));
+        assertEmpty(execute("SELECT content FROM %s WHERE time1 = 1 AND time2 = 1 AND author='foo' ALLOW FILTERING"));
+        assertEmpty(execute("SELECT content FROM %s WHERE time1 = 1 AND time2 > 0 AND author='foo' ALLOW FILTERING"));
+
+        assertInvalidMessage("Cannot execute this query as it might involve data filtering",
+                             "SELECT content FROM %s WHERE time2 >= 0 AND author='foo'");
+    }
+
+    @Test
+    public void testRangeQueryOnIndex() throws Throwable
+    {
+        createTable("CREATE TABLE %s (id int primary key, row int, setid int);");
+        createIndex("CREATE INDEX ON %s (setid)");
+
+        String q = "INSERT INTO %s (id, row, setid) VALUES (?, ?, ?);";
+        execute(q, 0, 0, 0);
+        execute(q, 1, 1, 0);
+        execute(q, 2, 2, 0);
+        execute(q, 3, 3, 0);
+
+        assertInvalidMessage("Cannot execute this query as it might involve data filtering",
+                             "SELECT * FROM %s WHERE setid = 0 AND row < 1;");
+        assertRows(execute("SELECT * FROM %s WHERE setid = 0 AND row < 1 ALLOW FILTERING;"), row(0, 0, 0));
+    }
+
+    @Test
+    public void testEmptyIN() throws Throwable
+    {
+        for (String compactOption : new String[] { "", " WITH COMPACT STORAGE" })
+        {
+            createTable("CREATE TABLE %s (k1 int, k2 int, v int, PRIMARY KEY (k1, k2))" + compactOption);
+
+            for (int i = 0; i <= 2; i++)
+                for (int j = 0; j <= 2; j++)
+                    execute("INSERT INTO %s (k1, k2, v) VALUES (?, ?, ?)", i, j, i + j);
+
+            assertEmpty(execute("SELECT v FROM %s WHERE k1 IN ()"));
+            assertEmpty(execute("SELECT v FROM %s WHERE k1 = 0 AND k2 IN ()"));
+        }
+    }
+
+    @Test
+    public void testINWithDuplicateValue() throws Throwable
+    {
+        for (String compactOption : new String[] { "", " WITH COMPACT STORAGE" })
+        {
+            createTable("CREATE TABLE %s (k1 int, k2 int, v int, PRIMARY KEY (k1, k2))" + compactOption);
+            execute("INSERT INTO %s (k1,  k2, v) VALUES (?, ?, ?)", 1, 1, 1);
+
+            assertRows(execute("SELECT * FROM %s WHERE k1 IN (?, ?)", 1, 1),
+                       row(1, 1, 1));
+
+            assertRows(execute("SELECT * FROM %s WHERE k1 IN (?, ?) AND k2 IN (?, ?)", 1, 1, 1, 1),
+                       row(1, 1, 1));
+
+            assertRows(execute("SELECT * FROM %s WHERE k1 = ? AND k2 IN (?, ?)", 1, 1, 1),
+                       row(1, 1, 1));
+        }
+    }
+
+    @Test
+    public void testLargeClusteringINValues() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int, c int, v int, PRIMARY KEY (k, c))");
+        execute("INSERT INTO %s (k, c, v) VALUES (0, 0, 0)");
+        List<Integer> inValues = new ArrayList<>(10000);
+        for (int i = 0; i < 10000; i++)
+            inValues.add(i);
+        assertRows(execute("SELECT * FROM %s WHERE k=? AND c IN ?", 0, inValues),
+                row(0, 0, 0));
+    }
+
+    @Test
+    public void testMultiplePartitionKeyWithIndex() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int, b int, c int, d int, e int, f int, PRIMARY KEY ((a, b), c, d, e))");
+        createIndex("CREATE INDEX ON %s (c)");
+        createIndex("CREATE INDEX ON %s (f)");
+
+        execute("INSERT INTO %s (a, b, c, d, e, f) VALUES (?, ?, ?, ?, ?, ?)", 0, 0, 0, 0, 0, 0);
+        execute("INSERT INTO %s (a, b, c, d, e, f) VALUES (?, ?, ?, ?, ?, ?)", 0, 0, 0, 1, 0, 1);
+        execute("INSERT INTO %s (a, b, c, d, e, f) VALUES (?, ?, ?, ?, ?, ?)", 0, 0, 0, 1, 1, 2);
+
+        execute("INSERT INTO %s (a, b, c, d, e, f) VALUES (?, ?, ?, ?, ?, ?)", 0, 0, 1, 0, 0, 3);
+        execute("INSERT INTO %s (a, b, c, d, e, f) VALUES (?, ?, ?, ?, ?, ?)", 0, 0, 1, 1, 0, 4);
+        execute("INSERT INTO %s (a, b, c, d, e, f) VALUES (?, ?, ?, ?, ?, ?)", 0, 0, 1, 1, 1, 5);
+
+        execute("INSERT INTO %s (a, b, c, d, e, f) VALUES (?, ?, ?, ?, ?, ?)", 0, 0, 2, 0, 0, 5);
+
+        assertInvalidMessage("Cannot execute this query as it might involve data filtering",
+                             "SELECT * FROM %s WHERE a = ? AND c = ?", 0, 1);
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND c = ? ALLOW FILTERING", 0, 1),
+                   row(0, 0, 1, 0, 0, 3),
+                   row(0, 0, 1, 1, 0, 4),
+                   row(0, 0, 1, 1, 1, 5));
+
+        assertInvalidMessage("Cannot execute this query as it might involve data filtering",
+                             "SELECT * FROM %s WHERE a = ? AND c = ? AND d = ?", 0, 1, 1);
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND c = ? AND d = ? ALLOW FILTERING", 0, 1, 1),
+                   row(0, 0, 1, 1, 0, 4),
+                   row(0, 0, 1, 1, 1, 5));
+
+        assertInvalidMessage("Partition key parts: b must be restricted as other parts are",
+                             "SELECT * FROM %s WHERE a = ? AND c IN (?) AND  d IN (?) ALLOW FILTERING", 0, 1, 1);
+
+        assertInvalidMessage("Partition key parts: b must be restricted as other parts are",
+                             "SELECT * FROM %s WHERE a = ? AND (c, d) >= (?, ?) ALLOW FILTERING", 0, 1, 1);
+
+        assertInvalidMessage("Cannot execute this query as it might involve data filtering",
+                             "SELECT * FROM %s WHERE a = ? AND c IN (?) AND f = ?", 0, 1, 5);
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND c IN (?) AND f = ? ALLOW FILTERING", 0, 1, 5),
+                   row(0, 0, 1, 1, 1, 5));
+
+        assertInvalidMessage("Cannot execute this query as it might involve data filtering",
+                             "SELECT * FROM %s WHERE a = ? AND c IN (?, ?) AND f = ?", 0, 1, 2, 5);
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND c IN (?, ?) AND f = ? ALLOW FILTERING", 0, 1, 2, 5),
+                   row(0, 0, 1, 1, 1, 5),
+                   row(0, 0, 2, 0, 0, 5));
+
+        assertInvalidMessage("Cannot execute this query as it might involve data filtering",
+                             "SELECT * FROM %s WHERE a = ? AND c IN (?) AND d IN (?) AND f = ?", 0, 1, 0, 3);
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND c IN (?) AND d IN (?) AND f = ? ALLOW FILTERING", 0, 1, 0, 3),
+                   row(0, 0, 1, 0, 0, 3));
+
+        assertInvalidMessage("Partition key parts: b must be restricted as other parts are",
+                             "SELECT * FROM %s WHERE a = ? AND c >= ? ALLOW FILTERING", 0, 1);
+
+        assertInvalidMessage("Cannot execute this query as it might involve data filtering",
+                             "SELECT * FROM %s WHERE a = ? AND c >= ? AND f = ?", 0, 1, 5);
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND c >= ? AND f = ? ALLOW FILTERING", 0, 1, 5),
+                   row(0, 0, 1, 1, 1, 5),
+                   row(0, 0, 2, 0, 0, 5));
+
+        assertInvalidMessage("Cannot execute this query as it might involve data filtering",
+                             "SELECT * FROM %s WHERE a = ? AND c = ? AND d >= ? AND f = ?", 0, 1, 1, 5);
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND c = ? AND d >= ? AND f = ? ALLOW FILTERING", 0, 1, 1, 5),
+                   row(0, 0, 1, 1, 1, 5));
+    }
+
+    @Test
+    public void testFunctionCallWithUnset() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, s text, i int)");
+
+        assertInvalidMessage("Invalid unset value for argument in call to function token",
+                             "SELECT * FROM %s WHERE token(k) >= token(?)", unset());
+        assertInvalidMessage("Invalid unset value for argument in call to function blobasint",
+                             "SELECT * FROM %s WHERE k = blobAsInt(?)", unset());
+    }
+
+    @Test
+    public void testLimitWithUnset() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, i int)");
+        execute("INSERT INTO %s (k, i) VALUES (1, 1)");
+        execute("INSERT INTO %s (k, i) VALUES (2, 1)");
+        assertRows(execute("SELECT k FROM %s LIMIT ?", unset()), // treat as 'unlimited'
+                row(1),
+                row(2)
+        );
+    }
+
+    @Test
+    public void testWithUnsetValues() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int, i int, j int, s text, PRIMARY KEY(k,i,j))");
+        createIndex("CREATE INDEX s_index ON %s (s)");
+        // partition key
+        assertInvalidMessage("Invalid unset value for column k", "SELECT * from %s WHERE k = ?", unset());
+        assertInvalidMessage("Invalid unset value for column k", "SELECT * from %s WHERE k IN ?", unset());
+        assertInvalidMessage("Invalid unset value for column k", "SELECT * from %s WHERE k IN(?)", unset());
+        assertInvalidMessage("Invalid unset value for column k", "SELECT * from %s WHERE k IN(?,?)", 1, unset());
+        // clustering column
+        assertInvalidMessage("Invalid unset value for column i", "SELECT * from %s WHERE k = 1 AND i = ?", unset());
+        assertInvalidMessage("Invalid unset value for column i", "SELECT * from %s WHERE k = 1 AND i IN ?", unset());
+        assertInvalidMessage("Invalid unset value for column i", "SELECT * from %s WHERE k = 1 AND i IN(?)", unset());
+        assertInvalidMessage("Invalid unset value for column i", "SELECT * from %s WHERE k = 1 AND i IN(?,?)", 1, unset());
+        assertInvalidMessage("Invalid unset value for column i", "SELECT * from %s WHERE i = ? ALLOW FILTERING", unset());
+        // indexed column
+        assertInvalidMessage("Unsupported unset value for indexed column s", "SELECT * from %s WHERE s = ?", unset());
+        // range
+        assertInvalidMessage("Invalid unset value for column i", "SELECT * from %s WHERE k = 1 AND i > ?", unset());
+    }
+}


[03/32] cassandra git commit: Migrate CQL tests from dtest to unit tests

Posted by jm...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/f797bfa4/test/unit/org/apache/cassandra/cql3/validation/operations/AlterTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/AlterTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/AlterTest.java
new file mode 100644
index 0000000..cc7d2a4
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/AlterTest.java
@@ -0,0 +1,201 @@
+/*
+ * 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.validation.operations;
+
+import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.exceptions.SyntaxException;
+
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+
+public class AlterTest extends CQLTester
+{
+    @Test
+    public void testAddList() throws Throwable
+    {
+        createTable("CREATE TABLE %s (id text PRIMARY KEY, content text);");
+        execute("ALTER TABLE %s ADD myCollection list<text>;");
+        execute("INSERT INTO %s (id, content , myCollection) VALUES ('test', 'first test', ['first element']);");
+
+        assertRows(execute("SELECT * FROM %s;"), row("test", "first test", list("first element")));
+    }
+
+    @Test
+    public void testDropList() throws Throwable
+    {
+        createTable("CREATE TABLE %s (id text PRIMARY KEY, content text, myCollection list<text>);");
+        execute("INSERT INTO %s (id, content , myCollection) VALUES ('test', 'first test', ['first element']);");
+        execute("ALTER TABLE %s DROP myCollection;");
+
+        assertRows(execute("SELECT * FROM %s;"), row("test", "first test"));
+    }
+    @Test
+    public void testAddMap() throws Throwable
+    {
+        createTable("CREATE TABLE %s (id text PRIMARY KEY, content text);");
+        execute("ALTER TABLE %s ADD myCollection map<text, text>;");
+        execute("INSERT INTO %s (id, content , myCollection) VALUES ('test', 'first test', { '1' : 'first element'});");
+
+        assertRows(execute("SELECT * FROM %s;"), row("test", "first test", map("1", "first element")));
+    }
+
+    @Test
+    public void testDropMap() throws Throwable
+    {
+        createTable("CREATE TABLE %s (id text PRIMARY KEY, content text, myCollection map<text, text>);");
+        execute("INSERT INTO %s (id, content , myCollection) VALUES ('test', 'first test', { '1' : 'first element'});");
+        execute("ALTER TABLE %s DROP myCollection;");
+
+        assertRows(execute("SELECT * FROM %s;"), row("test", "first test"));
+    }
+
+    @Test
+    public void testDropListAndAddListWithSameName() throws Throwable
+    {
+        createTable("CREATE TABLE %s (id text PRIMARY KEY, content text, myCollection list<text>);");
+        execute("INSERT INTO %s (id, content , myCollection) VALUES ('test', 'first test', ['first element']);");
+        execute("ALTER TABLE %s DROP myCollection;");
+        execute("ALTER TABLE %s ADD myCollection list<text>;");
+
+        assertRows(execute("SELECT * FROM %s;"), row("test", "first test", null));
+        execute("UPDATE %s set myCollection = ['second element'] WHERE id = 'test';");
+        assertRows(execute("SELECT * FROM %s;"), row("test", "first test", list("second element")));
+    }
+    @Test
+    public void testDropListAndAddMapWithSameName() throws Throwable
+    {
+        createTable("CREATE TABLE %s (id text PRIMARY KEY, content text, myCollection list<text>);");
+        execute("INSERT INTO %s (id, content , myCollection) VALUES ('test', 'first test', ['first element']);");
+        execute("ALTER TABLE %s DROP myCollection;");
+
+        assertInvalid("ALTER TABLE %s ADD myCollection map<int, int>;");
+    }
+
+    @Test
+    public void testChangeStrategyWithUnquotedAgrument() throws Throwable
+    {
+        createTable("CREATE TABLE %s (id text PRIMARY KEY);");
+
+        assertInvalidSyntaxMessage("no viable alternative at input '}'",
+                                   "ALTER TABLE %s WITH caching = {'keys' : 'all', 'rows_per_partition' : ALL};");
+    }
+
+    @Test
+    // tests CASSANDRA-7976
+    public void testAlterIndexInterval() throws Throwable
+    {
+        String tableName = createTable("CREATE TABLE IF NOT EXISTS %s (id uuid, album text, artist text, data blob, PRIMARY KEY (id))");
+        ColumnFamilyStore cfs = Keyspace.open(KEYSPACE).getColumnFamilyStore(tableName);
+
+        alterTable("ALTER TABLE %s WITH min_index_interval=256 AND max_index_interval=512");
+        assertEquals(256, cfs.metadata.getMinIndexInterval());
+        assertEquals(512, cfs.metadata.getMaxIndexInterval());
+
+        alterTable("ALTER TABLE %s WITH caching = 'none'");
+        assertEquals(256, cfs.metadata.getMinIndexInterval());
+        assertEquals(512, cfs.metadata.getMaxIndexInterval());
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.create_alter_options_test()
+     */
+    @Test
+    public void testCreateAlterKeyspaces() throws Throwable
+    {
+        assertInvalidThrow(SyntaxException.class, "CREATE KEYSPACE ks1");
+        assertInvalidThrow(ConfigurationException.class, "CREATE KEYSPACE ks1 WITH replication= { 'replication_factor' : 1 }");
+
+        execute("CREATE KEYSPACE ks1 WITH replication={ 'class' : 'SimpleStrategy', 'replication_factor' : 1 }");
+        execute("CREATE KEYSPACE ks2 WITH replication={ 'class' : 'SimpleStrategy', 'replication_factor' : 1 } AND durable_writes=false");
+
+        assertRows(execute("SELECT keyspace_name, durable_writes FROM system.schema_keyspaces"),
+                   row("ks1", true),
+                   row(KEYSPACE, true),
+                   row("ks2", false));
+
+        execute("ALTER KEYSPACE ks1 WITH replication = { 'class' : 'NetworkTopologyStrategy', 'dc1' : 1 } AND durable_writes=False");
+        execute("ALTER KEYSPACE ks2 WITH durable_writes=true");
+
+        assertRows(execute("SELECT keyspace_name, durable_writes, strategy_class FROM system.schema_keyspaces"),
+                   row("ks1", false, "org.apache.cassandra.locator.NetworkTopologyStrategy"),
+                   row(KEYSPACE, true, "org.apache.cassandra.locator.SimpleStrategy"),
+                   row("ks2", true, "org.apache.cassandra.locator.SimpleStrategy"));
+
+        execute("USE ks1");
+
+        assertInvalidThrow(ConfigurationException.class, "CREATE TABLE cf1 (a int PRIMARY KEY, b int) WITH compaction = { 'min_threshold' : 4 }");
+
+        execute("CREATE TABLE cf1 (a int PRIMARY KEY, b int) WITH compaction = { 'class' : 'SizeTieredCompactionStrategy', 'min_threshold' : 7 }");
+        assertRows(execute("SELECT columnfamily_name, min_compaction_threshold FROM system.schema_columnfamilies WHERE keyspace_name='ks1'"),
+                   row("cf1", 7));
+
+        // clean-up
+        execute("DROP KEYSPACE ks1");
+        execute("DROP KEYSPACE ks2");
+    }
+
+    /**
+     * Test for bug of 5232,
+     * migrated from cql_tests.py:TestCQL.alter_bug_test()
+     */
+    @Test
+    public void testAlterStatementWithAdd() throws Throwable
+    {
+        createTable("CREATE TABLE %s (id int PRIMARY KEY, t text)");
+
+        execute("UPDATE %s SET t = '111' WHERE id = 1");
+
+        execute("ALTER TABLE %s ADD l list<text>");
+        assertRows(execute("SELECT * FROM %s"),
+                   row(1, null, "111"));
+
+        execute("ALTER TABLE %s ADD m map<int, text>");
+        assertRows(execute("SELECT * FROM %s"),
+                   row(1, null, null, "111"));
+    }
+
+    /**
+     * Test for 7744,
+     * migrated from cql_tests.py:TestCQL.downgrade_to_compact_bug_test()
+     */
+    @Test
+    public void testDowngradeToCompact() throws Throwable
+    {
+        createTable("create table %s (k int primary key, v set<text>)");
+        execute("insert into %s (k, v) VALUES (0, {'f'})");
+        flush();
+        execute("alter table %s drop v");
+        execute("alter table %s add v int");
+    }
+
+    @Test
+    // tests CASSANDRA-9565
+    public void testDoubleWith() throws Throwable
+    {
+        String[] stmts = new String[] { "ALTER KEYSPACE WITH WITH DURABLE_WRITES = true",
+                                        "ALTER KEYSPACE ks WITH WITH DURABLE_WRITES = true" };
+
+        for (String stmt : stmts) {
+            assertInvalidSyntaxMessage("no viable alternative at input 'WITH'", stmt);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f797bfa4/test/unit/org/apache/cassandra/cql3/validation/operations/BatchTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/BatchTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/BatchTest.java
new file mode 100644
index 0000000..cb49e57
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/BatchTest.java
@@ -0,0 +1,88 @@
+/*
+ * 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.validation.operations;
+
+import org.junit.Test;
+
+import org.apache.cassandra.cql3.CQLTester;
+
+public class BatchTest extends CQLTester
+{
+    /**
+     * Test batch statements
+     * migrated from cql_tests.py:TestCQL.batch_test()
+     */
+    @Test
+    public void testBatch() throws Throwable
+    {
+        createTable("CREATE TABLE %s (userid text PRIMARY KEY, name text, password text)");
+
+        String query = "BEGIN BATCH\n"
+                       + "INSERT INTO %1$s (userid, password, name) VALUES ('user2', 'ch@ngem3b', 'second user');\n"
+                       + "UPDATE %1$s SET password = 'ps22dhds' WHERE userid = 'user3';\n"
+                       + "INSERT INTO %1$s (userid, password) VALUES ('user4', 'ch@ngem3c');\n"
+                       + "DELETE name FROM %1$s WHERE userid = 'user1';\n"
+                       + "APPLY BATCH;";
+
+        execute(query);
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.batch_and_list_test()
+     */
+    @Test
+    public void testBatchAndList() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, l list<int>)");
+
+        execute("BEGIN BATCH " +
+                "UPDATE %1$s SET l = l +[ 1 ] WHERE k = 0; " +
+                "UPDATE %1$s SET l = l + [ 2 ] WHERE k = 0; " +
+                "UPDATE %1$s SET l = l + [ 3 ] WHERE k = 0; " +
+                "APPLY BATCH");
+
+        assertRows(execute("SELECT l FROM %s WHERE k = 0"),
+                   row(list(1, 2, 3)));
+
+        execute("BEGIN BATCH " +
+                "UPDATE %1$s SET l =[ 1 ] + l WHERE k = 1; " +
+                "UPDATE %1$s SET l = [ 2 ] + l WHERE k = 1; " +
+                "UPDATE %1$s SET l = [ 3 ] + l WHERE k = 1; " +
+                "APPLY BATCH ");
+
+        assertRows(execute("SELECT l FROM %s WHERE k = 1"),
+                   row(list(3, 2, 1)));
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.bug_6115_test()
+     */
+    @Test
+    public void testBatchDeleteInsert() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int, v int, PRIMARY KEY (k, v))");
+
+        execute("INSERT INTO %s (k, v) VALUES (0, 1)");
+        execute("BEGIN BATCH DELETE FROM %1$s WHERE k=0 AND v=1; INSERT INTO %1$s (k, v) VALUES (0, 2); APPLY BATCH");
+
+        assertRows(execute("SELECT * FROM %s"),
+                   row(0, 2));
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f797bfa4/test/unit/org/apache/cassandra/cql3/validation/operations/CreateTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/CreateTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/CreateTest.java
new file mode 100644
index 0000000..3240c06
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/CreateTest.java
@@ -0,0 +1,462 @@
+/*
+ * 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.validation.operations;
+
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.UUID;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.config.TriggerDefinition;
+import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.db.ColumnFamily;
+import org.apache.cassandra.db.Mutation;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.exceptions.SyntaxException;
+import org.apache.cassandra.triggers.ITrigger;
+
+import static junit.framework.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+public class CreateTest extends CQLTester
+{
+    @Test
+    public void testCQL3PartitionKeyOnlyTable()
+    {
+        createTable("CREATE TABLE %s (id text PRIMARY KEY);");
+        assertFalse(currentTableMetadata().isThriftCompatible());
+    }
+
+    /**
+     * Creation and basic operations on a static table,
+     * migrated from cql_tests.py:TestCQL.static_cf_test()
+     */
+    @Test
+    public void testStaticTable() throws Throwable
+    {
+        createTable("CREATE TABLE %s (userid uuid PRIMARY KEY, firstname text, lastname text, age int)");
+
+        UUID id1 = UUID.fromString("550e8400-e29b-41d4-a716-446655440000");
+        UUID id2 = UUID.fromString("f47ac10b-58cc-4372-a567-0e02b2c3d479");
+
+        execute("INSERT INTO %s (userid, firstname, lastname, age) VALUES (?, ?, ?, ?)", id1, "Frodo", "Baggins", 32);
+        execute("UPDATE %s SET firstname = ?, lastname = ?, age = ? WHERE userid = ?", "Samwise", "Gamgee", 33, id2);
+
+        assertRows(execute("SELECT firstname, lastname FROM %s WHERE userid = ?", id1),
+                   row("Frodo", "Baggins"));
+
+        assertRows(execute("SELECT * FROM %s WHERE userid = ?", id1),
+                   row(id1, 32, "Frodo", "Baggins"));
+
+        assertRows(execute("SELECT * FROM %s"),
+                   row(id2, 33, "Samwise", "Gamgee"),
+                   row(id1, 32, "Frodo", "Baggins")
+        );
+
+        String batch = "BEGIN BATCH "
+                       + "INSERT INTO %1$s (userid, age) VALUES (?, ?) "
+                       + "UPDATE %1$s SET age = ? WHERE userid = ? "
+                       + "DELETE firstname, lastname FROM %1$s WHERE userid = ? "
+                       + "DELETE firstname, lastname FROM %1$s WHERE userid = ? "
+                       + "APPLY BATCH";
+
+        execute(batch, id1, 36, 37, id2, id1, id2);
+
+        assertRows(execute("SELECT * FROM %s"),
+                   row(id2, 37, null, null),
+                   row(id1, 36, null, null));
+    }
+
+    /**
+     * Creation and basic operations on a static table with compact storage,
+     * migrated from cql_tests.py:TestCQL.noncomposite_static_cf_test()
+     */
+    @Test
+    public void testDenseStaticTable() throws Throwable
+    {
+        createTable("CREATE TABLE %s (userid uuid PRIMARY KEY, firstname text, lastname text, age int) WITH COMPACT STORAGE");
+
+        UUID id1 = UUID.fromString("550e8400-e29b-41d4-a716-446655440000");
+        UUID id2 = UUID.fromString("f47ac10b-58cc-4372-a567-0e02b2c3d479");
+
+        execute("INSERT INTO %s (userid, firstname, lastname, age) VALUES (?, ?, ?, ?)", id1, "Frodo", "Baggins", 32);
+        execute("UPDATE %s SET firstname = ?, lastname = ?, age = ? WHERE userid = ?", "Samwise", "Gamgee", 33, id2);
+
+        assertRows(execute("SELECT firstname, lastname FROM %s WHERE userid = ?", id1),
+                   row("Frodo", "Baggins"));
+
+        assertRows(execute("SELECT * FROM %s WHERE userid = ?", id1),
+                   row(id1, 32, "Frodo", "Baggins"));
+
+        assertRows(execute("SELECT * FROM %s"),
+                   row(id2, 33, "Samwise", "Gamgee"),
+                   row(id1, 32, "Frodo", "Baggins")
+        );
+
+        String batch = "BEGIN BATCH "
+                       + "INSERT INTO %1$s (userid, age) VALUES (?, ?) "
+                       + "UPDATE %1$s SET age = ? WHERE userid = ? "
+                       + "DELETE firstname, lastname FROM %1$s WHERE userid = ? "
+                       + "DELETE firstname, lastname FROM %1$s WHERE userid = ? "
+                       + "APPLY BATCH";
+
+        execute(batch, id1, 36, 37, id2, id1, id2);
+
+        assertRows(execute("SELECT * FROM %s"),
+                   row(id2, 37, null, null),
+                   row(id1, 36, null, null));
+    }
+
+    /**
+     * Creation and basic operations on a non-composite table with compact storage,
+     * migrated from cql_tests.py:TestCQL.dynamic_cf_test()
+     */
+    @Test
+    public void testDenseNonCompositeTable() throws Throwable
+    {
+        createTable("CREATE TABLE %s (userid uuid, url text, time bigint, PRIMARY KEY (userid, url)) WITH COMPACT STORAGE");
+
+        UUID id1 = UUID.fromString("550e8400-e29b-41d4-a716-446655440000");
+        UUID id2 = UUID.fromString("f47ac10b-58cc-4372-a567-0e02b2c3d479");
+        UUID id3 = UUID.fromString("810e8500-e29b-41d4-a716-446655440000");
+
+        execute("INSERT INTO %s (userid, url, time) VALUES (?, ?, ?)", id1, "http://foo.bar", 42L);
+        execute("INSERT INTO %s (userid, url, time) VALUES (?, ?, ?)", id1, "http://foo-2.bar", 24L);
+        execute("INSERT INTO %s (userid, url, time) VALUES (?, ?, ?)", id1, "http://bar.bar", 128L);
+        execute("UPDATE %s SET time = 24 WHERE userid = ? and url = 'http://bar.foo'", id2);
+        execute("UPDATE %s SET time = 12 WHERE userid IN (?, ?) and url = 'http://foo-3'", id2, id1);
+
+        assertRows(execute("SELECT url, time FROM %s WHERE userid = ?", id1),
+                   row("http://bar.bar", 128L),
+                   row("http://foo-2.bar", 24L),
+                   row("http://foo-3", 12L),
+                   row("http://foo.bar", 42L));
+
+        assertRows(execute("SELECT * FROM %s WHERE userid = ?", id2),
+                   row(id2, "http://bar.foo", 24L),
+                   row(id2, "http://foo-3", 12L));
+
+        assertRows(execute("SELECT time FROM %s"),
+                   row(24L), // id2
+                   row(12L),
+                   row(128L), // id1
+                   row(24L),
+                   row(12L),
+                   row(42L)
+        );
+
+        // Check we don't allow empty values for url since this is the full underlying cell name (#6152)
+        assertInvalid("INSERT INTO %s (userid, url, time) VALUES (?, '', 42)", id3);
+    }
+
+    /**
+     * Creation and basic operations on a composite table with compact storage,
+     * migrated from cql_tests.py:TestCQL.dense_cf_test()
+     */
+    @Test
+    public void testDenseCompositeTable() throws Throwable
+    {
+        createTable("CREATE TABLE %s (userid uuid, ip text, port int, time bigint, PRIMARY KEY (userid, ip, port)) WITH COMPACT STORAGE");
+
+        UUID id1 = UUID.fromString("550e8400-e29b-41d4-a716-446655440000");
+        UUID id2 = UUID.fromString("f47ac10b-58cc-4372-a567-0e02b2c3d479");
+
+        execute("INSERT INTO %s (userid, ip, port, time) VALUES (?, '192.168.0.1', 80, 42)", id1);
+        execute("INSERT INTO %s (userid, ip, port, time) VALUES (?, '192.168.0.2', 80, 24)", id1);
+        execute("INSERT INTO %s (userid, ip, port, time) VALUES (?, '192.168.0.2', 90, 42)", id1);
+        execute("UPDATE %s SET time = 24 WHERE userid = ? AND ip = '192.168.0.2' AND port = 80", id2);
+
+        // we don't have to include all of the clustering columns (see CASSANDRA-7990)
+        execute("INSERT INTO %s (userid, ip, time) VALUES (?, '192.168.0.3', 42)", id2);
+        execute("UPDATE %s SET time = 42 WHERE userid = ? AND ip = '192.168.0.4'", id2);
+
+        assertRows(execute("SELECT ip, port, time FROM %s WHERE userid = ?", id1),
+                   row("192.168.0.1", 80, 42L),
+                   row("192.168.0.2", 80, 24L),
+                   row("192.168.0.2", 90, 42L));
+
+        assertRows(execute("SELECT ip, port, time FROM %s WHERE userid = ? and ip >= '192.168.0.2'", id1),
+                   row("192.168.0.2", 80, 24L),
+                   row("192.168.0.2", 90, 42L));
+
+        assertRows(execute("SELECT ip, port, time FROM %s WHERE userid = ? and ip = '192.168.0.2'", id1),
+                   row("192.168.0.2", 80, 24L),
+                   row("192.168.0.2", 90, 42L));
+
+        assertEmpty(execute("SELECT ip, port, time FROM %s WHERE userid = ? and ip > '192.168.0.2'", id1));
+
+        assertRows(execute("SELECT ip, port, time FROM %s WHERE userid = ? AND ip = '192.168.0.3'", id2),
+                   row("192.168.0.3", null, 42L));
+
+        assertRows(execute("SELECT ip, port, time FROM %s WHERE userid = ? AND ip = '192.168.0.4'", id2),
+                   row("192.168.0.4", null, 42L));
+
+        execute("DELETE time FROM %s WHERE userid = ? AND ip = '192.168.0.2' AND port = 80", id1);
+
+        assertRowCount(execute("SELECT * FROM %s WHERE userid = ?", id1), 2);
+
+        execute("DELETE FROM %s WHERE userid = ?", id1);
+        assertEmpty(execute("SELECT * FROM %s WHERE userid = ?", id1));
+
+        execute("DELETE FROM %s WHERE userid = ? AND ip = '192.168.0.3'", id2);
+        assertEmpty(execute("SELECT * FROM %s WHERE userid = ? AND ip = '192.168.0.3'", id2));
+    }
+
+    /**
+     * Creation and basic operations on a composite table,
+     * migrated from cql_tests.py:TestCQL.sparse_cf_test()
+     */
+    @Test
+    public void testSparseCompositeTable() throws Throwable
+    {
+        createTable("CREATE TABLE %s (userid uuid, posted_month int, posted_day int, body text, posted_by text, PRIMARY KEY (userid, posted_month, posted_day))");
+
+        UUID id1 = UUID.fromString("550e8400-e29b-41d4-a716-446655440000");
+        UUID id2 = UUID.fromString("f47ac10b-58cc-4372-a567-0e02b2c3d479");
+
+        execute("INSERT INTO %s (userid, posted_month, posted_day, body, posted_by) VALUES (?, 1, 12, 'Something else', 'Frodo Baggins')", id1);
+        execute("INSERT INTO %s (userid, posted_month, posted_day, body, posted_by) VALUES (?, 1, 24, 'Something something', 'Frodo Baggins')", id1);
+        execute("UPDATE %s SET body = 'Yo Froddo', posted_by = 'Samwise Gamgee' WHERE userid = ? AND posted_month = 1 AND posted_day = 3", id2);
+        execute("UPDATE %s SET body = 'Yet one more message' WHERE userid = ? AND posted_month = 1 and posted_day = 30", id1);
+
+        assertRows(execute("SELECT body, posted_by FROM %s WHERE userid = ? AND posted_month = 1 AND posted_day = 24", id1),
+                   row("Something something", "Frodo Baggins"));
+
+        assertRows(execute("SELECT posted_day, body, posted_by FROM %s WHERE userid = ? AND posted_month = 1 AND posted_day > 12", id1),
+                   row(24, "Something something", "Frodo Baggins"),
+                   row(30, "Yet one more message", null));
+
+        assertRows(execute("SELECT posted_day, body, posted_by FROM %s WHERE userid = ? AND posted_month = 1", id1),
+                   row(12, "Something else", "Frodo Baggins"),
+                   row(24, "Something something", "Frodo Baggins"),
+                   row(30, "Yet one more message", null));
+    }
+
+    /**
+     * Check invalid create table statements,
+     * migrated from cql_tests.py:TestCQL.create_invalid_test()
+     */
+    @Test
+    public void testInvalidCreateTableStatements() throws Throwable
+    {
+        assertInvalidThrow(SyntaxException.class, "CREATE TABLE test ()");
+
+        assertInvalid("CREATE TABLE test (c1 text, c2 text, c3 text)");
+        assertInvalid("CREATE TABLE test (key1 text PRIMARY KEY, key2 text PRIMARY KEY)");
+
+        assertInvalid("CREATE TABLE test (key text PRIMARY KEY, key int)");
+        assertInvalid("CREATE TABLE test (key text PRIMARY KEY, c int, c text)");
+
+        assertInvalid("CREATE TABLE test (key text, key2 text, c int, d text, PRIMARY KEY (key, key2)) WITH COMPACT STORAGE");
+    }
+
+    /**
+     * Check obsolete properties from CQL2 are rejected
+     * migrated from cql_tests.py:TestCQL.invalid_old_property_test()
+     */
+    @Test
+    public void testObsoleteTableProperties() throws Throwable
+    {
+        assertInvalidThrow(SyntaxException.class, "CREATE TABLE test (foo text PRIMARY KEY, c int) WITH default_validation=timestamp");
+
+        createTable("CREATE TABLE %s (foo text PRIMARY KEY, c int)");
+        assertInvalidThrow(SyntaxException.class, "ALTER TABLE %s WITH default_validation=int");
+    }
+
+    /**
+     * Test create and drop keyspace
+     * migrated from cql_tests.py:TestCQL.keyspace_test()
+     */
+    @Test
+    public void testKeyspace() throws Throwable
+    {
+        assertInvalidThrow(SyntaxException.class, "CREATE KEYSPACE %s testXYZ ");
+
+        execute("CREATE KEYSPACE testXYZ WITH replication = { 'class' : 'SimpleStrategy', 'replication_factor' : 1 }");
+
+        assertInvalid(
+                     "CREATE KEYSPACE My_much_much_too_long_identifier_that_should_not_work WITH replication = { 'class' : 'SimpleStrategy', 'replication_factor' : 1 }");
+
+        execute("DROP KEYSPACE testXYZ");
+        assertInvalidThrow(ConfigurationException.class, "DROP KEYSPACE non_existing");
+
+        execute("CREATE KEYSPACE testXYZ WITH replication = { 'class' : 'SimpleStrategy', 'replication_factor' : 1 }");
+
+        // clean-up
+        execute("DROP KEYSPACE testXYZ");
+    }
+
+    /**
+     * Test create and drop table
+     * migrated from cql_tests.py:TestCQL.table_test()
+     */
+    @Test
+    public void testTable() throws Throwable
+    {
+        String table1 = createTable(" CREATE TABLE %s (k int PRIMARY KEY, c int)");
+        createTable(" CREATE TABLE %s (k int, name int, value int, PRIMARY KEY(k, name)) WITH COMPACT STORAGE ");
+        createTable(" CREATE TABLE %s (k int, c int, PRIMARY KEY (k),)");
+
+        String table4 = createTableName();
+
+        // repeated column
+        assertInvalidMessage("Multiple definition of identifier k", String.format("CREATE TABLE %s (k int PRIMARY KEY, c int, k text)", table4));
+
+        // compact storage limitations
+        assertInvalidThrow(SyntaxException.class,
+                           String.format("CREATE TABLE %s (k int, name, int, c1 int, c2 int, PRIMARY KEY(k, name)) WITH COMPACT STORAGE", table4));
+
+        execute(String.format("DROP TABLE %s.%s", keyspace(), table1));
+
+        createTable(String.format("CREATE TABLE %s.%s ( k int PRIMARY KEY, c1 int, c2 int, ) ", keyspace(), table1));
+    }
+
+    /**
+     * Test truncate statement,
+     * migrated from cql_tests.py:TestCQL.table_test().
+     */
+    @Test
+    public void testTruncate() throws Throwable
+    {
+        createTable(" CREATE TABLE %s (k int, name int, value int, PRIMARY KEY(k, name)) WITH COMPACT STORAGE ");
+        execute("TRUNCATE %s");
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.multiordering_validation_test()
+     */
+    @Test
+    public void testMultiOrderingValidation() throws Throwable
+    {
+        String tableName = KEYSPACE + "." + createTableName();
+        assertInvalid(String.format("CREATE TABLE test (k int, c1 int, c2 int, PRIMARY KEY (k, c1, c2)) WITH CLUSTERING ORDER BY (c2 DESC)", tableName));
+
+        tableName = KEYSPACE + "." + createTableName();
+        assertInvalid(String.format("CREATE TABLE test (k int, c1 int, c2 int, PRIMARY KEY (k, c1, c2)) WITH CLUSTERING ORDER BY (c2 ASC, c1 DESC)", tableName));
+
+        tableName = KEYSPACE + "." + createTableName();
+        assertInvalid(String.format("CREATE TABLE test (k int, c1 int, c2 int, PRIMARY KEY (k, c1, c2)) WITH CLUSTERING ORDER BY (c1 DESC, c2 DESC, c3 DESC)", tableName));
+
+        createTable("CREATE TABLE %s (k int, c1 int, c2 int, PRIMARY KEY (k, c1, c2)) WITH CLUSTERING ORDER BY (c1 DESC, c2 DESC)");
+        createTable("CREATE TABLE %s (k int, c1 int, c2 int, PRIMARY KEY (k, c1, c2)) WITH CLUSTERING ORDER BY (c1 ASC, c2 DESC)");
+    }
+
+    @Test
+    public void testCreateTrigger() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int, b int, c int, PRIMARY KEY (a))");
+        execute("CREATE TRIGGER trigger_1 ON %s USING '" + TestTrigger.class.getName() + "'");
+        assertTriggerExists("trigger_1", TestTrigger.class);
+        execute("CREATE TRIGGER trigger_2 ON %s USING '" + TestTrigger.class.getName() + "'");
+        assertTriggerExists("trigger_2", TestTrigger.class);
+        assertInvalid("CREATE TRIGGER trigger_1 ON %s USING '" + TestTrigger.class.getName() + "'");
+        execute("CREATE TRIGGER \"Trigger 3\" ON %s USING '" + TestTrigger.class.getName() + "'");
+        assertTriggerExists("Trigger 3", TestTrigger.class);
+    }
+
+    @Test
+    public void testCreateTriggerIfNotExists() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int, b int, c int, PRIMARY KEY (a, b))");
+
+        execute("CREATE TRIGGER IF NOT EXISTS trigger_1 ON %s USING '" + TestTrigger.class.getName() + "'");
+        assertTriggerExists("trigger_1", TestTrigger.class);
+
+        execute("CREATE TRIGGER IF NOT EXISTS trigger_1 ON %s USING '" + TestTrigger.class.getName() + "'");
+        assertTriggerExists("trigger_1", TestTrigger.class);
+    }
+
+    @Test
+    public void testDropTrigger() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int, b int, c int, PRIMARY KEY (a))");
+
+        execute("CREATE TRIGGER trigger_1 ON %s USING '" + TestTrigger.class.getName() + "'");
+        assertTriggerExists("trigger_1", TestTrigger.class);
+
+        execute("DROP TRIGGER trigger_1 ON %s");
+        assertTriggerDoesNotExists("trigger_1", TestTrigger.class);
+
+        execute("CREATE TRIGGER trigger_1 ON %s USING '" + TestTrigger.class.getName() + "'");
+        assertTriggerExists("trigger_1", TestTrigger.class);
+
+        assertInvalid("DROP TRIGGER trigger_2 ON %s");
+
+        execute("CREATE TRIGGER \"Trigger 3\" ON %s USING '" + TestTrigger.class.getName() + "'");
+        assertTriggerExists("Trigger 3", TestTrigger.class);
+
+        execute("DROP TRIGGER \"Trigger 3\" ON %s");
+        assertTriggerDoesNotExists("Trigger 3", TestTrigger.class);
+    }
+
+    @Test
+    public void testDropTriggerIfExists() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int, b int, c int, PRIMARY KEY (a))");
+
+        execute("DROP TRIGGER IF EXISTS trigger_1 ON %s");
+        assertTriggerDoesNotExists("trigger_1", TestTrigger.class);
+
+        execute("CREATE TRIGGER trigger_1 ON %s USING '" + TestTrigger.class.getName() + "'");
+        assertTriggerExists("trigger_1", TestTrigger.class);
+
+        execute("DROP TRIGGER IF EXISTS trigger_1 ON %s");
+        assertTriggerDoesNotExists("trigger_1", TestTrigger.class);
+    }
+
+    @Test
+    // tests CASSANDRA-9565
+    public void testDoubleWith() throws Throwable
+    {
+        String[] stmts = new String[] { "CREATE KEYSPACE WITH WITH DURABLE_WRITES = true",
+                                        "CREATE KEYSPACE ks WITH WITH DURABLE_WRITES = true" };
+
+        for (String stmt : stmts) {
+            assertInvalidSyntaxMessage("no viable alternative at input 'WITH'", stmt);
+        }
+    }
+
+    private void assertTriggerExists(String name, Class<?> clazz)
+    {
+        CFMetaData cfm = Schema.instance.getCFMetaData(keyspace(), currentTable()).copy();
+        assertTrue("the trigger does not exist", cfm.containsTriggerDefinition(TriggerDefinition.create(name,
+                                                                                                        clazz.getName())));
+    }
+
+    private void assertTriggerDoesNotExists(String name, Class<?> clazz)
+    {
+        CFMetaData cfm = Schema.instance.getCFMetaData(keyspace(), currentTable()).copy();
+        Assert.assertFalse("the trigger exists", cfm.containsTriggerDefinition(TriggerDefinition.create(name,
+                                                                                                        clazz.getName())));
+    }
+
+    public static class TestTrigger implements ITrigger
+    {
+        public TestTrigger() { }
+        public Collection<Mutation> augment(ByteBuffer key, ColumnFamily update)
+        {
+            return Collections.emptyList();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f797bfa4/test/unit/org/apache/cassandra/cql3/validation/operations/DeleteTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/DeleteTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/DeleteTest.java
new file mode 100644
index 0000000..476ec83
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/DeleteTest.java
@@ -0,0 +1,329 @@
+/*
+ * 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.validation.operations;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+
+import org.junit.Test;
+
+import org.apache.cassandra.cql3.CQLTester;
+
+import static org.junit.Assert.assertEquals;
+
+public class DeleteTest extends CQLTester
+{
+    /** Test for cassandra 8558 */
+    @Test
+    public void testRangeDeletion() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int, b int, c int, d int, PRIMARY KEY (a, b, c))");
+
+        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 1, 1, 1, 1);
+        flush();
+        execute("DELETE FROM %s WHERE a=? AND b=?", 1, 1);
+        flush();
+        assertEmpty(execute("SELECT * FROM %s WHERE a=? AND b=? AND c=?", 1, 1, 1));
+    }
+
+    /**
+     * Test simple deletion and in particular check for #4193 bug
+     * migrated from cql_tests.py:TestCQL.deletion_test()
+     */
+    @Test
+    public void testDeletion() throws Throwable
+    {
+        createTable("CREATE TABLE %s (username varchar, id int, name varchar, stuff varchar, PRIMARY KEY(username, id))");
+
+        execute("INSERT INTO %s (username, id, name, stuff) VALUES (?, ?, ?, ?)", "abc", 2, "rst", "some value");
+        execute("INSERT INTO %s (username, id, name, stuff) VALUES (?, ?, ?, ?)", "abc", 4, "xyz", "some other value");
+
+        assertRows(execute("SELECT * FROM %s"),
+                   row("abc", 2, "rst", "some value"),
+                   row("abc", 4, "xyz", "some other value"));
+
+        execute("DELETE FROM %s WHERE username='abc' AND id=2");
+
+        assertRows(execute("SELECT * FROM %s"),
+                   row("abc", 4, "xyz", "some other value"));
+
+        createTable("CREATE TABLE %s (username varchar, id int, name varchar, stuff varchar, PRIMARY KEY(username, id, name)) WITH COMPACT STORAGE");
+
+        execute("INSERT INTO %s (username, id, name, stuff) VALUES (?, ?, ?, ?)", "abc", 2, "rst", "some value");
+        execute("INSERT INTO %s (username, id, name, stuff) VALUES (?, ?, ?, ?)", "abc", 4, "xyz", "some other value");
+
+        assertRows(execute("SELECT * FROM %s"),
+                   row("abc", 2, "rst", "some value"),
+                   row("abc", 4, "xyz", "some other value"));
+
+        execute("DELETE FROM %s WHERE username='abc' AND id=2");
+
+        assertRows(execute("SELECT * FROM %s"),
+                   row("abc", 4, "xyz", "some other value"));
+    }
+
+    /**
+     * Test deletion by 'composite prefix' (range tombstones)
+     * migrated from cql_tests.py:TestCQL.range_tombstones_test()
+     */
+    @Test
+    public void testDeleteByCompositePrefix() throws Throwable
+    { // This test used 3 nodes just to make sure RowMutation are correctly serialized
+
+        createTable("CREATE TABLE %s ( k int, c1 int, c2 int, v1 int, v2 int, PRIMARY KEY (k, c1, c2))");
+
+        int numRows = 5;
+        int col1 = 2;
+        int col2 = 2;
+        int cpr = col1 * col2;
+
+        for (int i = 0; i < numRows; i++)
+            for (int j = 0; j < col1; j++)
+                for (int k = 0; k < col2; k++)
+                {
+                    int n = (i * cpr) + (j * col2) + k;
+                    execute("INSERT INTO %s (k, c1, c2, v1, v2) VALUES (?, ?, ?, ?, ?)", i, j, k, n, n);
+                }
+
+        for (int i = 0; i < numRows; i++)
+        {
+            Object[][] rows = getRows(execute("SELECT v1, v2 FROM %s where k = ?", i));
+            for (int x = i * cpr; x < (i + 1) * cpr; x++)
+            {
+                assertEquals(x, rows[x - i * cpr][0]);
+                assertEquals(x, rows[x - i * cpr][1]);
+            }
+        }
+
+        for (int i = 0; i < numRows; i++)
+            execute("DELETE FROM %s WHERE k = ? AND c1 = 0", i);
+
+        for (int i = 0; i < numRows; i++)
+        {
+            Object[][] rows = getRows(execute("SELECT v1, v2 FROM %s WHERE k = ?", i));
+            for (int x = i * cpr + col1; x < (i + 1) * cpr; x++)
+            {
+                assertEquals(x, rows[x - i * cpr - col1][0]);
+                assertEquals(x, rows[x - i * cpr - col1][1]);
+            }
+        }
+
+        for (int i = 0; i < numRows; i++)
+        {
+            Object[][] rows = getRows(execute("SELECT v1, v2 FROM %s WHERE k = ?", i));
+            for (int x = i * cpr + col1; x < (i + 1) * cpr; x++)
+            {
+                assertEquals(x, rows[x - i * cpr - col1][0]);
+                assertEquals(x, rows[x - i * cpr - col1][1]);
+            }
+        }
+    }
+
+    /**
+     * Test deletion by 'composite prefix' (range tombstones) with compaction
+     * migrated from cql_tests.py:TestCQL.range_tombstones_compaction_test()
+     */
+    @Test
+    public void testDeleteByCompositePrefixWithCompaction() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int, c1 int, c2 int, v1 text, PRIMARY KEY (k, c1, c2))");
+
+        for (int c1 = 0; c1 < 4; c1++)
+            for (int c2 = 0; c2 < 2; c2++)
+                execute("INSERT INTO %s (k, c1, c2, v1) VALUES (0, ?, ?, ?)", c1, c2, String.format("%d%d", c1, c2));
+
+        flush();
+
+        execute("DELETE FROM %s WHERE k = 0 AND c1 = 1");
+
+        flush();
+        compact();
+
+        Object[][] rows = getRows(execute("SELECT v1 FROM %s WHERE k = 0"));
+
+        int idx = 0;
+        for (int c1 = 0; c1 < 4; c1++)
+            for (int c2 = 0; c2 < 2; c2++)
+                if (c1 != 1)
+                    assertEquals(String.format("%d%d", c1, c2), rows[idx++][0]);
+    }
+
+    /**
+     * Test deletion of rows
+     * migrated from cql_tests.py:TestCQL.delete_row_test()
+     */
+    @Test
+    public void testRowDeletion() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int, c1 int, c2 int, v1 int, v2 int, PRIMARY KEY (k, c1, c2))");
+
+        execute("INSERT INTO %s (k, c1, c2, v1, v2) VALUES (?, ?, ?, ?, ?)", 0, 0, 0, 0, 0);
+        execute("INSERT INTO %s (k, c1, c2, v1, v2) VALUES (?, ?, ?, ?, ?)", 0, 0, 1, 1, 1);
+        execute("INSERT INTO %s (k, c1, c2, v1, v2) VALUES (?, ?, ?, ?, ?)", 0, 0, 2, 2, 2);
+        execute("INSERT INTO %s (k, c1, c2, v1, v2) VALUES (?, ?, ?, ?, ?)", 0, 1, 0, 3, 3);
+
+        execute("DELETE FROM %s WHERE k = 0 AND c1 = 0 AND c2 = 0");
+
+        assertRowCount(execute("SELECT * FROM %s"), 3);
+    }
+
+    /**
+     * Check the semantic of CQL row existence (part of #4361),
+     * migrated from cql_tests.py:TestCQL.row_existence_test()
+     */
+    @Test
+    public void testRowExistence() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int, c int, v1 int, v2 int, PRIMARY KEY (k, c))");
+
+        execute("INSERT INTO %s (k, c, v1, v2) VALUES (1, 1, 1, 1)");
+        assertRows(execute("SELECT * FROM %s"),
+                   row(1, 1, 1, 1));
+
+        assertInvalid("DELETE c FROM %s WHERE k = 1 AND c = 1");
+
+        execute("DELETE v2 FROM %s WHERE k = 1 AND c = 1");
+        assertRows(execute("SELECT * FROM %s"),
+                   row(1, 1, 1, null));
+
+        execute("DELETE v1 FROM %s WHERE k = 1 AND c = 1");
+        assertRows(execute("SELECT * FROM %s"),
+                   row(1, 1, null, null));
+
+        execute("DELETE FROM %s WHERE k = 1 AND c = 1");
+        assertEmpty(execute("SELECT * FROM %s"));
+
+        execute("INSERT INTO %s (k, c) VALUES (2, 2)");
+        assertRows(execute("SELECT * FROM %s"),
+                   row(2, 2, null, null));
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.remove_range_slice_test()
+     */
+    @Test
+    public void testRemoveRangeSlice() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, v int)");
+
+        for (int i = 0; i < 3; i++)
+            execute("INSERT INTO %s (k, v) VALUES (?, ?)", i, i);
+
+        execute("DELETE FROM %s WHERE k = 1");
+        assertRows(execute("SELECT * FROM %s"),
+                   row(0, 0),
+                   row(2, 2));
+    }
+
+    /**
+     * Test deletions
+     * migrated from cql_tests.py:TestCQL.no_range_ghost_test()
+     */
+    @Test
+    public void testNoRangeGhost() throws Throwable
+    {
+        createTable("CREATE TABLE %s ( k int PRIMARY KEY, v int ) ");
+
+        for (int k = 0; k < 5; k++)
+            execute("INSERT INTO %s (k, v) VALUES (?, 0)", k);
+
+        Object[][] rows = getRows(execute("SELECT k FROM %s"));
+
+        int[] ordered = sortIntRows(rows);
+        for (int k = 0; k < 5; k++)
+            assertEquals(k, ordered[k]);
+
+        execute("DELETE FROM %s WHERE k=2");
+
+        rows = getRows(execute("SELECT k FROM %s"));
+        ordered = sortIntRows(rows);
+
+        int idx = 0;
+        for (int k = 0; k < 5; k++)
+            if (k != 2)
+                assertEquals(k, ordered[idx++]);
+
+        // Example from #3505
+        createTable("CREATE TABLE %s ( KEY varchar PRIMARY KEY, password varchar, gender varchar, birth_year bigint)");
+        execute("INSERT INTO %s (KEY, password) VALUES ('user1', 'ch@ngem3a')");
+        execute("UPDATE %s SET gender = 'm', birth_year = 1980 WHERE KEY = 'user1'");
+
+        assertRows(execute("SELECT * FROM %s WHERE KEY='user1'"),
+                   row("user1", 1980L, "m", "ch@ngem3a"));
+
+        execute("TRUNCATE %s");
+        assertEmpty(execute("SELECT * FROM %s"));
+
+        assertEmpty(execute("SELECT * FROM %s WHERE KEY='user1'"));
+    }
+
+    private int[] sortIntRows(Object[][] rows)
+    {
+        int[] ret = new int[rows.length];
+        for (int i = 0; i < ret.length; i++)
+            ret[i] = rows[i][0] == null ? Integer.MIN_VALUE : (Integer) rows[i][0];
+        Arrays.sort(ret);
+        return ret;
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.range_with_deletes_test()
+     */
+    @Test
+    public void testRandomDeletions() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, v int,)");
+
+        int nb_keys = 30;
+        int nb_deletes = 5;
+
+        List<Integer> deletions = new ArrayList<>(nb_keys);
+        for (int i = 0; i < nb_keys; i++)
+        {
+            execute("INSERT INTO %s (k, v) VALUES (?, ?)", i, i);
+            deletions.add(i);
+        }
+
+        Collections.shuffle(deletions);
+
+        for (int i = 0; i < nb_deletes; i++)
+            execute("DELETE FROM %s WHERE k = ?", deletions.get(i));
+
+        assertRowCount(execute("SELECT * FROM %s LIMIT ?", (nb_keys / 2)), nb_keys / 2);
+    }
+
+    /**
+     * Test for CASSANDRA-8558, deleted row still can be selected out
+     * migrated from cql_tests.py:TestCQL.bug_8558_test()
+     */
+    @Test
+    public void testDeletedRowCannotBeSelected() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int, b int, c text,primary key(a,b))");
+        execute("INSERT INTO %s (a,b,c) VALUES(1,1,'1')");
+        flush();
+
+        execute("DELETE FROM %s  where a=1 and b=1");
+        flush();
+
+        assertEmpty(execute("select * from %s  where a=1 and b=1"));
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f797bfa4/test/unit/org/apache/cassandra/cql3/validation/operations/InsertUpdateIfCondition.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/InsertUpdateIfCondition.java b/test/unit/org/apache/cassandra/cql3/validation/operations/InsertUpdateIfCondition.java
new file mode 100644
index 0000000..e2ebfcb
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/InsertUpdateIfCondition.java
@@ -0,0 +1,861 @@
+/*
+ * 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.validation.operations;
+
+import org.junit.Test;
+
+import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.exceptions.SyntaxException;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+public class InsertUpdateIfCondition extends CQLTester
+{
+    /**
+     * Migrated from cql_tests.py:TestCQL.cas_simple_test()
+     */
+    @Test
+    public void testSimpleCas() throws Throwable
+    {
+        createTable("CREATE TABLE %s (tkn int, consumed boolean, PRIMARY KEY (tkn))");
+
+        for (int i = 0; i < 10; i++)
+        {
+            execute("INSERT INTO %s (tkn, consumed) VALUES (?, FALSE)", i);
+
+            assertRows(execute("UPDATE %s SET consumed = TRUE WHERE tkn = ? IF consumed = FALSE", i), row(true));
+            assertRows(execute("UPDATE %s SET consumed = TRUE WHERE tkn = ? IF consumed = FALSE", i), row(false, true));
+        }
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.conditional_update_test()
+     */
+    @Test
+    public void testConditionalUpdate() throws Throwable
+    {
+        createTable(" CREATE TABLE %s (k int PRIMARY KEY, v1 int, v2 text, v3 int)");
+
+        // Shouldn't apply
+        assertRows(execute("UPDATE %s SET v1 = 3, v2 = 'bar' WHERE k = 0 IF v1 = 4"), row(false));
+        assertRows(execute("UPDATE %s SET v1 = 3, v2 = 'bar' WHERE k = 0 IF EXISTS"), row(false));
+
+        // Should apply
+        assertRows(execute("INSERT INTO %s (k, v1, v2) VALUES (0, 2, 'foo') IF NOT EXISTS"), row(true));
+
+        // Shouldn't apply
+        assertRows(execute("INSERT INTO %s (k, v1, v2) VALUES (0, 5, 'bar') IF NOT EXISTS"), row(false, 0, 2, "foo", null));
+        assertRows(execute("SELECT * FROM %s"), row(0, 2, "foo", null));
+
+        // Shouldn't apply
+        assertRows(execute("UPDATE %s SET v1 = 3, v2 = 'bar' WHERE k = 0 IF v1 = 4"), row(false, 2));
+        assertRows(execute("SELECT * FROM %s"), row(0, 2, "foo", null));
+
+        // Should apply (note: we want v2 before v1 in the statement order to exercise #5786)
+        assertRows(execute("UPDATE %s SET v2 = 'bar', v1 = 3 WHERE k = 0 IF v1 = 2"), row(true));
+        assertRows(execute("UPDATE %s SET v2 = 'bar', v1 = 3 WHERE k = 0 IF EXISTS"), row(true));
+        assertRows(execute("SELECT * FROM %s"), row(0, 3, "bar", null));
+
+        // Shouldn't apply, only one condition is ok
+        assertRows(execute("UPDATE %s SET v1 = 5, v2 = 'foobar' WHERE k = 0 IF v1 = 3 AND v2 = 'foo'"), row(false, 3, "bar"));
+        assertRows(execute("SELECT * FROM %s"), row(0, 3, "bar", null));
+
+        // Should apply
+        assertRows(execute("UPDATE %s SET v1 = 5, v2 = 'foobar' WHERE k = 0 IF v1 = 3 AND v2 = 'bar'"), row(true));
+        assertRows(execute("SELECT * FROM %s"), row(0, 5, "foobar", null));
+
+        // Shouldn't apply
+        assertRows(execute("DELETE v2 FROM %s WHERE k = 0 IF v1 = 3"), row(false, 5));
+        assertRows(execute("SELECT * FROM %s"), row(0, 5, "foobar", null));
+
+        // Shouldn't apply
+        assertRows(execute("DELETE v2 FROM %s WHERE k = 0 IF v1 = null"), row(false, 5));
+        assertRows(execute("SELECT * FROM %s"), row(0, 5, "foobar", null));
+
+        // Should apply
+        assertRows(execute("DELETE v2 FROM %s WHERE k = 0 IF v1 = 5"), row(true));
+        assertRows(execute("SELECT * FROM %s"), row(0, 5, null, null));
+
+        // Shouln't apply
+        assertRows(execute("DELETE v1 FROM %s WHERE k = 0 IF v3 = 4"), row(false, null));
+
+        // Should apply
+        assertRows(execute("DELETE v1 FROM %s WHERE k = 0 IF v3 = null"), row(true));
+        assertRows(execute("SELECT * FROM %s"), row(0, null, null, null));
+
+        // Should apply
+        assertRows(execute("DELETE FROM %s WHERE k = 0 IF v1 = null"), row(true));
+        assertEmpty(execute("SELECT * FROM %s"));
+
+        // Shouldn't apply
+        assertRows(execute("UPDATE %s SET v1 = 3, v2 = 'bar' WHERE k = 0 IF EXISTS"), row(false));
+
+        // Should apply
+        assertRows(execute("DELETE FROM %s WHERE k = 0 IF v1 IN (null)"), row(true));
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.non_eq_conditional_update_test()
+     */
+    @Test
+    public void testNonEqConditionalUpdate() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, v1 int, v2 text, v3 int)");
+
+        // non-EQ conditions
+        execute("INSERT INTO %s (k, v1, v2) VALUES (0, 2, 'foo')");
+
+        assertRows(execute("UPDATE %s SET v2 = 'bar' WHERE k = 0 IF v1 < 3"), row(true));
+        assertRows(execute("UPDATE %s SET v2 = 'bar' WHERE k = 0 IF v1 <= 3"), row(true));
+        assertRows(execute("UPDATE %s SET v2 = 'bar' WHERE k = 0 IF v1 > 1"), row(true));
+        assertRows(execute("UPDATE %s SET v2 = 'bar' WHERE k = 0 IF v1 >= 1"), row(true));
+        assertRows(execute("UPDATE %s SET v2 = 'bar' WHERE k = 0 IF v1 != 1"), row(true));
+        assertRows(execute("UPDATE %s SET v2 = 'bar' WHERE k = 0 IF v1 != 2"), row(false, 2));
+        assertRows(execute("UPDATE %s SET v2 = 'bar' WHERE k = 0 IF v1 IN (0, 1, 2)"), row(true));
+        assertRows(execute("UPDATE %s SET v2 = 'bar' WHERE k = 0 IF v1 IN (142, 276)"), row(false, 2));
+        assertRows(execute("UPDATE %s SET v2 = 'bar' WHERE k = 0 IF v1 IN ()"), row(false, 2));
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.conditional_delete_test()
+     */
+    @Test
+    public void testConditionalDelete() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, v1 int,)");
+
+        assertRows(execute("DELETE FROM %s WHERE k=1 IF EXISTS"), row(false));
+
+        execute("INSERT INTO %s (k, v1) VALUES (1, 2)");
+        assertRows(execute("DELETE FROM %s WHERE k=1 IF EXISTS"), row(true));
+        assertEmpty(execute("SELECT * FROM %s WHERE k=1"));
+        assertRows(execute("DELETE FROM %s WHERE k=1 IF EXISTS"), row(false));
+
+        execute("UPDATE %s USING TTL 1 SET v1=2 WHERE k=1");
+        Thread.sleep(1001);
+        assertRows(execute("DELETE FROM %s WHERE k=1 IF EXISTS"), row(false));
+        assertEmpty(execute("SELECT * FROM %s WHERE k=1"));
+
+        execute("INSERT INTO %s (k, v1) VALUES (2, 2) USING TTL 1");
+        Thread.sleep(1001);
+        assertRows(execute("DELETE FROM %s WHERE k=2 IF EXISTS"), row(false));
+        assertEmpty(execute("SELECT * FROM %s WHERE k=2"));
+
+        execute("INSERT INTO %s (k, v1) VALUES (3, 2)");
+        assertRows(execute("DELETE v1 FROM %s WHERE k=3 IF EXISTS"), row(true));
+        assertRows(execute("SELECT * FROM %s WHERE k=3"), row(3, null));
+        assertRows(execute("DELETE v1 FROM %s WHERE k=3 IF EXISTS"), row(true));
+        assertRows(execute("DELETE FROM %s WHERE k=3 IF EXISTS"), row(true));
+
+        // static columns
+        createTable("CREATE TABLE %s (k text, s text static, i int, v text, PRIMARY KEY (k, i) )");
+
+        execute("INSERT INTO %s (k, s, i, v) VALUES ('k', 's', 0, 'v')");
+        assertRows(execute("DELETE v FROM %s WHERE k='k' AND i=0 IF EXISTS"), row(true));
+        assertRows(execute("DELETE FROM %s WHERE k='k' AND i=0 IF EXISTS"), row(true));
+        assertRows(execute("DELETE v FROM %s WHERE k='k' AND i=0 IF EXISTS"), row(false));
+        assertRows(execute("DELETE FROM %s WHERE k='k' AND i=0 IF EXISTS"), row(false));
+
+        // CASSANDRA-6430
+        assertInvalid("DELETE FROM %s WHERE k = 'k' IF EXISTS");
+        assertInvalid("DELETE FROM %s WHERE k = 'k' IF v = 'foo'");
+        assertInvalid("DELETE FROM %s WHERE i = 0 IF EXISTS");
+        assertInvalid("DELETE FROM %s WHERE k = 0 AND i > 0 IF EXISTS");
+        assertInvalid("DELETE FROM %s WHERE k = 0 AND i > 0 IF v = 'foo'");
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.static_columns_cas_test()
+     */
+    @Test
+    public void testStaticColumnsCas() throws Throwable
+    {
+        createTable("CREATE TABLE %s (id int, k text, version int static, v text, PRIMARY KEY (id, k))");
+
+        // Test that INSERT IF NOT EXISTS concerns only the static column if no clustering nor regular columns
+        // is provided, but concerns the CQL3 row targetted by the clustering columns otherwise
+        execute("INSERT INTO %s (id, k, v) VALUES (1, 'foo', 'foo')");
+        assertRows(execute("INSERT INTO %s (id, k, version) VALUES (1, 'foo', 1) IF NOT EXISTS"), row(false, 1, "foo", null, "foo"));
+        assertRows(execute("INSERT INTO %s (id, version) VALUES (1, 1) IF NOT EXISTS"), row(true));
+        assertRows(execute("SELECT * FROM %s"), row(1, "foo", 1, "foo"));
+        execute("DELETE FROM %s WHERE id = 1");
+
+        execute("INSERT INTO %s(id, version) VALUES (0, 0)");
+
+        assertRows(execute("UPDATE %s SET v='foo', version=1 WHERE id=0 AND k='k1' IF version = 0"), row(true));
+        assertRows(execute("SELECT * FROM %s"), row(0, "k1", 1, "foo"));
+
+        assertRows(execute("UPDATE %s SET v='bar', version=1 WHERE id=0 AND k='k2' IF version = 0"), row(false, 1));
+        assertRows(execute("SELECT * FROM %s"), row(0, "k1", 1, "foo"));
+
+        assertRows(execute("UPDATE %s SET v='bar', version=2 WHERE id=0 AND k='k2' IF version = 1"), row(true));
+        assertRows(execute("SELECT * FROM %s"), row(0, "k1", 2, "foo"), row(0, "k2", 2, "bar"));
+
+        // Testing batches
+        assertRows(execute("BEGIN BATCH " +
+                           "UPDATE %1$s SET v='foobar' WHERE id=0 AND k='k1'; " +
+                           "UPDATE %1$s SET v='barfoo' WHERE id=0 AND k='k2'; " +
+                           "UPDATE %1$s SET version=3 WHERE id=0 IF version=1; " +
+                           "APPLY BATCH "),
+                   row(false, 0, null, 2));
+
+        assertRows(execute("BEGIN BATCH " +
+                           "UPDATE %1$s SET v = 'foobar' WHERE id = 0 AND k = 'k1'; " +
+                           "UPDATE %1$s SET v = 'barfoo' WHERE id = 0 AND k = 'k2'; " +
+                           "UPDATE %1$s SET version = 3 WHERE id = 0 IF version = 2; " +
+                           "APPLY BATCH "),
+                   row(true));
+
+        assertRows(execute("SELECT * FROM %s"),
+                   row(0, "k1", 3, "foobar"),
+                   row(0, "k2", 3, "barfoo"));
+
+        assertRows(execute("BEGIN BATCH " +
+                           "UPDATE %1$s SET version = 4 WHERE id = 0 IF version = 3; " +
+                           "UPDATE %1$s SET v='row1' WHERE id=0 AND k='k1' IF v='foo'; " +
+                           "UPDATE %1$s SET v='row2' WHERE id=0 AND k='k2' IF v='bar'; " +
+                           "APPLY BATCH "),
+                   row(false, 0, "k1", 3, "foobar"),
+                   row(false, 0, "k2", 3, "barfoo"));
+
+        assertRows(execute("BEGIN BATCH " +
+                           "UPDATE %1$s SET version = 4 WHERE id = 0 IF version = 3; " +
+                           "UPDATE %1$s SET v='row1' WHERE id = 0 AND k='k1' IF v='foobar'; " +
+                           "UPDATE %1$s SET v='row2' WHERE id = 0 AND k='k2' IF v='barfoo'; " +
+                           "APPLY BATCH "),
+                   row(true));
+
+        assertRows(execute("SELECT * FROM %s"),
+                   row(0, "k1", 4, "row1"),
+                   row(0, "k2", 4, "row2"));
+
+        assertInvalid("BEGIN BATCH " +
+                      "UPDATE %1$s SET version=5 WHERE id=0 IF version=4; " +
+                      "UPDATE %1$s SET v='row1' WHERE id=0 AND k='k1'; " +
+                      "UPDATE %1$s SET v='row2' WHERE id=1 AND k='k2'; " +
+                      "APPLY BATCH ");
+
+        assertRows(execute("BEGIN BATCH " +
+                           "INSERT INTO %1$s (id, k, v) VALUES (1, 'k1', 'val1') IF NOT EXISTS; " +
+                           "INSERT INTO %1$s (id, k, v) VALUES (1, 'k2', 'val2') IF NOT EXISTS; " +
+                           "APPLY BATCH "),
+                   row(true));
+
+        assertRows(execute("SELECT * FROM %s WHERE id=1"),
+                   row(1, "k1", null, "val1"),
+                   row(1, "k2", null, "val2"));
+
+        assertRows(execute("INSERT INTO %s (id, k, v) VALUES (1, 'k2', 'val2') IF NOT EXISTS"), row(false, 1, "k2", null, "val2"));
+
+        assertRows(execute("BEGIN BATCH " +
+                           "INSERT INTO %1$s (id, k, v) VALUES (1, 'k2', 'val2') IF NOT EXISTS; " +
+                           "INSERT INTO %1$s (id, k, v) VALUES (1, 'k3', 'val3') IF NOT EXISTS; " +
+                           "APPLY BATCH"),
+                   row(false, 1, "k2", null, "val2"));
+
+        assertRows(execute("BEGIN BATCH " +
+                           "UPDATE %1$s SET v = 'newVal' WHERE id = 1 AND k = 'k2' IF v = 'val0'; " +
+                           "INSERT INTO %1$s (id, k, v) VALUES (1, 'k3', 'val3') IF NOT EXISTS; " +
+                           "APPLY BATCH"),
+                   row(false, 1, "k2", null, "val2"));
+
+        assertRows(execute("SELECT * FROM %s WHERE id=1"),
+                   row(1, "k1", null, "val1"),
+                   row(1, "k2", null, "val2"));
+
+        assertRows(execute("BEGIN BATCH " +
+                           "UPDATE %1$s SET v = 'newVal' WHERE id = 1 AND k = 'k2' IF v = 'val2'; " +
+                           "INSERT INTO %1$s (id, k, v, version) VALUES(1, 'k3', 'val3', 1) IF NOT EXISTS; " +
+                           "APPLY BATCH"),
+                   row(true));
+
+        assertRows(execute("SELECT * FROM %s WHERE id=1"),
+                   row(1, "k1", 1, "val1"),
+                   row(1, "k2", 1, "newVal"),
+                   row(1, "k3", 1, "val3"));
+
+        assertRows(execute("BEGIN BATCH " +
+                           "UPDATE %1$s SET v = 'newVal1' WHERE id = 1 AND k = 'k2' IF v = 'val2'; " +
+                           "UPDATE %1$s SET v = 'newVal2' WHERE id = 1 AND k = 'k2' IF v = 'val3'; " +
+                           "APPLY BATCH"),
+                   row(false, 1, "k2", "newVal"));
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.bug_6069_test()
+     */
+    @Test
+    public void testInsertSetIfNotExists() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, s set<int>)");
+
+        assertRows(execute("INSERT INTO %s (k, s) VALUES (0, {1, 2, 3}) IF NOT EXISTS"),
+                   row(true));
+        assertRows(execute("SELECT * FROM %s "), row(0, set(1, 2, 3)));
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.cas_and_ttl_test()
+     */
+    @Test
+    public void testCasAndTTL() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, v int, lock boolean)");
+
+        execute("INSERT INTO %s (k, v, lock) VALUES (0, 0, false)");
+        execute("UPDATE %s USING TTL 1 SET lock=true WHERE k=0");
+
+        Thread.sleep(1001);
+        assertRows(execute("UPDATE %s SET v = 1 WHERE k = 0 IF lock = null"),
+                   row(true));
+    }
+
+    /**
+     * Test for CAS with compact storage table, and #6813 in particular,
+     * migrated from cql_tests.py:TestCQL.cas_and_compact_test()
+     */
+    @Test
+    public void testCompactStorage() throws Throwable
+    {
+        createTable("CREATE TABLE %s (partition text, key text, owner text, PRIMARY KEY (partition, key) ) WITH COMPACT STORAGE");
+
+        execute("INSERT INTO %s (partition, key, owner) VALUES ('a', 'b', null)");
+        assertRows(execute("UPDATE %s SET owner='z' WHERE partition='a' AND key='b' IF owner=null"), row(true));
+
+        assertRows(execute("UPDATE %s SET owner='b' WHERE partition='a' AND key='b' IF owner='a'"), row(false, "z"));
+        assertRows(execute("UPDATE %s SET owner='b' WHERE partition='a' AND key='b' IF owner='z'"), row(true));
+
+        assertRows(execute("INSERT INTO %s (partition, key, owner) VALUES ('a', 'c', 'x') IF NOT EXISTS"), row(true));
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.whole_list_conditional_test()
+     */
+    @Test
+    public void testWholeList() throws Throwable
+    {
+        for (boolean frozen : new boolean[] {false, true})
+        {
+            createTable(String.format("CREATE TABLE %%s (k int PRIMARY KEY, l %s)",
+                                      frozen
+                                      ? "frozen<list<text>>"
+                                      : "list<text>"));
+
+            execute("INSERT INTO %s(k, l) VALUES (0, ['foo', 'bar', 'foobar'])");
+
+            check_applies_list("l = ['foo', 'bar', 'foobar']");
+            check_applies_list("l != ['baz']");
+            check_applies_list("l > ['a']");
+            check_applies_list("l >= ['a']");
+            check_applies_list("l < ['z']");
+            check_applies_list("l <= ['z']");
+            check_applies_list("l IN (null, ['foo', 'bar', 'foobar'], ['a'])");
+
+            // multiple conditions
+            check_applies_list("l > ['aaa', 'bbb'] AND l > ['aaa']");
+            check_applies_list("l != null AND l IN (['foo', 'bar', 'foobar'])");
+
+            // should not apply
+            check_does_not_apply_list("l = ['baz']");
+            check_does_not_apply_list("l != ['foo', 'bar', 'foobar']");
+            check_does_not_apply_list("l > ['z']");
+            check_does_not_apply_list("l >= ['z']");
+            check_does_not_apply_list("l < ['a']");
+            check_does_not_apply_list("l <= ['a']");
+            check_does_not_apply_list("l IN (['a'], null)");
+            check_does_not_apply_list("l IN ()");
+
+            // multiple conditions
+            check_does_not_apply_list("l IN () AND l IN (['foo', 'bar', 'foobar'])");
+            check_does_not_apply_list("l > ['zzz'] AND l < ['zzz']");
+
+            check_invalid_list("l = [null]", InvalidRequestException.class);
+            check_invalid_list("l < null", InvalidRequestException.class);
+            check_invalid_list("l <= null", InvalidRequestException.class);
+            check_invalid_list("l > null", InvalidRequestException.class);
+            check_invalid_list("l >= null", InvalidRequestException.class);
+            check_invalid_list("l IN null", SyntaxException.class);
+            check_invalid_list("l IN 367", SyntaxException.class);
+            check_invalid_list("l CONTAINS KEY 123", SyntaxException.class);
+
+            // not supported yet
+            check_invalid_list("m CONTAINS 'bar'", SyntaxException.class);
+        }
+    }
+
+    void check_applies_list(String condition) throws Throwable
+    {
+        assertRows(execute("UPDATE %s SET l = ['foo', 'bar', 'foobar'] WHERE k=0 IF " + condition), row(true));
+        assertRows(execute("SELECT * FROM %s"), row(0, list("foo", "bar", "foobar")));
+    }
+
+    void check_does_not_apply_list(String condition) throws Throwable
+    {
+        assertRows(execute("UPDATE %s SET l = ['foo', 'bar', 'foobar'] WHERE k=0 IF " + condition),
+                   row(false, list("foo", "bar", "foobar")));
+        assertRows(execute("SELECT * FROM %s"), row(0, list("foo", "bar", "foobar")));
+    }
+
+    void check_invalid_list(String condition, Class<? extends Throwable> expected) throws Throwable
+    {
+        assertInvalidThrow(expected, "UPDATE %s SET l = ['foo', 'bar', 'foobar'] WHERE k=0 IF " + condition);
+        assertRows(execute("SELECT * FROM %s"), row(0, list("foo", "bar", "foobar")));
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.list_item_conditional_test()
+     */
+    @Test
+    public void testListItem() throws Throwable
+    {
+        for (boolean frozen : new boolean[]{ false, true })
+        {
+            createTable(String.format("CREATE TABLE %%s (k int PRIMARY KEY, l %s)",
+                                      frozen
+                                      ? "frozen<list<text>>"
+                                      : "list<text>"));
+
+            execute("INSERT INTO %s(k, l) VALUES (0, ['foo', 'bar', 'foobar'])");
+
+            assertInvalid("DELETE FROM %s WHERE k=0 IF l[null] = 'foobar'");
+            assertInvalid("DELETE FROM %s WHERE k=0 IF l[-2] = 'foobar'");
+
+            assertRows(execute("DELETE FROM %s WHERE k=0 IF l[1] = null"), row(false, list("foo", "bar", "foobar")));
+            assertRows(execute("DELETE FROM %s WHERE k=0 IF l[1] = 'foobar'"), row(false, list("foo", "bar", "foobar")));
+            assertRows(execute("SELECT * FROM %s"), row(0, list("foo", "bar", "foobar")));
+
+            assertRows(execute("DELETE FROM %s WHERE k=0 IF l[1] = 'bar'"), row(true));
+            assertEmpty(execute("SELECT * FROM %s"));
+        }
+    }
+
+    /**
+     * Test expanded functionality from CASSANDRA-6839,
+     * migrated from cql_tests.py:TestCQL.expanded_list_item_conditional_test()
+     */
+    @Test
+    public void testExpandedListItem() throws Throwable
+    {
+        for (boolean frozen : new boolean[] {false, true})
+        {
+            createTable(String.format("CREATE TABLE %%s (k int PRIMARY KEY, l %s)",
+                                      frozen
+                                      ? "frozen<list<text>>"
+                                      : "list<text>"));
+
+            execute("INSERT INTO %s (k, l) VALUES (0, ['foo', 'bar', 'foobar'])");
+
+            check_applies_list("l[1] < 'zzz'");
+            check_applies_list("l[1] <= 'bar'");
+            check_applies_list("l[1] > 'aaa'");
+            check_applies_list("l[1] >= 'bar'");
+            check_applies_list("l[1] != 'xxx'");
+            check_applies_list("l[1] != null");
+            check_applies_list("l[1] IN (null, 'xxx', 'bar')");
+            check_applies_list("l[1] > 'aaa' AND l[1] < 'zzz'");
+
+            // check beyond end of list
+            check_applies_list("l[3] = null");
+            check_applies_list("l[3] IN (null, 'xxx', 'bar')");
+
+            check_does_not_apply_list("l[1] < 'aaa'");
+            check_does_not_apply_list("l[1] <= 'aaa'");
+            check_does_not_apply_list("l[1] > 'zzz'");
+            check_does_not_apply_list("l[1] >= 'zzz'");
+            check_does_not_apply_list("l[1] != 'bar'");
+            check_does_not_apply_list("l[1] IN (null, 'xxx')");
+            check_does_not_apply_list("l[1] IN ()");
+            check_does_not_apply_list("l[1] != null AND l[1] IN ()");
+
+            // check beyond end of list
+            check_does_not_apply_list("l[3] != null");
+            check_does_not_apply_list("l[3] = 'xxx'");
+
+            check_invalid_list("l[1] < null", InvalidRequestException.class);
+            check_invalid_list("l[1] <= null", InvalidRequestException.class);
+            check_invalid_list("l[1] > null", InvalidRequestException.class);
+            check_invalid_list("l[1] >= null", InvalidRequestException.class);
+            check_invalid_list("l[1] IN null", SyntaxException.class);
+            check_invalid_list("l[1] IN 367", SyntaxException.class);
+            check_invalid_list("l[1] IN (1, 2, 3)", InvalidRequestException.class);
+            check_invalid_list("l[1] CONTAINS 367", SyntaxException.class);
+            check_invalid_list("l[1] CONTAINS KEY 367", SyntaxException.class);
+            check_invalid_list("l[null] = null", InvalidRequestException.class);
+        }
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.whole_set_conditional_test()
+     */
+    @Test
+    public void testWholeSet() throws Throwable
+    {
+        for (boolean frozen : new boolean[] {false, true})
+        {
+            createTable(String.format("CREATE TABLE %%s (k int PRIMARY KEY, s %s)",
+                                      frozen
+                                      ? "frozen<set<text>>"
+                                      : "set<text>"));
+
+            execute("INSERT INTO %s (k, s) VALUES (0, {'bar', 'foo'})");
+
+            check_applies_set("s = {'bar', 'foo'}");
+            check_applies_set("s = {'foo', 'bar'}");
+            check_applies_set("s != {'baz'}");
+            check_applies_set("s > {'a'}");
+            check_applies_set("s >= {'a'}");
+            check_applies_set("s < {'z'}");
+            check_applies_set("s <= {'z'}");
+            check_applies_set("s IN (null, {'bar', 'foo'}, {'a'})");
+
+            // multiple conditions
+            check_applies_set("s > {'a'} AND s < {'z'}");
+            check_applies_set("s IN (null, {'bar', 'foo'}, {'a'}) AND s IN ({'a'}, {'bar', 'foo'}, null)");
+
+            // should not apply
+            check_does_not_apply_set("s = {'baz'}");
+            check_does_not_apply_set("s != {'bar', 'foo'}");
+            check_does_not_apply_set("s > {'z'}");
+            check_does_not_apply_set("s >= {'z'}");
+            check_does_not_apply_set("s < {'a'}");
+            check_does_not_apply_set("s <= {'a'}");
+            check_does_not_apply_set("s IN ({'a'}, null)");
+            check_does_not_apply_set("s IN ()");
+            check_does_not_apply_set("s != null AND s IN ()");
+
+            check_invalid_set("s = {null}", InvalidRequestException.class);
+            check_invalid_set("s < null", InvalidRequestException.class);
+            check_invalid_set("s <= null", InvalidRequestException.class);
+            check_invalid_set("s > null", InvalidRequestException.class);
+            check_invalid_set("s >= null", InvalidRequestException.class);
+            check_invalid_set("s IN null", SyntaxException.class);
+            check_invalid_set("s IN 367", SyntaxException.class);
+            check_invalid_set("s CONTAINS KEY 123", SyntaxException.class);
+
+            // element access is not allow for sets
+            check_invalid_set("s['foo'] = 'foobar'", InvalidRequestException.class);
+
+            // not supported yet
+            check_invalid_set("m CONTAINS 'bar'", SyntaxException.class);
+        }
+    }
+
+    void check_applies_set(String condition) throws Throwable
+    {
+        assertRows(execute("UPDATE %s SET s = {'bar', 'foo'} WHERE k=0 IF " + condition), row(true));
+        assertRows(execute("SELECT * FROM %s"), row(0, set("bar", "foo")));
+    }
+
+    void check_does_not_apply_set(String condition) throws Throwable
+    {
+        assertRows(execute("UPDATE %s SET s = {'bar', 'foo'} WHERE k=0 IF " + condition), row(false, set("bar", "foo")));
+        assertRows(execute("SELECT * FROM %s"), row(0, set("bar", "foo")));
+    }
+
+    void check_invalid_set(String condition, Class<? extends Throwable> expected) throws Throwable
+    {
+        assertInvalidThrow(expected, "UPDATE %s SET s = {'bar', 'foo'} WHERE k=0 IF " + condition);
+        assertRows(execute("SELECT * FROM %s"), row(0, set("bar", "foo")));
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.whole_map_conditional_test()
+     */
+    @Test
+    public void testWholeMap() throws Throwable
+    {
+        for (boolean frozen : new boolean[] {false, true})
+        {
+            createTable(String.format("CREATE TABLE %%s (k int PRIMARY KEY, m %s)",
+                                      frozen
+                                      ? "frozen<map<text, text>>"
+                                      : "map<text, text>"));
+
+            execute("INSERT INTO %s (k, m) VALUES (0, {'foo' : 'bar'})");
+
+            check_applies_map("m = {'foo': 'bar'}");
+            check_applies_map("m > {'a': 'a'}");
+            check_applies_map("m >= {'a': 'a'}");
+            check_applies_map("m < {'z': 'z'}");
+            check_applies_map("m <= {'z': 'z'}");
+            check_applies_map("m != {'a': 'a'}");
+            check_applies_map("m IN (null, {'a': 'a'}, {'foo': 'bar'})");
+
+            // multiple conditions
+            check_applies_map("m > {'a': 'a'} AND m < {'z': 'z'}");
+            check_applies_map("m != null AND m IN (null, {'a': 'a'}, {'foo': 'bar'})");
+
+            // should not apply
+            check_does_not_apply_map("m = {'a': 'a'}");
+            check_does_not_apply_map("m > {'z': 'z'}");
+            check_does_not_apply_map("m >= {'z': 'z'}");
+            check_does_not_apply_map("m < {'a': 'a'}");
+            check_does_not_apply_map("m <= {'a': 'a'}");
+            check_does_not_apply_map("m != {'foo': 'bar'}");
+            check_does_not_apply_map("m IN ({'a': 'a'}, null)");
+            check_does_not_apply_map("m IN ()");
+            check_does_not_apply_map("m = null AND m != null");
+
+            check_invalid_map("m = {null: null}", InvalidRequestException.class);
+            check_invalid_map("m = {'a': null}", InvalidRequestException.class);
+            check_invalid_map("m = {null: 'a'}", InvalidRequestException.class);
+            check_invalid_map("m < null", InvalidRequestException.class);
+            check_invalid_map("m IN null", SyntaxException.class);
+
+            // not supported yet
+            check_invalid_map("m CONTAINS 'bar'", SyntaxException.class);
+            check_invalid_map("m CONTAINS KEY 'foo'", SyntaxException.class);
+            check_invalid_map("m CONTAINS null", SyntaxException.class);
+            check_invalid_map("m CONTAINS KEY null", SyntaxException.class);
+        }
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.map_item_conditional_test()
+     */
+    @Test
+    public void testMapItem() throws Throwable
+    {
+        for (boolean frozen : new boolean[]{ false, true })
+        {
+            createTable(String.format("CREATE TABLE %%s (k int PRIMARY KEY, m %s)",
+                                      frozen
+                                      ? "frozen<map<text, text>>"
+                                      : "map<text, text>"));
+
+            execute("INSERT INTO %s (k, m) VALUES (0, {'foo' : 'bar'})");
+            assertInvalid("DELETE FROM %s WHERE k=0 IF m[null] = 'foo'");
+            assertRows(execute("DELETE FROM %s WHERE k=0 IF m['foo'] = 'foo'"), row(false, map("foo", "bar")));
+            assertRows(execute("DELETE FROM %s WHERE k=0 IF m['foo'] = null"), row(false, map("foo", "bar")));
+            assertRows(execute("SELECT * FROM %s"), row(0, map("foo", "bar")));
+
+            assertRows(execute("DELETE FROM %s WHERE k=0 IF m['foo'] = 'bar'"), row(true));
+            assertEmpty(execute("SELECT * FROM %s"));
+
+            execute("INSERT INTO %s(k, m) VALUES (1, null)");
+            if (frozen)
+                assertInvalid("UPDATE %s set m['foo'] = 'bar', m['bar'] = 'foo' WHERE k = 1 IF m['foo'] IN ('blah', null)");
+            else
+                assertRows(execute("UPDATE %s set m['foo'] = 'bar', m['bar'] = 'foo' WHERE k = 1 IF m['foo'] IN ('blah', null)"), row(true));
+        }
+    }
+
+    /**
+     * Test expanded functionality from CASSANDRA-6839,
+     * migrated from cql_tests.py:TestCQL.expanded_map_item_conditional_test()
+     */
+    @Test
+    public void testExpandedMapItem() throws Throwable
+    {
+        for (boolean frozen : new boolean[]{ false, true })
+        {
+            createTable(String.format("CREATE TABLE %%s (k int PRIMARY KEY, m %s)",
+                                      frozen
+                                      ? "frozen<map<text, text>>"
+                                      : "map<text, text>"));
+
+            execute("INSERT INTO %s (k, m) VALUES (0, {'foo' : 'bar'})");
+
+            check_applies_map("m['xxx'] = null");
+            check_applies_map("m['foo'] < 'zzz'");
+            check_applies_map("m['foo'] <= 'bar'");
+            check_applies_map("m['foo'] > 'aaa'");
+            check_applies_map("m['foo'] >= 'bar'");
+            check_applies_map("m['foo'] != 'xxx'");
+            check_applies_map("m['foo'] != null");
+            check_applies_map("m['foo'] IN (null, 'xxx', 'bar')");
+            check_applies_map("m['xxx'] IN (null, 'xxx', 'bar')"); // m['xxx'] is not set
+
+            // multiple conditions
+            check_applies_map("m['foo'] < 'zzz' AND m['foo'] > 'aaa'");
+
+            check_does_not_apply_map("m['foo'] < 'aaa'");
+            check_does_not_apply_map("m['foo'] <= 'aaa'");
+            check_does_not_apply_map("m['foo'] > 'zzz'");
+            check_does_not_apply_map("m['foo'] >= 'zzz'");
+            check_does_not_apply_map("m['foo'] != 'bar'");
+            check_does_not_apply_map("m['xxx'] != null");  // m['xxx'] is not set
+            check_does_not_apply_map("m['foo'] IN (null, 'xxx')");
+            check_does_not_apply_map("m['foo'] IN ()");
+            check_does_not_apply_map("m['foo'] != null AND m['foo'] = null");
+
+            check_invalid_map("m['foo'] < null", InvalidRequestException.class);
+            check_invalid_map("m['foo'] <= null", InvalidRequestException.class);
+            check_invalid_map("m['foo'] > null", InvalidRequestException.class);
+            check_invalid_map("m['foo'] >= null", InvalidRequestException.class);
+            check_invalid_map("m['foo'] IN null", SyntaxException.class);
+            check_invalid_map("m['foo'] IN 367", SyntaxException.class);
+            check_invalid_map("m['foo'] IN (1, 2, 3)", InvalidRequestException.class);
+            check_invalid_map("m['foo'] CONTAINS 367", SyntaxException.class);
+            check_invalid_map("m['foo'] CONTAINS KEY 367", SyntaxException.class);
+            check_invalid_map("m[null] = null", InvalidRequestException.class);
+        }
+    }
+
+    void check_applies_map(String condition) throws Throwable
+    {
+        assertRows(execute("UPDATE %s SET m = {'foo': 'bar'} WHERE k=0 IF " + condition), row(true));
+        assertRows(execute("SELECT * FROM %s"), row(0, map("foo", "bar")));
+    }
+
+    void check_does_not_apply_map(String condition) throws Throwable
+    {
+        assertRows(execute("UPDATE %s SET m = {'foo': 'bar'} WHERE k=0 IF " + condition), row(false, map("foo", "bar")));
+        assertRows(execute("SELECT * FROM %s"), row(0, map("foo", "bar")));
+    }
+
+    void check_invalid_map(String condition, Class<? extends Throwable> expected) throws Throwable
+    {
+        assertInvalidThrow(expected, "UPDATE %s SET m = {'foo': 'bar'} WHERE k=0 IF " + condition);
+        assertRows(execute("SELECT * FROM %s"), row(0, map("foo", "bar")));
+    }
+    
+    /**
+     * Test for 7499,
+     * migrated from cql_tests.py:TestCQL.cas_and_list_index_test()
+     */
+    @Test
+    public void testCasAndListIndex() throws Throwable
+    {
+        createTable("CREATE TABLE %s ( k int PRIMARY KEY, v text, l list<text>)");
+
+        execute("INSERT INTO %s (k, v, l) VALUES(0, 'foobar', ['foi', 'bar'])");
+
+        assertRows(execute("UPDATE %s SET l[0] = 'foo' WHERE k = 0 IF v = 'barfoo'"), row(false, "foobar"));
+        assertRows(execute("UPDATE %s SET l[0] = 'foo' WHERE k = 0 IF v = 'foobar'"), row(true));
+
+        assertRows(execute("SELECT * FROM %s"), row(0, list("foo", "bar"), "foobar"));
+
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.conditional_ddl_keyspace_test()
+     */
+    @Test
+    public void testDropCreateKeyspaceIfNotExists() throws Throwable
+    {
+        String keyspace =  KEYSPACE;
+
+        schemaChange("DROP KEYSPACE IF EXISTS " + keyspace);
+
+        // try dropping when doesn't exist
+        schemaChange("DROP KEYSPACE IF EXISTS " + keyspace);
+
+        // create and confirm
+        schemaChange("CREATE KEYSPACE IF NOT EXISTS " + keyspace + " WITH replication = { 'class':'SimpleStrategy', 'replication_factor':1} and durable_writes = true ");
+        assertRows(execute("select durable_writes from system.schema_keyspaces where keyspace_name = ?", keyspace), row(true));
+
+        // unsuccessful create since it's already there, confirm settings don't change
+        schemaChange("CREATE KEYSPACE IF NOT EXISTS " + keyspace + " WITH replication = {'class':'SimpleStrategy', 'replication_factor':1} and durable_writes = false ");
+
+        assertRows(execute("select durable_writes from system.schema_keyspaces where keyspace_name = ?", keyspace), row(true));
+
+        // drop and confirm
+        schemaChange("DROP KEYSPACE IF EXISTS " + keyspace);
+
+        assertEmpty(execute("select * from system.schema_keyspaces where keyspace_name = ?", keyspace));
+    }
+
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.conditional_ddl_table_test()
+     */
+    @Test
+    public void testDropCreateTableIfNotExists() throws Throwable
+    {
+        String tableName = createTableName();
+        String fullTableName = KEYSPACE + "." + tableName;
+
+        // try dropping when doesn't exist
+        schemaChange("DROP TABLE IF EXISTS " + fullTableName);
+
+        // create and confirm
+        schemaChange("CREATE TABLE IF NOT EXISTS " + fullTableName + " (id text PRIMARY KEY, value1 blob) with comment = 'foo'");
+
+        assertRows(execute("select comment from system.schema_columnfamilies where keyspace_name = ? and columnfamily_name = ?", KEYSPACE, tableName),
+                   row("foo"));
+
+        // unsuccessful create since it's already there, confirm settings don't change
+        schemaChange("CREATE TABLE IF NOT EXISTS " + fullTableName + " (id text PRIMARY KEY, value2 blob)with comment = 'bar'");
+
+        assertRows(execute("select comment from system.schema_columnfamilies where keyspace_name = ? and columnfamily_name = ?", KEYSPACE, tableName),
+                   row("foo"));
+
+        // drop and confirm
+        schemaChange("DROP TABLE IF EXISTS " + fullTableName);
+
+        assertEmpty(execute("select * from system.schema_columnfamilies where keyspace_name = ? and columnfamily_name = ?", KEYSPACE, tableName));
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.conditional_ddl_index_test()
+     */
+    @Test
+    public void testDropCreateIndexIfNotExists() throws Throwable
+    {
+        String tableName = createTable("CREATE TABLE %s (id text PRIMARY KEY, value1 blob, value2 blob)with comment = 'foo'");
+
+        execute("use " + KEYSPACE);
+
+        // try dropping when doesn't exist
+        schemaChange("DROP INDEX IF EXISTS myindex");
+
+        // create and confirm
+        createIndex("CREATE INDEX IF NOT EXISTS myindex ON %s (value1)");
+
+        assertTrue(waitForIndex(KEYSPACE, tableName, "myindex"));
+
+        // unsuccessful create since it's already there
+        execute("CREATE INDEX IF NOT EXISTS myindex ON %s (value1)");
+
+        // drop and confirm
+        execute("DROP INDEX IF EXISTS myindex");
+
+        Object[][] rows = getRows(execute("select index_name from system.\"IndexInfo\" where table_name = ?", tableName));
+        assertEquals(0, rows.length);
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.conditional_ddl_type_test()
+     */
+    @Test
+    public void testDropCreateTypeIfNotExists() throws Throwable
+    {
+        execute("use " + KEYSPACE);
+
+        // try dropping when doesn 't exist
+        execute("DROP TYPE IF EXISTS mytype");
+
+        // create and confirm
+        execute("CREATE TYPE IF NOT EXISTS mytype (somefield int)");
+        assertRows(execute("SELECT type_name from system.schema_usertypes where keyspace_name = ? and type_name = ?", KEYSPACE, "mytype"),
+                   row("mytype"));
+
+        // unsuccessful create since it 's already there
+        // TODO: confirm this create attempt doesn't alter type field from int to blob
+        execute("CREATE TYPE IF NOT EXISTS mytype (somefield blob)");
+
+        // drop and confirm
+        execute("DROP TYPE IF EXISTS mytype");
+        assertEmpty(execute("SELECT type_name from system.schema_usertypes where keyspace_name = ? and type_name = ?", KEYSPACE, "mytype"));
+    }
+}


[11/32] cassandra git commit: 2.2 commit for CASSANDRA-9160

Posted by jm...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/01115f72/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
new file mode 100644
index 0000000..506bdaf
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/SelectTest.java
@@ -0,0 +1,1336 @@
+/*
+ * 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.validation.operations;
+
+import java.nio.ByteBuffer;
+import java.util.UUID;
+
+import org.junit.Test;
+
+import junit.framework.Assert;
+import org.apache.cassandra.cql3.UntypedResultSet;
+import org.apache.cassandra.cql3.CQLTester;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Test column ranges and ordering with static column in table
+ */
+public class SelectTest extends CQLTester
+{
+    @Test
+    public void testSingleClustering() throws Throwable
+    {
+        createTable("CREATE TABLE %s (p text, c text, v text, s text static, PRIMARY KEY (p, c))");
+
+        execute("INSERT INTO %s(p, c, v, s) values (?, ?, ?, ?)", "p1", "k1", "v1", "sv1");
+        execute("INSERT INTO %s(p, c, v) values (?, ?, ?)", "p1", "k2", "v2");
+        execute("INSERT INTO %s(p, s) values (?, ?)", "p2", "sv2");
+
+        assertRows(execute("SELECT * FROM %s WHERE p=?", "p1"),
+            row("p1", "k1", "sv1", "v1"),
+            row("p1", "k2", "sv1", "v2")
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE p=?", "p2"),
+            row("p2", null, "sv2", null)
+        );
+
+        // Ascending order
+
+        assertRows(execute("SELECT * FROM %s WHERE p=? ORDER BY c ASC", "p1"),
+            row("p1", "k1", "sv1", "v1"),
+            row("p1", "k2", "sv1", "v2")
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE p=? ORDER BY c ASC", "p2"),
+            row("p2", null, "sv2", null)
+        );
+
+        // Descending order
+
+        assertRows(execute("SELECT * FROM %s WHERE p=? ORDER BY c DESC", "p1"),
+            row("p1", "k2", "sv1", "v2"),
+            row("p1", "k1", "sv1", "v1")
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE p=? ORDER BY c DESC", "p2"),
+            row("p2", null, "sv2", null)
+        );
+
+        // No order with one relation
+
+        assertRows(execute("SELECT * FROM %s WHERE p=? AND c>=?", "p1", "k1"),
+            row("p1", "k1", "sv1", "v1"),
+            row("p1", "k2", "sv1", "v2")
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE p=? AND c>=?", "p1", "k2"),
+            row("p1", "k2", "sv1", "v2")
+        );
+
+        assertEmpty(execute("SELECT * FROM %s WHERE p=? AND c>=?", "p1", "k3"));
+
+        assertRows(execute("SELECT * FROM %s WHERE p=? AND c =?", "p1", "k1"),
+            row("p1", "k1", "sv1", "v1")
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE p=? AND c<=?", "p1", "k1"),
+            row("p1", "k1", "sv1", "v1")
+        );
+
+        assertEmpty(execute("SELECT * FROM %s WHERE p=? AND c<=?", "p1", "k0"));
+
+        // Ascending with one relation
+
+        assertRows(execute("SELECT * FROM %s WHERE p=? AND c>=? ORDER BY c ASC", "p1", "k1"),
+            row("p1", "k1", "sv1", "v1"),
+            row("p1", "k2", "sv1", "v2")
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE p=? AND c>=? ORDER BY c ASC", "p1", "k2"),
+            row("p1", "k2", "sv1", "v2")
+        );
+
+        assertEmpty(execute("SELECT * FROM %s WHERE p=? AND c>=? ORDER BY c ASC", "p1", "k3"));
+
+        assertRows(execute("SELECT * FROM %s WHERE p=? AND c =? ORDER BY c ASC", "p1", "k1"),
+            row("p1", "k1", "sv1", "v1")
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE p=? AND c<=? ORDER BY c ASC", "p1", "k1"),
+            row("p1", "k1", "sv1", "v1")
+        );
+
+        assertEmpty(execute("SELECT * FROM %s WHERE p=? AND c<=? ORDER BY c ASC", "p1", "k0"));
+
+        // Descending with one relation
+
+        assertRows(execute("SELECT * FROM %s WHERE p=? AND c>=? ORDER BY c DESC", "p1", "k1"),
+            row("p1", "k2", "sv1", "v2"),
+            row("p1", "k1", "sv1", "v1")
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE p=? AND c>=? ORDER BY c DESC", "p1", "k2"),
+            row("p1", "k2", "sv1", "v2")
+        );
+
+        assertEmpty(execute("SELECT * FROM %s WHERE p=? AND c>=? ORDER BY c DESC", "p1", "k3"));
+
+        assertRows(execute("SELECT * FROM %s WHERE p=? AND c =? ORDER BY c DESC", "p1", "k1"),
+            row("p1", "k1", "sv1", "v1")
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE p=? AND c<=? ORDER BY c DESC", "p1", "k1"),
+            row("p1", "k1", "sv1", "v1")
+        );
+
+        assertEmpty(execute("SELECT * FROM %s WHERE p=? AND c<=? ORDER BY c DESC", "p1", "k0"));
+
+        // IN
+
+        assertRows(execute("SELECT * FROM %s WHERE p=? AND c IN (?, ?)", "p1", "k1", "k2"),
+            row("p1", "k1", "sv1", "v1"),
+            row("p1", "k2", "sv1", "v2")
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE p=? AND c IN (?, ?) ORDER BY c ASC", "p1", "k1", "k2"),
+            row("p1", "k1", "sv1", "v1"),
+            row("p1", "k2", "sv1", "v2")
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE p=? AND c IN (?, ?) ORDER BY c DESC", "p1", "k1", "k2"),
+            row("p1", "k2", "sv1", "v2"),
+            row("p1", "k1", "sv1", "v1")
+        );
+    }
+
+    @Test
+    public void testSingleClusteringReversed() throws Throwable
+    {
+        createTable("CREATE TABLE %s (p text, c text, v text, s text static, PRIMARY KEY (p, c)) WITH CLUSTERING ORDER BY (c DESC)");
+
+        execute("INSERT INTO %s(p, c, v, s) values (?, ?, ?, ?)", "p1", "k1", "v1", "sv1");
+        execute("INSERT INTO %s(p, c, v) values (?, ?, ?)", "p1", "k2", "v2");
+        execute("INSERT INTO %s(p, s) values (?, ?)", "p2", "sv2");
+
+        assertRows(execute("SELECT * FROM %s WHERE p=?", "p1"),
+            row("p1", "k2", "sv1", "v2"),
+            row("p1", "k1", "sv1", "v1")
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE p=?", "p2"),
+            row("p2", null, "sv2", null)
+        );
+
+        // Ascending order
+
+        assertRows(execute("SELECT * FROM %s WHERE p=? ORDER BY c ASC", "p1"),
+            row("p1", "k1", "sv1", "v1"),
+            row("p1", "k2", "sv1", "v2")
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE p=? ORDER BY c ASC", "p2"),
+            row("p2", null, "sv2", null)
+        );
+
+        // Descending order
+
+        assertRows(execute("SELECT * FROM %s WHERE p=? ORDER BY c DESC", "p1"),
+            row("p1", "k2", "sv1", "v2"),
+            row("p1", "k1", "sv1", "v1")
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE p=? ORDER BY c DESC", "p2"),
+            row("p2", null, "sv2", null)
+        );
+
+        // No order with one relation
+
+        assertRows(execute("SELECT * FROM %s WHERE p=? AND c>=?", "p1", "k1"),
+            row("p1", "k2", "sv1", "v2"),
+            row("p1", "k1", "sv1", "v1")
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE p=? AND c>=?", "p1", "k2"),
+            row("p1", "k2", "sv1", "v2")
+        );
+
+        assertEmpty(execute("SELECT * FROM %s WHERE p=? AND c>=?", "p1", "k3"));
+
+        assertRows(execute("SELECT * FROM %s WHERE p=? AND c=?", "p1", "k1"),
+            row("p1", "k1", "sv1", "v1")
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE p=? AND c<=?", "p1", "k1"),
+            row("p1", "k1", "sv1", "v1")
+        );
+
+        assertEmpty(execute("SELECT * FROM %s WHERE p=? AND c<=?", "p1", "k0"));
+
+        // Ascending with one relation
+
+        assertRows(execute("SELECT * FROM %s WHERE p=? AND c>=? ORDER BY c ASC", "p1", "k1"),
+            row("p1", "k1", "sv1", "v1"),
+            row("p1", "k2", "sv1", "v2")
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE p=? AND c>=? ORDER BY c ASC", "p1", "k2"),
+            row("p1", "k2", "sv1", "v2")
+        );
+
+        assertEmpty(execute("SELECT * FROM %s WHERE p=? AND c>=? ORDER BY c ASC", "p1", "k3"));
+
+        assertRows(execute("SELECT * FROM %s WHERE p=? AND c=? ORDER BY c ASC", "p1", "k1"),
+            row("p1", "k1", "sv1", "v1")
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE p=? AND c<=? ORDER BY c ASC", "p1", "k1"),
+            row("p1", "k1", "sv1", "v1")
+        );
+
+        assertEmpty(execute("SELECT * FROM %s WHERE p=? AND c<=? ORDER BY c ASC", "p1", "k0"));
+
+        // Descending with one relation
+
+        assertRows(execute("SELECT * FROM %s WHERE p=? AND c>=? ORDER BY c DESC", "p1", "k1"),
+            row("p1", "k2", "sv1", "v2"),
+            row("p1", "k1", "sv1", "v1")
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE p=? AND c>=? ORDER BY c DESC", "p1", "k2"),
+            row("p1", "k2", "sv1", "v2")
+        );
+
+        assertEmpty(execute("SELECT * FROM %s WHERE p=? AND c>=? ORDER BY c DESC", "p1", "k3"));
+
+        assertRows(execute("SELECT * FROM %s WHERE p=? AND c=? ORDER BY c DESC", "p1", "k1"),
+            row("p1", "k1", "sv1", "v1")
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE p=? AND c<=? ORDER BY c DESC", "p1", "k1"),
+            row("p1", "k1", "sv1", "v1")
+        );
+
+        assertEmpty(execute("SELECT * FROM %s WHERE p=? AND c<=? ORDER BY c DESC", "p1", "k0"));
+
+        // IN
+
+        assertRows(execute("SELECT * FROM %s WHERE p=? AND c IN (?, ?)", "p1", "k1", "k2"),
+            row("p1", "k2", "sv1", "v2"),
+            row("p1", "k1", "sv1", "v1")
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE p=? AND c IN (?, ?) ORDER BY c ASC", "p1", "k1", "k2"),
+            row("p1", "k1", "sv1", "v1"),
+            row("p1", "k2", "sv1", "v2")
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE p=? AND c IN (?, ?) ORDER BY c DESC", "p1", "k1", "k2"),
+            row("p1", "k2", "sv1", "v2"),
+            row("p1", "k1", "sv1", "v1")
+        );
+    }
+
+    /**
+     * Check query with KEY IN clause
+     * migrated from cql_tests.py:TestCQL.select_key_in_test()
+     */
+    @Test
+    public void testSelectKeyIn() throws Throwable
+    {
+        createTable("CREATE TABLE %s (userid uuid PRIMARY KEY, firstname text, lastname text, age int)");
+
+        UUID id1 = UUID.fromString("550e8400-e29b-41d4-a716-446655440000");
+        UUID id2 = UUID.fromString("f47ac10b-58cc-4372-a567-0e02b2c3d479");
+
+        execute("INSERT INTO %s (userid, firstname, lastname, age) VALUES (?, 'Frodo', 'Baggins', 32)", id1);
+        execute("INSERT INTO %s (userid, firstname, lastname, age) VALUES (?, 'Samwise', 'Gamgee', 33)", id2);
+
+        assertRowCount(execute("SELECT firstname, lastname FROM %s WHERE userid IN (?, ?)", id1, id2), 2);
+    }
+
+    /**
+     * Check query with KEY IN clause for wide row tables
+     * migrated from cql_tests.py:TestCQL.in_clause_wide_rows_test()
+     */
+    @Test
+    public void testSelectKeyInForWideRows() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int, c int, v int, PRIMARY KEY (k, c)) WITH COMPACT STORAGE");
+
+        for (int i = 0; i < 10; i++)
+            execute("INSERT INTO %s (k, c, v) VALUES (0, ?, ?)", i, i);
+
+        assertRows(execute("SELECT v FROM %s WHERE k = 0 AND c IN (5, 2, 8)"),
+                   row(2), row(5), row(8));
+
+        createTable("CREATE TABLE %s (k int, c1 int, c2 int, v int, PRIMARY KEY (k, c1, c2)) WITH COMPACT STORAGE");
+
+        for (int i = 0; i < 10; i++)
+            execute("INSERT INTO %s (k, c1, c2, v) VALUES (0, 0, ?, ?)", i, i);
+
+        assertEmpty(execute("SELECT v FROM %s WHERE k = 0 AND c1 IN (5, 2, 8) AND c2 = 3"));
+
+        assertRows(execute("SELECT v FROM %s WHERE k = 0 AND c1 = 0 AND c2 IN (5, 2, 8)"),
+                   row(2), row(5), row(8));
+    }
+
+    /**
+     * Check SELECT respects inclusive and exclusive bounds
+     * migrated from cql_tests.py:TestCQL.exclusive_slice_test()
+     */
+    @Test
+    public void testSelectBounds() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int, c int, v int, PRIMARY KEY (k, c)) WITH COMPACT STORAGE");
+
+        for (int i = 0; i < 10; i++)
+            execute("INSERT INTO %s (k, c, v) VALUES (0, ?, ?)", i, i);
+
+        assertRowCount(execute("SELECT v FROM %s WHERE k = 0"), 10);
+
+        assertRows(execute("SELECT v FROM %s WHERE k = 0 AND c >= 2 AND c <= 6"),
+                   row(2), row(3), row(4), row(5), row(6));
+
+        assertRows(execute("SELECT v FROM %s WHERE k = 0 AND c > 2 AND c <= 6"),
+                   row(3), row(4), row(5), row(6));
+
+        assertRows(execute("SELECT v FROM %s WHERE k = 0 AND c >= 2 AND c < 6"),
+                   row(2), row(3), row(4), row(5));
+
+        assertRows(execute("SELECT v FROM %s WHERE k = 0 AND c > 2 AND c < 6"),
+                   row(3), row(4), row(5));
+
+        assertRows(execute("SELECT v FROM %s WHERE k = 0 AND c > 2 AND c <= 6 LIMIT 2"),
+                   row(3), row(4));
+
+        assertRows(execute("SELECT v FROM %s WHERE k = 0 AND c >= 2 AND c < 6 ORDER BY c DESC LIMIT 2"),
+                   row(5), row(4));
+    }
+
+    @Test
+    public void testSetContains() throws Throwable
+    {
+        createTable("CREATE TABLE %s (account text, id int, categories set<text>, PRIMARY KEY (account, id))");
+        createIndex("CREATE INDEX ON %s(categories)");
+
+        execute("INSERT INTO %s (account, id , categories) VALUES (?, ?, ?)", "test", 5, set("lmn"));
+
+        assertEmpty(execute("SELECT * FROM %s WHERE account = ? AND categories CONTAINS ?", "xyz", "lmn"));
+
+        assertRows(execute("SELECT * FROM %s WHERE categories CONTAINS ?", "lmn"),
+                   row("test", 5, set("lmn"))
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE account = ? AND categories CONTAINS ?", "test", "lmn"),
+                   row("test", 5, set("lmn"))
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE account = ? AND id = ? AND categories CONTAINS ?", "test", 5, "lmn"),
+                   row("test", 5, set("lmn"))
+        );
+
+        assertInvalidMessage("Unsupported null value for indexed column categories",
+                             "SELECT * FROM %s WHERE account = ? AND id = ? AND categories CONTAINS ?", "test", 5, null);
+
+        assertInvalidMessage("Unsupported unset value for indexed column categories",
+                             "SELECT * FROM %s WHERE account = ? AND id = ? AND categories CONTAINS ?", "test", 5, unset());
+
+        assertInvalidMessage("Cannot execute this query as it might involve data filtering and thus may have unpredictable performance. If you want to execute this query despite the performance unpredictability, use ALLOW FILTERING",
+                             "SELECT * FROM %s WHERE account = ? AND categories CONTAINS ? AND categories CONTAINS ?", "xyz", "lmn", "notPresent");
+        assertEmpty(execute("SELECT * FROM %s WHERE account = ? AND categories CONTAINS ? AND categories CONTAINS ? ALLOW FILTERING", "xyz", "lmn", "notPresent"));
+    }
+
+    @Test
+    public void testListContains() throws Throwable
+    {
+        createTable("CREATE TABLE %s (account text, id int, categories list<text>, PRIMARY KEY (account, id))");
+        createIndex("CREATE INDEX ON %s(categories)");
+
+        execute("INSERT INTO %s (account, id , categories) VALUES (?, ?, ?)", "test", 5, list("lmn"));
+
+        assertEmpty(execute("SELECT * FROM %s WHERE account = ? AND categories CONTAINS ?", "xyz", "lmn"));
+
+        assertRows(execute("SELECT * FROM %s WHERE account = ? AND categories CONTAINS ?;", "test", "lmn"),
+                   row("test", 5, list("lmn"))
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE categories CONTAINS ?", "lmn"),
+                   row("test", 5, list("lmn"))
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE account = ? AND id = ? AND categories CONTAINS ?;", "test", 5, "lmn"),
+                   row("test", 5, list("lmn"))
+        );
+
+        assertInvalidMessage("Unsupported null value for indexed column categories",
+                             "SELECT * FROM %s WHERE account = ? AND id = ? AND categories CONTAINS ?", "test", 5, null);
+
+        assertInvalidMessage("Unsupported unset value for indexed column categories",
+                             "SELECT * FROM %s WHERE account = ? AND id = ? AND categories CONTAINS ?", "test", 5, unset());
+
+        assertInvalidMessage("Cannot execute this query as it might involve data filtering and thus may have unpredictable performance. If you want to execute this query despite the performance unpredictability, use ALLOW FILTERING",
+                             "SELECT * FROM %s WHERE account = ? AND id = ? AND categories CONTAINS ? AND categories CONTAINS ?",
+                             "test", 5, "lmn", "notPresent");
+        assertEmpty(execute("SELECT * FROM %s WHERE account = ? AND id = ? AND categories CONTAINS ? AND categories CONTAINS ? ALLOW FILTERING",
+                            "test", 5, "lmn", "notPresent"));
+    }
+
+    @Test
+    public void testListContainsWithFiltering() throws Throwable
+    {
+        createTable("CREATE TABLE %s (e int PRIMARY KEY, f list<text>, s int)");
+        createIndex("CREATE INDEX ON %s(f)");
+        for(int i = 0; i < 3; i++)
+        {
+            execute("INSERT INTO %s (e, f, s) VALUES (?, ?, ?)", i, list("Dubai"), 4);
+        }
+        for(int i = 3; i < 5; i++)
+        {
+            execute("INSERT INTO %s (e, f, s) VALUES (?, ?, ?)", i, list("Dubai"), 3);
+        }
+        assertRows(execute("SELECT * FROM %s WHERE f CONTAINS ? AND s=? allow filtering", "Dubai", 3),
+                   row(4, list("Dubai"), 3),
+                   row(3, list("Dubai"), 3));
+    }
+
+    @Test
+    public void testMapKeyContains() throws Throwable
+    {
+        createTable("CREATE TABLE %s (account text, id int, categories map<text,text>, PRIMARY KEY (account, id))");
+        createIndex("CREATE INDEX ON %s(keys(categories))");
+
+        execute("INSERT INTO %s (account, id , categories) VALUES (?, ?, ?)", "test", 5, map("lmn", "foo"));
+
+        assertEmpty(execute("SELECT * FROM %s WHERE account = ? AND categories CONTAINS KEY ?", "xyz", "lmn"));
+
+        assertRows(execute("SELECT * FROM %s WHERE account = ? AND categories CONTAINS KEY ?", "test", "lmn"),
+                   row("test", 5, map("lmn", "foo"))
+        );
+        assertRows(execute("SELECT * FROM %s WHERE categories CONTAINS KEY ?", "lmn"),
+                   row("test", 5, map("lmn", "foo"))
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE account = ? AND id = ? AND categories CONTAINS KEY ?", "test", 5, "lmn"),
+                   row("test", 5, map("lmn", "foo"))
+        );
+
+        assertInvalidMessage("Unsupported null value for indexed column categories",
+                             "SELECT * FROM %s WHERE account = ? AND id = ? AND categories CONTAINS KEY ?", "test", 5, null);
+
+        assertInvalidMessage("Unsupported unset value for indexed column categories",
+                             "SELECT * FROM %s WHERE account = ? AND id = ? AND categories CONTAINS KEY ?", "test", 5, unset());
+
+        assertInvalidMessage("Cannot execute this query as it might involve data filtering and thus may have unpredictable performance. If you want to execute this query despite the performance unpredictability, use ALLOW FILTERING",
+                             "SELECT * FROM %s WHERE account = ? AND id = ? AND categories CONTAINS KEY ? AND categories CONTAINS KEY ?",
+                             "test", 5, "lmn", "notPresent");
+        assertEmpty(execute("SELECT * FROM %s WHERE account = ? AND id = ? AND categories CONTAINS KEY ? AND categories CONTAINS KEY ? ALLOW FILTERING",
+                            "test", 5, "lmn", "notPresent"));
+
+        assertInvalidMessage("Cannot execute this query as it might involve data filtering and thus may have unpredictable performance. If you want to execute this query despite the performance unpredictability, use ALLOW FILTERING",
+                             "SELECT * FROM %s WHERE account = ? AND id = ? AND categories CONTAINS KEY ? AND categories CONTAINS ?",
+                             "test", 5, "lmn", "foo");
+    }
+
+    @Test
+    public void testMapValueContains() throws Throwable
+    {
+        createTable("CREATE TABLE %s (account text, id int, categories map<text,text>, PRIMARY KEY (account, id))");
+        createIndex("CREATE INDEX ON %s(categories)");
+
+        execute("INSERT INTO %s (account, id , categories) VALUES (?, ?, ?)", "test", 5, map("lmn", "foo"));
+
+        assertEmpty(execute("SELECT * FROM %s WHERE account = ? AND categories CONTAINS ?", "xyz", "foo"));
+
+        assertRows(execute("SELECT * FROM %s WHERE account = ? AND categories CONTAINS ?", "test", "foo"),
+                   row("test", 5, map("lmn", "foo"))
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE categories CONTAINS ?", "foo"),
+                   row("test", 5, map("lmn", "foo"))
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE account = ? AND id = ? AND categories CONTAINS ?", "test", 5, "foo"),
+                   row("test", 5, map("lmn", "foo"))
+        );
+
+        assertInvalidMessage("Unsupported null value for indexed column categories",
+                             "SELECT * FROM %s WHERE account = ? AND id = ? AND categories CONTAINS ?", "test", 5, null);
+
+        assertInvalidMessage("Unsupported unset value for indexed column categories",
+                             "SELECT * FROM %s WHERE account = ? AND id = ? AND categories CONTAINS ?", "test", 5, unset());
+
+        assertInvalidMessage("Cannot execute this query as it might involve data filtering and thus may have unpredictable performance. If you want to execute this query despite the performance unpredictability, use ALLOW FILTERING",
+                             "SELECT * FROM %s WHERE account = ? AND id = ? AND categories CONTAINS ? AND categories CONTAINS ?"
+                            , "test", 5, "foo", "notPresent");
+
+        assertEmpty(execute("SELECT * FROM %s WHERE account = ? AND id = ? AND categories CONTAINS ? AND categories CONTAINS ? ALLOW FILTERING"
+                           , "test", 5, "foo", "notPresent"));
+    }
+
+    // See CASSANDRA-7525
+    @Test
+    public void testQueryMultipleIndexTypes() throws Throwable
+    {
+        createTable("CREATE TABLE %s (account text, id int, categories map<text,text>, PRIMARY KEY (account, id))");
+
+        // create an index on
+        createIndex("CREATE INDEX id_index ON %s(id)");
+        createIndex("CREATE INDEX categories_values_index ON %s(categories)");
+
+        execute("INSERT INTO %s (account, id , categories) VALUES (?, ?, ?)", "test", 5, map("lmn", "foo"));
+
+        assertRows(execute("SELECT * FROM %s WHERE categories CONTAINS ? AND id = ? ALLOW FILTERING", "foo", 5),
+                   row("test", 5, map("lmn", "foo"))
+        );
+
+        assertRows(
+                  execute("SELECT * FROM %s WHERE account = ? AND categories CONTAINS ? AND id = ? ALLOW FILTERING", "test", "foo", 5),
+                  row("test", 5, map("lmn", "foo"))
+        );
+    }
+
+    // See CASSANDRA-8033
+    @Test
+    public void testFilterForContains() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k1 int, k2 int, v set<int>, PRIMARY KEY ((k1, k2)))");
+        createIndex("CREATE INDEX ON %s(k2)");
+
+        execute("INSERT INTO %s (k1, k2, v) VALUES (?, ?, ?)", 0, 0, set(1, 2, 3));
+        execute("INSERT INTO %s (k1, k2, v) VALUES (?, ?, ?)", 0, 1, set(2, 3, 4));
+        execute("INSERT INTO %s (k1, k2, v) VALUES (?, ?, ?)", 1, 0, set(3, 4, 5));
+        execute("INSERT INTO %s (k1, k2, v) VALUES (?, ?, ?)", 1, 1, set(4, 5, 6));
+
+        assertRows(execute("SELECT * FROM %s WHERE k2 = ?", 1),
+                   row(0, 1, set(2, 3, 4)),
+                   row(1, 1, set(4, 5, 6))
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE k2 = ? AND v CONTAINS ? ALLOW FILTERING", 1, 6),
+                   row(1, 1, set(4, 5, 6))
+        );
+
+        assertEmpty(execute("SELECT * FROM %s WHERE k2 = ? AND v CONTAINS ? ALLOW FILTERING", 1, 7));
+    }
+
+    // See CASSANDRA-8073
+    @Test
+    public void testIndexLookupWithClusteringPrefix() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int, b int, c int, d set<int>, PRIMARY KEY (a, b, c))");
+        createIndex("CREATE INDEX ON %s(d)");
+        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 0, set(1, 2, 3));
+        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 1, set(3, 4, 5));
+        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 0, set(1, 2, 3));
+        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 1, set(3, 4, 5));
+
+        assertRows(execute("SELECT * FROM %s WHERE a=? AND b=? AND d CONTAINS ?", 0, 1, 3),
+                   row(0, 1, 0, set(1, 2, 3)),
+                   row(0, 1, 1, set(3, 4, 5))
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE a=? AND b=? AND d CONTAINS ?", 0, 1, 2),
+                   row(0, 1, 0, set(1, 2, 3))
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE a=? AND b=? AND d CONTAINS ?", 0, 1, 5),
+                   row(0, 1, 1, set(3, 4, 5))
+        );
+    }
+
+    @Test
+    public void testContainsKeyAndContainsWithIndexOnMapKey() throws Throwable
+    {
+        createTable("CREATE TABLE %s (account text, id int, categories map<text,text>, PRIMARY KEY (account, id))");
+        createIndex("CREATE INDEX ON %s(keys(categories))");
+
+        execute("INSERT INTO %s (account, id , categories) VALUES (?, ?, ?)", "test", 5, map("lmn", "foo"));
+        execute("INSERT INTO %s (account, id , categories) VALUES (?, ?, ?)", "test", 6, map("lmn", "foo2"));
+
+        assertInvalidMessage("No secondary indexes on the restricted columns support the provided operators: 'categories CONTAINS <value>'",
+                             "SELECT * FROM %s WHERE account = ? AND categories CONTAINS ?", "test", "foo");
+
+        assertRows(execute("SELECT * FROM %s WHERE account = ? AND categories CONTAINS KEY ?", "test", "lmn"),
+                   row("test", 5, map("lmn", "foo")),
+                   row("test", 6, map("lmn", "foo2")));
+        assertRows(execute("SELECT * FROM %s WHERE account = ? AND categories CONTAINS KEY ? AND categories CONTAINS ? ALLOW FILTERING",
+                           "test", "lmn", "foo"),
+                   row("test", 5, map("lmn", "foo")));
+        assertRows(execute("SELECT * FROM %s WHERE account = ? AND categories CONTAINS ? AND categories CONTAINS KEY ? ALLOW FILTERING",
+                           "test", "foo", "lmn"),
+                   row("test", 5, map("lmn", "foo")));
+    }
+
+    @Test
+    public void testContainsKeyAndContainsWithIndexOnMapValue() throws Throwable
+    {
+        createTable("CREATE TABLE %s (account text, id int, categories map<text,text>, PRIMARY KEY (account, id))");
+        createIndex("CREATE INDEX ON %s(categories)");
+
+        execute("INSERT INTO %s (account, id , categories) VALUES (?, ?, ?)", "test", 5, map("lmn", "foo"));
+        execute("INSERT INTO %s (account, id , categories) VALUES (?, ?, ?)", "test", 6, map("lmn2", "foo"));
+
+        assertInvalidMessage("No secondary indexes on the restricted columns support the provided operators: 'categories CONTAINS KEY <value>'",
+                             "SELECT * FROM %s WHERE account = ? AND categories CONTAINS KEY ?", "test", "lmn");
+
+        assertRows(execute("SELECT * FROM %s WHERE account = ? AND categories CONTAINS ?", "test", "foo"),
+                   row("test", 5, map("lmn", "foo")),
+                   row("test", 6, map("lmn2", "foo")));
+        assertRows(execute("SELECT * FROM %s WHERE account = ? AND categories CONTAINS KEY ? AND categories CONTAINS ? ALLOW FILTERING",
+                           "test", "lmn", "foo"),
+                   row("test", 5, map("lmn", "foo")));
+        assertRows(execute("SELECT * FROM %s WHERE account = ? AND categories CONTAINS ? AND categories CONTAINS KEY ? ALLOW FILTERING",
+                           "test", "foo", "lmn"),
+                   row("test", 5, map("lmn", "foo")));
+    }
+
+    /**
+     * Test token ranges
+     * migrated from cql_tests.py:TestCQL.token_range_test()
+     */
+    @Test
+    public void testTokenRange() throws Throwable
+    {
+        createTable(" CREATE TABLE %s (k int PRIMARY KEY, c int, v int)");
+
+        int c = 100;
+        for (int i = 0; i < c; i++)
+            execute("INSERT INTO %s (k, c, v) VALUES (?, ?, ?)", i, i, i);
+
+        Object[][] res = getRows(execute("SELECT k FROM %s"));
+        assertEquals(c, res.length);
+
+        Object[] inOrder = new Object[res.length];
+        for (int i = 0; i < res.length; i++)
+            inOrder[i] = res[i][0];
+
+        Long min_token = Long.MIN_VALUE;
+
+        res = getRows(execute(String.format("SELECT k FROM %s.%s WHERE token(k) >= %d",
+                                            keyspace(), currentTable(), min_token)));
+        assertEquals(c, res.length);
+
+        res = getRows(execute(String.format("SELECT k FROM %s.%s WHERE token(k) >= token(%d) AND token(k) < token(%d)",
+                                            keyspace(), currentTable(), inOrder[32], inOrder[65])));
+
+        for (int i = 32; i < 65; i++)
+            Assert.assertEquals(inOrder[i], res[i - 32][0]);
+    }
+
+    /**
+     * Test select count
+     * migrated from cql_tests.py:TestCQL.count_test()
+     */
+    @Test
+    public void testSelectCount() throws Throwable
+    {
+        createTable(" CREATE TABLE %s (kind text, time int, value1 int, value2 int, PRIMARY KEY(kind, time))");
+
+        execute("INSERT INTO %s (kind, time, value1, value2) VALUES ('ev1', ?, ?, ?)", 0, 0, 0);
+        execute("INSERT INTO %s (kind, time, value1, value2) VALUES ('ev1', ?, ?, ?)", 1, 1, 1);
+        execute("INSERT INTO %s (kind, time, value1) VALUES ('ev1', ?, ?)", 2, 2);
+        execute("INSERT INTO %s (kind, time, value1, value2) VALUES ('ev1', ?, ?, ?)", 3, 3, 3);
+        execute("INSERT INTO %s (kind, time, value1) VALUES ('ev1', ?, ?)", 4, 4);
+        execute("INSERT INTO %s (kind, time, value1, value2) VALUES ('ev2', 0, 0, 0)");
+
+        assertRows(execute("SELECT COUNT(*) FROM %s WHERE kind = 'ev1'"),
+                   row(5L));
+
+        assertRows(execute("SELECT COUNT(1) FROM %s WHERE kind IN ('ev1', 'ev2') AND time=0"),
+                   row(2L));
+    }
+
+    /**
+     * Range test query from #4372
+     * migrated from cql_tests.py:TestCQL.range_query_test()
+     */
+    @Test
+    public void testRangeQuery() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int, b int, c int, d int, e int, f text, PRIMARY KEY (a, b, c, d, e) )");
+
+        execute("INSERT INTO %s (a, b, c, d, e, f) VALUES (1, 1, 1, 1, 2, '2')");
+        execute("INSERT INTO %s (a, b, c, d, e, f) VALUES (1, 1, 1, 1, 1, '1')");
+        execute("INSERT INTO %s (a, b, c, d, e, f) VALUES (1, 1, 1, 2, 1, '1')");
+        execute("INSERT INTO %s (a, b, c, d, e, f) VALUES (1, 1, 1, 1, 3, '3')");
+        execute("INSERT INTO %s (a, b, c, d, e, f) VALUES (1, 1, 1, 1, 5, '5')");
+
+        assertRows(execute("SELECT a, b, c, d, e, f FROM %s WHERE a = 1 AND b = 1 AND c = 1 AND d = 1 AND e >= 2"),
+                   row(1, 1, 1, 1, 2, "2"),
+                   row(1, 1, 1, 1, 3, "3"),
+                   row(1, 1, 1, 1, 5, "5"));
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.composite_row_key_test()
+     */
+    @Test
+    public void testCompositeRowKey() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k1 int, k2 int, c int, v int, PRIMARY KEY ((k1, k2), c))");
+
+        for (int i = 0; i < 4; i++)
+            execute("INSERT INTO %s (k1, k2, c, v) VALUES (?, ?, ?, ?)", 0, i, i, i);
+
+        assertRows(execute("SELECT * FROM %s"),
+                   row(0, 2, 2, 2),
+                   row(0, 3, 3, 3),
+                   row(0, 0, 0, 0),
+                   row(0, 1, 1, 1));
+
+        assertRows(execute("SELECT * FROM %s WHERE k1 = 0 and k2 IN (1, 3)"),
+                   row(0, 1, 1, 1),
+                   row(0, 3, 3, 3));
+
+        assertInvalid("SELECT * FROM %s WHERE k2 = 3");
+
+        assertRows(execute("SELECT * FROM %s WHERE token(k1, k2) = token(0, 1)"),
+                   row(0, 1, 1, 1));
+
+
+        assertRows(execute("SELECT * FROM %s WHERE token(k1, k2) > ?", Long.MIN_VALUE),
+                   row(0, 2, 2, 2),
+                   row(0, 3, 3, 3),
+                   row(0, 0, 0, 0),
+                   row(0, 1, 1, 1));
+    }
+
+    /**
+     * Test for #4532, NPE when trying to select a slice from a composite table
+     * migrated from cql_tests.py:TestCQL.bug_4532_test()
+     */
+    @Test
+    public void testSelectSliceFromComposite() throws Throwable
+    {
+        createTable("CREATE TABLE %s (status ascii, ctime bigint, key ascii, nil ascii, PRIMARY KEY (status, ctime, key))");
+
+        execute("INSERT INTO %s (status,ctime,key,nil) VALUES ('C',12345678,'key1','')");
+        execute("INSERT INTO %s (status,ctime,key,nil) VALUES ('C',12345678,'key2','')");
+        execute("INSERT INTO %s (status,ctime,key,nil) VALUES ('C',12345679,'key3','')");
+        execute("INSERT INTO %s (status,ctime,key,nil) VALUES ('C',12345679,'key4','')");
+        execute("INSERT INTO %s (status,ctime,key,nil) VALUES ('C',12345679,'key5','')");
+        execute("INSERT INTO %s (status,ctime,key,nil) VALUES ('C',12345680,'key6','')");
+
+        assertInvalid("SELECT * FROM %s WHERE ctime>=12345679 AND key='key3' AND ctime<=12345680 LIMIT 3;");
+        assertInvalid("SELECT * FROM %s WHERE ctime=12345679  AND key='key3' AND ctime<=12345680 LIMIT 3");
+    }
+
+    /**
+     * Test for #4716 bug and more generally for good behavior of ordering,
+     * migrated from cql_tests.py:TestCQL.reversed_compact_test()
+     */
+    @Test
+    public void testReverseCompact() throws Throwable
+    {
+        createTable("CREATE TABLE %s ( k text, c int, v int, PRIMARY KEY (k, c) ) WITH COMPACT STORAGE AND CLUSTERING ORDER BY (c DESC)");
+
+        for (int i = 0; i < 10; i++)
+            execute("INSERT INTO %s (k, c, v) VALUES ('foo', ?, ?)", i, i);
+
+        assertRows(execute("SELECT c FROM %s WHERE c > 2 AND c < 6 AND k = 'foo'"),
+                   row(5), row(4), row(3));
+
+        assertRows(execute("SELECT c FROM %s WHERE c >= 2 AND c <= 6 AND k = 'foo'"),
+                   row(6), row(5), row(4), row(3), row(2));
+
+        assertRows(execute("SELECT c FROM %s WHERE c > 2 AND c < 6 AND k = 'foo' ORDER BY c ASC"),
+                   row(3), row(4), row(5));
+
+        assertRows(execute("SELECT c FROM %s WHERE c >= 2 AND c <= 6 AND k = 'foo' ORDER BY c ASC"),
+                   row(2), row(3), row(4), row(5), row(6));
+
+        assertRows(execute("SELECT c FROM %s WHERE c > 2 AND c < 6 AND k = 'foo' ORDER BY c DESC"),
+                   row(5), row(4), row(3));
+
+        assertRows(execute("SELECT c FROM %s WHERE c >= 2 AND c <= 6 AND k = 'foo' ORDER BY c DESC"),
+                   row(6), row(5), row(4), row(3), row(2));
+
+        createTable("CREATE TABLE %s ( k text, c int, v int, PRIMARY KEY (k, c) ) WITH COMPACT STORAGE");
+
+        for (int i = 0; i < 10; i++)
+            execute("INSERT INTO %s(k, c, v) VALUES ('foo', ?, ?)", i, i);
+
+        assertRows(execute("SELECT c FROM %s WHERE c > 2 AND c < 6 AND k = 'foo'"),
+                   row(3), row(4), row(5));
+
+        assertRows(execute("SELECT c FROM %s WHERE c >= 2 AND c <= 6 AND k = 'foo'"),
+                   row(2), row(3), row(4), row(5), row(6));
+
+        assertRows(execute("SELECT c FROM %s WHERE c > 2 AND c < 6 AND k = 'foo' ORDER BY c ASC"),
+                   row(3), row(4), row(5));
+
+        assertRows(execute("SELECT c FROM %s WHERE c >= 2 AND c <= 6 AND k = 'foo' ORDER BY c ASC"),
+                   row(2), row(3), row(4), row(5), row(6));
+
+        assertRows(execute("SELECT c FROM %s WHERE c > 2 AND c < 6 AND k = 'foo' ORDER BY c DESC"),
+                   row(5), row(4), row(3));
+
+        assertRows(execute("SELECT c FROM %s WHERE c >= 2 AND c <= 6 AND k = 'foo' ORDER BY c DESC"),
+                   row(6), row(5), row(4), row(3), row(2));
+    }
+
+    /**
+     * Test for the bug from #4760 and #4759,
+     * migrated from cql_tests.py:TestCQL.reversed_compact_multikey_test()
+     */
+    @Test
+    public void testReversedCompactMultikey() throws Throwable
+    {
+        createTable("CREATE TABLE %s (key text, c1 int, c2 int, value text, PRIMARY KEY(key, c1, c2) ) WITH COMPACT STORAGE AND CLUSTERING ORDER BY(c1 DESC, c2 DESC)");
+
+        for (int i = 0; i < 3; i++)
+            for (int j = 0; j < 3; j++)
+                execute("INSERT INTO %s (key, c1, c2, value) VALUES ('foo', ?, ?, 'bar')", i, j);
+
+        // Equalities
+        assertRows(execute("SELECT c1, c2 FROM %s WHERE key='foo' AND c1 = 1"),
+                   row(1, 2), row(1, 1), row(1, 0));
+
+        assertRows(execute("SELECT c1, c2 FROM %s WHERE key='foo' AND c1 = 1 ORDER BY c1 ASC, c2 ASC"),
+                   row(1, 0), row(1, 1), row(1, 2));
+
+        assertRows(execute("SELECT c1, c2 FROM %s WHERE key='foo' AND c1 = 1 ORDER BY c1 DESC, c2 DESC"),
+                   row(1, 2), row(1, 1), row(1, 0));
+
+        // GT
+        assertRows(execute("SELECT c1, c2 FROM %s WHERE key='foo' AND c1 > 1"),
+                   row(2, 2), row(2, 1), row(2, 0));
+
+        assertRows(execute("SELECT c1, c2 FROM %s WHERE key='foo' AND c1 > 1 ORDER BY c1 ASC, c2 ASC"),
+                   row(2, 0), row(2, 1), row(2, 2));
+
+        assertRows(execute("SELECT c1, c2 FROM %s WHERE key='foo' AND c1 > 1 ORDER BY c1 DESC, c2 DESC"),
+                   row(2, 2), row(2, 1), row(2, 0));
+
+        assertRows(execute("SELECT c1, c2 FROM %s WHERE key='foo' AND c1 >= 1"),
+                   row(2, 2), row(2, 1), row(2, 0), row(1, 2), row(1, 1), row(1, 0));
+
+        assertRows(execute("SELECT c1, c2 FROM %s WHERE key='foo' AND c1 >= 1 ORDER BY c1 ASC, c2 ASC"),
+                   row(1, 0), row(1, 1), row(1, 2), row(2, 0), row(2, 1), row(2, 2));
+
+        assertRows(execute("SELECT c1, c2 FROM %s WHERE key='foo' AND c1 >= 1 ORDER BY c1 ASC"),
+                   row(1, 0), row(1, 1), row(1, 2), row(2, 0), row(2, 1), row(2, 2));
+
+        assertRows(execute("SELECT c1, c2 FROM %s WHERE key='foo' AND c1 >= 1 ORDER BY c1 DESC, c2 DESC"),
+                   row(2, 2), row(2, 1), row(2, 0), row(1, 2), row(1, 1), row(1, 0));
+
+        // LT
+        assertRows(execute("SELECT c1, c2 FROM %s WHERE key='foo' AND c1 < 1"),
+                   row(0, 2), row(0, 1), row(0, 0));
+
+        assertRows(execute("SELECT c1, c2 FROM %s WHERE key='foo' AND c1 < 1 ORDER BY c1 ASC, c2 ASC"),
+                   row(0, 0), row(0, 1), row(0, 2));
+
+        assertRows(execute("SELECT c1, c2 FROM %s WHERE key='foo' AND c1 < 1 ORDER BY c1 DESC, c2 DESC"),
+                   row(0, 2), row(0, 1), row(0, 0));
+
+        assertRows(execute("SELECT c1, c2 FROM %s WHERE key='foo' AND c1 <= 1"),
+                   row(1, 2), row(1, 1), row(1, 0), row(0, 2), row(0, 1), row(0, 0));
+
+        assertRows(execute("SELECT c1, c2 FROM %s WHERE key='foo' AND c1 <= 1 ORDER BY c1 ASC, c2 ASC"),
+                   row(0, 0), row(0, 1), row(0, 2), row(1, 0), row(1, 1), row(1, 2));
+
+        assertRows(execute("SELECT c1, c2 FROM %s WHERE key='foo' AND c1 <= 1 ORDER BY c1 ASC"),
+                   row(0, 0), row(0, 1), row(0, 2), row(1, 0), row(1, 1), row(1, 2));
+
+        assertRows(execute("SELECT c1, c2 FROM %s WHERE key='foo' AND c1 <= 1 ORDER BY c1 DESC, c2 DESC"),
+                   row(1, 2), row(1, 1), row(1, 0), row(0, 2), row(0, 1), row(0, 0));
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.bug_4882_test()
+     */
+    @Test
+    public void testDifferentOrdering() throws Throwable
+    {
+        createTable(" CREATE TABLE %s ( k int, c1 int, c2 int, v int, PRIMARY KEY (k, c1, c2) ) WITH CLUSTERING ORDER BY (c1 ASC, c2 DESC)");
+
+        execute("INSERT INTO %s (k, c1, c2, v) VALUES (0, 0, 0, 0)");
+        execute("INSERT INTO %s (k, c1, c2, v) VALUES (0, 1, 1, 1)");
+        execute("INSERT INTO %s (k, c1, c2, v) VALUES (0, 0, 2, 2)");
+        execute("INSERT INTO %s (k, c1, c2, v) VALUES (0, 1, 3, 3)");
+
+        assertRows(execute("select * from %s where k = 0 limit 1"),
+                   row(0, 0, 2, 2));
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.allow_filtering_test()
+     */
+    @Test
+    public void testAllowFiltering() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int, c int, v int, PRIMARY KEY (k, c))");
+
+        for (int i = 0; i < 3; i++)
+            for (int j = 0; j < 3; j++)
+                execute("INSERT INTO %s (k, c, v) VALUES (?, ?, ?)", i, j, j);
+
+        // Don't require filtering, always allowed
+        String[] queries = new String[]
+                           {
+                           "SELECT * FROM %s WHERE k = 1",
+                           "SELECT * FROM %s WHERE k = 1 AND c > 2",
+                           "SELECT * FROM %s WHERE k = 1 AND c = 2"
+                           };
+
+        for (String q : queries)
+        {
+            execute(q);
+            execute(q + " ALLOW FILTERING");
+        }
+
+        // Require filtering, allowed only with ALLOW FILTERING
+        queries = new String[]
+                  {
+                  "SELECT * FROM %s WHERE c = 2",
+                  "SELECT * FROM %s WHERE c > 2 AND c <= 4"
+                  };
+
+        for (String q : queries)
+        {
+            assertInvalid(q);
+            execute(q + " ALLOW FILTERING");
+        }
+
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, a int, b int,)");
+        createIndex("CREATE INDEX ON %s (a)");
+
+        for (int i = 0; i < 5; i++)
+            execute("INSERT INTO %s (k, a, b) VALUES (?, ?, ?)", i, i * 10, i * 100);
+
+        // Don't require filtering, always allowed
+        queries = new String[]
+                  {
+                  "SELECT * FROM %s WHERE k = 1",
+                  "SELECT * FROM %s WHERE a = 20"
+                  };
+
+        for (String q : queries)
+        {
+            execute(q);
+            execute(q + " ALLOW FILTERING");
+        }
+
+        // Require filtering, allowed only with ALLOW FILTERING
+        queries = new String[]
+                  {
+                  "SELECT * FROM %s WHERE a = 20 AND b = 200"
+                  };
+
+        for (String q : queries)
+        {
+            assertInvalid(q);
+            execute(q + " ALLOW FILTERING");
+        }
+    }
+
+    /**
+     * Test for bug from #5122,
+     * migrated from cql_tests.py:TestCQL.composite_partition_key_validation_test()
+     */
+    @Test
+    public void testSelectOnCompositeInvalid() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int, b text, c uuid, PRIMARY KEY ((a, b)))");
+
+        execute("INSERT INTO %s (a, b , c ) VALUES (1, 'aze', 4d481800-4c5f-11e1-82e0-3f484de45426)");
+        execute("INSERT INTO %s (a, b , c ) VALUES (1, 'ert', 693f5800-8acb-11e3-82e0-3f484de45426)");
+        execute("INSERT INTO %s (a, b , c ) VALUES (1, 'opl', d4815800-2d8d-11e0-82e0-3f484de45426)");
+
+        assertRowCount(execute("SELECT * FROM %s"), 3);
+        assertInvalid("SELECT * FROM %s WHERE a=1");
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.multi_in_test()
+     */
+    @Test
+    public void testMultiSelects() throws Throwable
+    {
+        doTestVariousSelects(false);
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.multi_in_compact_test()
+     */
+    @Test
+    public void testMultiSelectsCompactStorage() throws Throwable
+    {
+        doTestVariousSelects(true);
+    }
+
+
+    public void doTestVariousSelects(boolean compact) throws Throwable
+    {
+        createTable(
+                   "CREATE TABLE %s (group text, zipcode text, state text, fips_regions int, city text, PRIMARY KEY (group, zipcode, state, fips_regions))"
+                   + (compact
+                      ? " WITH COMPACT STORAGE"
+                      : ""));
+
+        String str = "INSERT INTO %s (group, zipcode, state, fips_regions, city) VALUES (?, ?, ?, ?, ?)";
+        execute(str, "test", "06029", "CT", 9, "Ellington");
+        execute(str, "test", "06031", "CT", 9, "Falls Village");
+        execute(str, "test", "06902", "CT", 9, "Stamford");
+        execute(str, "test", "06927", "CT", 9, "Stamford");
+        execute(str, "test", "10015", "NY", 36, "New York");
+        execute(str, "test", "07182", "NJ", 34, "Newark");
+        execute(str, "test", "73301", "TX", 48, "Austin");
+        execute(str, "test", "94102", "CA", 06, "San Francisco");
+
+        execute(str, "test2", "06029", "CT", 9, "Ellington");
+        execute(str, "test2", "06031", "CT", 9, "Falls Village");
+        execute(str, "test2", "06902", "CT", 9, "Stamford");
+        execute(str, "test2", "06927", "CT", 9, "Stamford");
+        execute(str, "test2", "10015", "NY", 36, "New York");
+        execute(str, "test2", "07182", "NJ", 34, "Newark");
+        execute(str, "test2", "73301", "TX", 48, "Austin");
+        execute(str, "test2", "94102", "CA", 06, "San Francisco");
+
+        assertRowCount(execute("select zipcode from %s"), 16);
+        assertRowCount(execute("select zipcode from %s where group='test'"), 8);
+        assertInvalid("select zipcode from %s where zipcode='06902'");
+        assertRowCount(execute("select zipcode from %s where zipcode='06902' ALLOW FILTERING"), 2);
+        assertRowCount(execute("select zipcode from %s where group='test' and zipcode='06902'"), 1);
+        assertRowCount(execute("select zipcode from %s where group='test' and zipcode IN ('06902','73301','94102')"), 3);
+        assertRowCount(execute("select zipcode from %s where group='test' AND zipcode IN ('06902','73301','94102') and state IN ('CT','CA')"), 2);
+        assertRowCount(execute("select zipcode from %s where group='test' AND zipcode IN ('06902','73301','94102') and state IN ('CT','CA') and fips_regions = 9"), 1);
+        assertRowCount(execute("select zipcode from %s where group='test' AND zipcode IN ('06902','73301','94102') and state IN ('CT','CA') ORDER BY zipcode DESC"), 2);
+        assertRowCount(execute("select zipcode from %s where group='test' AND zipcode IN ('06902','73301','94102') and state IN ('CT','CA') and fips_regions > 0"), 2);
+        assertEmpty(execute("select zipcode from %s where group='test' AND zipcode IN ('06902','73301','94102') and state IN ('CT','CA') and fips_regions < 0"));
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.multi_in_compact_non_composite_test()
+     */
+    @Test
+    public void testMultiSelectsNonCompositeCompactStorage() throws Throwable
+    {
+        createTable("CREATE TABLE %s (key int, c int, v int, PRIMARY KEY (key, c)) WITH COMPACT STORAGE");
+
+        execute("INSERT INTO %s (key, c, v) VALUES (0, 0, 0)");
+        execute("INSERT INTO %s (key, c, v) VALUES (0, 1, 1)");
+        execute("INSERT INTO %s (key, c, v) VALUES (0, 2, 2)");
+
+        assertRows(execute("SELECT * FROM %s WHERE key=0 AND c IN (0, 2)"),
+                   row(0, 0, 0), row(0, 2, 2));
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.ticket_5230_test()
+     */
+    @Test
+    public void testMultipleClausesOnPrimaryKey() throws Throwable
+    {
+        createTable("CREATE TABLE %s (key text, c text, v text, PRIMARY KEY(key, c))");
+
+        execute("INSERT INTO %s (key, c, v) VALUES ('foo', '1', '1')");
+        execute("INSERT INTO %s(key, c, v) VALUES ('foo', '2', '2')");
+        execute("INSERT INTO %s(key, c, v) VALUES ('foo', '3', '3')");
+
+        assertRows(execute("SELECT c FROM %s WHERE key = 'foo' AND c IN ('1', '2')"),
+                   row("1"), row("2"));
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.bug_5404()
+     */
+    @Test
+    public void testSelectWithToken() throws Throwable
+    {
+        createTable("CREATE TABLE %s (key text PRIMARY KEY)");
+
+        // We just want to make sure this doesn 't NPE server side
+        assertInvalid("select * from %s where token(key) > token(int(3030343330393233)) limit 1");
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.clustering_order_and_functions_test()
+     */
+    @Test
+    public void testFunctionsWithClusteringDesc() throws Throwable
+    {
+        createTable("CREATE TABLE %s ( k int, t timeuuid, PRIMARY KEY (k, t) ) WITH CLUSTERING ORDER BY (t DESC)");
+
+        for (int i = 0; i < 5; i++)
+            execute("INSERT INTO %s (k, t) VALUES (?, now())", i);
+
+        execute("SELECT dateOf(t) FROM %s");
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.select_with_alias_test()
+     */
+    @Test
+    public void testSelectWithAlias() throws Throwable
+    {
+        createTable("CREATE TABLE %s (id int PRIMARY KEY, name text)");
+
+        for (int id = 0; id < 5; id++)
+            execute("INSERT INTO %s (id, name) VALUES (?, ?) USING TTL 10 AND TIMESTAMP 0", id, "name" + id);
+
+        // test aliasing count( *)
+        UntypedResultSet rs = execute("SELECT count(*) AS user_count FROM %s");
+        assertEquals("user_count", rs.metadata().get(0).name.toString());
+        assertEquals(5L, rs.one().getLong(rs.metadata().get(0).name.toString()));
+
+        // test aliasing regular value
+        rs = execute("SELECT name AS user_name FROM %s WHERE id = 0");
+        assertEquals("user_name", rs.metadata().get(0).name.toString());
+        assertEquals("name0", rs.one().getString(rs.metadata().get(0).name.toString()));
+
+        // test aliasing writetime
+        rs = execute("SELECT writeTime(name) AS name_writetime FROM %s WHERE id = 0");
+        assertEquals("name_writetime", rs.metadata().get(0).name.toString());
+        assertEquals(0, rs.one().getInt(rs.metadata().get(0).name.toString()));
+
+        // test aliasing ttl
+        rs = execute("SELECT ttl(name) AS name_ttl FROM %s WHERE id = 0");
+        assertEquals("name_ttl", rs.metadata().get(0).name.toString());
+        int ttl = rs.one().getInt(rs.metadata().get(0).name.toString());
+        assertTrue(ttl == 9 || ttl == 10);
+
+        // test aliasing a regular function
+        rs = execute("SELECT intAsBlob(id) AS id_blob FROM %s WHERE id = 0");
+        assertEquals("id_blob", rs.metadata().get(0).name.toString());
+        assertEquals(ByteBuffer.wrap(new byte[4]), rs.one().getBlob(rs.metadata().get(0).name.toString()));
+
+        // test that select throws a meaningful exception for aliases in where clause
+        assertInvalidMessage("Aliases aren't allowed in the where clause",
+                             "SELECT id AS user_id, name AS user_name FROM %s WHERE user_id = 0");
+
+        // test that select throws a meaningful exception for aliases in order by clause
+        assertInvalidMessage("Aliases are not allowed in order by clause",
+                             "SELECT id AS user_id, name AS user_name FROM %s WHERE id IN (0) ORDER BY user_name");
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.select_distinct_test()
+     */
+    @Test
+    public void testSelectDistinct() throws Throwable
+    {
+        // Test a regular(CQL3) table.
+        createTable("CREATE TABLE %s (pk0 int, pk1 int, ck0 int, val int, PRIMARY KEY((pk0, pk1), ck0))");
+
+        for (int i = 0; i < 3; i++)
+        {
+            execute("INSERT INTO %s (pk0, pk1, ck0, val) VALUES (?, ?, 0, 0)", i, i);
+            execute("INSERT INTO %s (pk0, pk1, ck0, val) VALUES (?, ?, 1, 1)", i, i);
+        }
+
+        assertRows(execute("SELECT DISTINCT pk0, pk1 FROM %s LIMIT 1"),
+                   row(0, 0));
+
+        assertRows(execute("SELECT DISTINCT pk0, pk1 FROM %s LIMIT 3"),
+                   row(0, 0),
+                   row(2, 2),
+                   row(1, 1));
+
+        // Test selection validation.
+        assertInvalidMessage("queries must request all the partition key columns", "SELECT DISTINCT pk0 FROM %s");
+        assertInvalidMessage("queries must only request partition key columns", "SELECT DISTINCT pk0, pk1, ck0 FROM %s");
+
+        //Test a 'compact storage' table.
+        createTable("CREATE TABLE %s (pk0 int, pk1 int, val int, PRIMARY KEY((pk0, pk1))) WITH COMPACT STORAGE");
+
+        for (int i = 0; i < 3; i++)
+            execute("INSERT INTO %s (pk0, pk1, val) VALUES (?, ?, ?)", i, i, i);
+
+        assertRows(execute("SELECT DISTINCT pk0, pk1 FROM %s LIMIT 1"),
+                   row(0, 0));
+
+        assertRows(execute("SELECT DISTINCT pk0, pk1 FROM %s LIMIT 3"),
+                   row(0, 0),
+                   row(2, 2),
+                   row(1, 1));
+
+        // Test a 'wide row' thrift table.
+        createTable("CREATE TABLE %s (pk int, name text, val int, PRIMARY KEY(pk, name)) WITH COMPACT STORAGE");
+
+        for (int i = 0; i < 3; i++)
+        {
+            execute("INSERT INTO %s (pk, name, val) VALUES (?, 'name0', 0)", i);
+            execute("INSERT INTO %s (pk, name, val) VALUES (?, 'name1', 1)", i);
+        }
+
+        assertRows(execute("SELECT DISTINCT pk FROM %s LIMIT 1"),
+                   row(1));
+
+        assertRows(execute("SELECT DISTINCT pk FROM %s LIMIT 3"),
+                   row(1),
+                   row(0),
+                   row(2));
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.select_distinct_with_deletions_test()
+     */
+    @Test
+    public void testSelectDistinctWithDeletions() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, c int, v int)");
+
+        for (int i = 0; i < 10; i++)
+            execute("INSERT INTO %s (k, c, v) VALUES (?, ?, ?)", i, i, i);
+
+        Object[][] rows = getRows(execute("SELECT DISTINCT k FROM %s"));
+        Assert.assertEquals(10, rows.length);
+        Object key_to_delete = rows[3][0];
+
+        execute("DELETE FROM %s WHERE k=?", key_to_delete);
+
+        rows = getRows(execute("SELECT DISTINCT k FROM %s"));
+        Assert.assertEquals(9, rows.length);
+
+        rows = getRows(execute("SELECT DISTINCT k FROM %s LIMIT 5"));
+        Assert.assertEquals(5, rows.length);
+
+        rows = getRows(execute("SELECT DISTINCT k FROM %s"));
+        Assert.assertEquals(9, rows.length);
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.bug_6327_test()
+     */
+    @Test
+    public void testSelectInClauseAtOne() throws Throwable
+    {
+        createTable("CREATE TABLE %s ( k int, v int, PRIMARY KEY (k, v))");
+
+        execute("INSERT INTO %s (k, v) VALUES (0, 0)");
+
+        flush();
+
+        assertRows(execute("SELECT v FROM %s WHERE k=0 AND v IN (1, 0)"),
+                   row(0));
+    }
+
+    /**
+     * Test for the #6579 'select count' paging bug,
+     * migrated from cql_tests.py:TestCQL.select_count_paging_test()
+     */
+    @Test
+    public void testSelectCountPaging() throws Throwable
+    {
+        createTable("create table %s (field1 text, field2 timeuuid, field3 boolean, primary key(field1, field2))");
+        createIndex("create index test_index on %s (field3)");
+
+        execute("insert into %s (field1, field2, field3) values ('hola', now(), false)");
+        execute("insert into %s (field1, field2, field3) values ('hola', now(), false)");
+
+        assertRows(execute("select count(*) from %s where field3 = false limit 1"),
+                   row(2L));
+    }
+
+    /**
+     * Test for #7105 bug,
+     * migrated from cql_tests.py:TestCQL.clustering_order_in_test()
+     */
+    @Test
+    public void testClusteringOrder() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int, b int, c int, PRIMARY KEY ((a, b), c) ) with clustering order by (c desc)");
+
+        execute("INSERT INTO %s (a, b, c) VALUES (1, 2, 3)");
+        execute("INSERT INTO %s (a, b, c) VALUES (4, 5, 6)");
+
+        assertRows(execute("SELECT * FROM %s WHERE a=1 AND b=2 AND c IN (3)"),
+                   row(1, 2, 3));
+        assertRows(execute("SELECT * FROM %s WHERE a=1 AND b=2 AND c IN (3, 4)"),
+                   row(1, 2, 3));
+    }
+
+    /**
+     * Test for #7105 bug,
+     * SELECT with IN on final column of composite and compound primary key fails
+     * migrated from cql_tests.py:TestCQL.bug7105_test()
+     */
+    @Test
+    public void testSelectInFinalColumn() 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 (1, 2, 3, 3)");
+        execute("INSERT INTO %s (a, b, c, d) VALUES (1, 4, 6, 5)");
+
+        assertRows(execute("SELECT * FROM %s WHERE a=1 AND b=2 ORDER BY b DESC"),
+                   row(1, 2, 3, 3));
+    }
+
+    @Test
+    public void testAlias() throws Throwable
+    {
+        createTable("CREATE TABLE %s (id int PRIMARY KEY, name text)");
+
+        for (int i = 0; i < 5; i++)
+            execute("INSERT INTO %s (id, name) VALUES (?, ?) USING TTL 10 AND TIMESTAMP 0", i, Integer.toString(i));
+
+        assertInvalidMessage("Aliases aren't allowed in the where clause",
+                             "SELECT id AS user_id, name AS user_name FROM %s WHERE user_id = 0");
+
+        // test that select throws a meaningful exception for aliases in order by clause
+        assertInvalidMessage("Aliases are not allowed in order by clause",
+                             "SELECT id AS user_id, name AS user_name FROM %s WHERE id IN (0) ORDER BY user_name");
+
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/01115f72/test/unit/org/apache/cassandra/cql3/validation/operations/UpdateTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/UpdateTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/UpdateTest.java
new file mode 100644
index 0000000..c5d153f
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/UpdateTest.java
@@ -0,0 +1,86 @@
+/*
+ * 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.validation.operations;
+
+import org.junit.Test;
+
+import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+public class UpdateTest extends CQLTester
+{
+    /**
+     * Test altering the type of a column, including the one in the primary key (#4041)
+     * migrated from cql_tests.py:TestCQL.update_type_test()
+     */
+    @Test
+    public void testUpdateColumnType() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k text, c text, s set <text>, v text, PRIMARY KEY(k, c))");
+
+        // using utf8 character so that we can see the transition to BytesType
+        execute("INSERT INTO %s (k, c, v, s) VALUES ('ɸ', 'ɸ', 'ɸ', {'ɸ'})");
+
+        assertRows(execute("SELECT * FROM %s"),
+                   row("ɸ", "ɸ", set("ɸ"), "ɸ"));
+
+        execute("ALTER TABLE %s ALTER v TYPE blob");
+        assertRows(execute("SELECT * FROM %s"),
+                   row("ɸ", "ɸ", set("ɸ"), ByteBufferUtil.bytes("ɸ")));
+
+        execute("ALTER TABLE %s ALTER k TYPE blob");
+        assertRows(execute("SELECT * FROM %s"),
+                   row(ByteBufferUtil.bytes("ɸ"), "ɸ", set("ɸ"), ByteBufferUtil.bytes("ɸ")));
+
+        execute("ALTER TABLE %s ALTER c TYPE blob");
+        assertRows(execute("SELECT * FROM %s"),
+                   row(ByteBufferUtil.bytes("ɸ"), ByteBufferUtil.bytes("ɸ"), set("ɸ"), ByteBufferUtil.bytes("ɸ")));
+
+        execute("ALTER TABLE %s ALTER s TYPE set<blob>");
+        assertRows(execute("SELECT * FROM %s"),
+                   row(ByteBufferUtil.bytes("ɸ"), ByteBufferUtil.bytes("ɸ"), set(ByteBufferUtil.bytes("ɸ")), ByteBufferUtil.bytes("ɸ")));
+    }
+
+    @Test
+    public void testTypeCasts() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, t text, a ascii, d double, i int)");
+
+        // The followings is fine
+        execute("UPDATE %s SET t = 'foo' WHERE k = ?", 0);
+        execute("UPDATE %s SET t = (ascii)'foo' WHERE k = ?", 0);
+        execute("UPDATE %s SET t = (text)(ascii)'foo' WHERE k = ?", 0);
+        execute("UPDATE %s SET a = 'foo' WHERE k = ?", 0);
+        execute("UPDATE %s SET a = (ascii)'foo' WHERE k = ?", 0);
+
+        // But trying to put some explicitely type-casted text into an ascii
+        // column should be rejected (even though the text is actually ascci)
+        assertInvalid("UPDATE %s SET a = (text)'foo' WHERE k = ?", 0);
+
+        // This is also fine because integer constants works for both integer and float types
+        execute("UPDATE %s SET i = 3 WHERE k = ?", 0);
+        execute("UPDATE %s SET i = (int)3 WHERE k = ?", 0);
+        execute("UPDATE %s SET d = 3 WHERE k = ?", 0);
+        execute("UPDATE %s SET d = (double)3 WHERE k = ?", 0);
+
+        // But values for ints and doubles are not truly compatible (their binary representation differs)
+        assertInvalid("UPDATE %s SET d = (int)3 WHERE k = ?", 0);
+        assertInvalid("UPDATE %s SET i = (double)3 WHERE k = ?", 0);
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/01115f72/test/unit/org/apache/cassandra/cql3/validation/operations/UseTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/UseTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/UseTest.java
new file mode 100644
index 0000000..e1498b6
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/UseTest.java
@@ -0,0 +1,31 @@
+/*
+ * 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.validation.operations;
+
+import org.junit.Test;
+
+import org.apache.cassandra.cql3.CQLTester;
+
+public class UseTest extends CQLTester
+{
+    @Test
+    public void testUseStatementWithBindVariable() throws Throwable
+    {
+        assertInvalidSyntaxMessage("Bind variables cannot be used for keyspace names", "USE ?");
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/01115f72/test/unit/org/apache/cassandra/service/ClientWarningsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/service/ClientWarningsTest.java b/test/unit/org/apache/cassandra/service/ClientWarningsTest.java
index da7577e..363fc7d 100644
--- a/test/unit/org/apache/cassandra/service/ClientWarningsTest.java
+++ b/test/unit/org/apache/cassandra/service/ClientWarningsTest.java
@@ -22,8 +22,9 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.cql3.CQLTester;
 import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.dht.ByteOrderedPartitioner;
 import org.apache.cassandra.transport.Message;
 import org.apache.cassandra.transport.Server;
 import org.apache.cassandra.transport.SimpleClient;
@@ -37,6 +38,8 @@ public class ClientWarningsTest extends CQLTester
     @BeforeClass
     public static void setUp()
     {
+        DatabaseDescriptor.setPartitioner(ByteOrderedPartitioner.instance);
+
         requireNetwork();
         DatabaseDescriptor.setBatchSizeWarnThresholdInKB(1);
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/01115f72/test/unit/org/apache/cassandra/transport/MessagePayloadTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/transport/MessagePayloadTest.java b/test/unit/org/apache/cassandra/transport/MessagePayloadTest.java
index 73daa48..1dd3c5d 100644
--- a/test/unit/org/apache/cassandra/transport/MessagePayloadTest.java
+++ b/test/unit/org/apache/cassandra/transport/MessagePayloadTest.java
@@ -29,14 +29,16 @@ import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.cql3.BatchQueryOptions;
 import org.apache.cassandra.cql3.CQLStatement;
-import org.apache.cassandra.cql3.CQLTester;
 import org.apache.cassandra.cql3.QueryHandler;
 import org.apache.cassandra.cql3.QueryOptions;
 import org.apache.cassandra.cql3.QueryProcessor;
 import org.apache.cassandra.cql3.statements.BatchStatement;
 import org.apache.cassandra.cql3.statements.ParsedStatement;
+import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.dht.ByteOrderedPartitioner;
 import org.apache.cassandra.exceptions.RequestExecutionException;
 import org.apache.cassandra.exceptions.RequestValidationException;
 import org.apache.cassandra.service.ClientState;
@@ -63,6 +65,8 @@ public class MessagePayloadTest extends CQLTester
     {
         try
         {
+            DatabaseDescriptor.setPartitioner(ByteOrderedPartitioner.instance);
+
             cqlQueryHandlerField = ClientState.class.getDeclaredField("cqlQueryHandler");
             cqlQueryHandlerField.setAccessible(true);
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/01115f72/tools/stress/src/org/apache/cassandra/stress/generate/values/TimeUUIDs.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/generate/values/TimeUUIDs.java b/tools/stress/src/org/apache/cassandra/stress/generate/values/TimeUUIDs.java
index 7bfabf5..537d54e 100644
--- a/tools/stress/src/org/apache/cassandra/stress/generate/values/TimeUUIDs.java
+++ b/tools/stress/src/org/apache/cassandra/stress/generate/values/TimeUUIDs.java
@@ -46,6 +46,6 @@ public class TimeUUIDs extends Generator<UUID>
     @Override
     public UUID generate()
     {
-        return UUIDGen.getTimeUUID(dateGen.generate().getTime(), clockSeqAndNode);
+        return UUIDGen.getTimeUUID(dateGen.generate().getTime(), 0L, clockSeqAndNode);
     }
 }


[17/32] cassandra git commit: 2.2 commit for CASSANDRA-9160

Posted by jm...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/01115f72/test/unit/org/apache/cassandra/cql3/validation/entities/UFTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/UFTest.java b/test/unit/org/apache/cassandra/cql3/validation/entities/UFTest.java
new file mode 100644
index 0000000..2e7c2f1
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/UFTest.java
@@ -0,0 +1,2596 @@
+/*
+ * 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.validation.entities;
+
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Date;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.TreeSet;
+import java.util.UUID;
+
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import com.datastax.driver.core.*;
+import com.datastax.driver.core.exceptions.InvalidQueryException;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.cql3.QueryProcessor;
+import org.apache.cassandra.cql3.UntypedResultSet;
+import org.apache.cassandra.cql3.functions.FunctionName;
+import org.apache.cassandra.cql3.functions.Functions;
+import org.apache.cassandra.cql3.functions.UDFunction;
+import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.db.marshal.CollectionType;
+import org.apache.cassandra.dht.ByteOrderedPartitioner;
+import org.apache.cassandra.exceptions.FunctionExecutionException;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.transport.Event;
+import org.apache.cassandra.transport.Server;
+import org.apache.cassandra.transport.messages.ResultMessage;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.UUIDGen;
+
+public class UFTest extends CQLTester
+{
+    @BeforeClass
+    public static void setUp()
+    {
+        DatabaseDescriptor.setPartitioner(ByteOrderedPartitioner.instance);
+    }
+
+    @Test
+    public void testSchemaChange() throws Throwable
+    {
+        String f = createFunction(KEYSPACE,
+                                  "double, double",
+                                  "CREATE OR REPLACE FUNCTION %s(state double, val double) " +
+                                  "RETURNS NULL ON NULL INPUT " +
+                                  "RETURNS double " +
+                                  "LANGUAGE javascript " +
+                                  "AS '\"string\";';");
+
+        assertLastSchemaChange(Event.SchemaChange.Change.CREATED, Event.SchemaChange.Target.FUNCTION,
+                               KEYSPACE, parseFunctionName(f).name,
+                               "double", "double");
+
+        createFunctionOverload(f,
+                               "double, double",
+                               "CREATE OR REPLACE FUNCTION %s(state int, val int) " +
+                               "RETURNS NULL ON NULL INPUT " +
+                               "RETURNS int " +
+                               "LANGUAGE javascript " +
+                               "AS '\"string\";';");
+
+        assertLastSchemaChange(Event.SchemaChange.Change.CREATED, Event.SchemaChange.Target.FUNCTION,
+                               KEYSPACE, parseFunctionName(f).name,
+                               "int", "int");
+
+        schemaChange("CREATE OR REPLACE FUNCTION " + f + "(state int, val int) " +
+                     "RETURNS NULL ON NULL INPUT " +
+                     "RETURNS int " +
+                     "LANGUAGE javascript " +
+                     "AS '\"string\";';");
+
+        assertLastSchemaChange(Event.SchemaChange.Change.UPDATED, Event.SchemaChange.Target.FUNCTION,
+                               KEYSPACE, parseFunctionName(f).name,
+                               "int", "int");
+
+        schemaChange("DROP FUNCTION " + f + "(double, double)");
+
+        assertLastSchemaChange(Event.SchemaChange.Change.DROPPED, Event.SchemaChange.Target.FUNCTION,
+                               KEYSPACE, parseFunctionName(f).name,
+                               "double", "double");
+    }
+
+    @Test
+    public void testFunctionDropOnKeyspaceDrop() throws Throwable
+    {
+        String fSin = createFunction(KEYSPACE_PER_TEST, "double",
+                                     "CREATE FUNCTION %s ( input double ) " +
+                                     "CALLED ON NULL INPUT " +
+                                     "RETURNS double " +
+                                     "LANGUAGE java " +
+                                     "AS 'return Double.valueOf(Math.sin(input.doubleValue()));'");
+
+        FunctionName fSinName = parseFunctionName(fSin);
+
+        Assert.assertEquals(1, Functions.find(parseFunctionName(fSin)).size());
+
+        assertRows(execute("SELECT function_name, language FROM system.schema_functions WHERE keyspace_name=?", KEYSPACE_PER_TEST),
+                   row(fSinName.name, "java"));
+
+        dropPerTestKeyspace();
+
+        assertRows(execute("SELECT function_name, language FROM system.schema_functions WHERE keyspace_name=?", KEYSPACE_PER_TEST));
+
+        Assert.assertEquals(0, Functions.find(fSinName).size());
+    }
+
+    @Test
+    public void testFunctionDropPreparedStatement() throws Throwable
+    {
+        createTable("CREATE TABLE %s (key int PRIMARY KEY, d double)");
+
+        String fSin = createFunction(KEYSPACE_PER_TEST, "double",
+                                     "CREATE FUNCTION %s ( input double ) " +
+                                     "CALLED ON NULL INPUT " +
+                                     "RETURNS double " +
+                                     "LANGUAGE java " +
+                                     "AS 'return Double.valueOf(Math.sin(input.doubleValue()));'");
+
+        FunctionName fSinName = parseFunctionName(fSin);
+
+        Assert.assertEquals(1, Functions.find(parseFunctionName(fSin)).size());
+
+        // create a pairs of Select and Inserts. One statement in each pair uses the function so when we
+        // drop it those statements should be removed from the cache in QueryProcessor. The other statements
+        // should be unaffected.
+
+        ResultMessage.Prepared preparedSelect1 = QueryProcessor.prepare(
+                                                                       String.format("SELECT key, %s(d) FROM %s.%s", fSin, KEYSPACE, currentTable()),
+                                                                       ClientState.forInternalCalls(), false);
+        ResultMessage.Prepared preparedSelect2 = QueryProcessor.prepare(
+                                                    String.format("SELECT key FROM %s.%s", KEYSPACE, currentTable()),
+                                                    ClientState.forInternalCalls(), false);
+        ResultMessage.Prepared preparedInsert1 = QueryProcessor.prepare(
+                                                      String.format("INSERT INTO %s.%s (key, d) VALUES (?, %s(?))", KEYSPACE, currentTable(), fSin),
+                                                      ClientState.forInternalCalls(), false);
+        ResultMessage.Prepared preparedInsert2 = QueryProcessor.prepare(
+                                                      String.format("INSERT INTO %s.%s (key, d) VALUES (?, ?)", KEYSPACE, currentTable()),
+                                                      ClientState.forInternalCalls(), false);
+
+        Assert.assertNotNull(QueryProcessor.instance.getPrepared(preparedSelect1.statementId));
+        Assert.assertNotNull(QueryProcessor.instance.getPrepared(preparedSelect2.statementId));
+        Assert.assertNotNull(QueryProcessor.instance.getPrepared(preparedInsert1.statementId));
+        Assert.assertNotNull(QueryProcessor.instance.getPrepared(preparedInsert2.statementId));
+
+        execute("DROP FUNCTION " + fSin + "(double);");
+
+        // the statements which use the dropped function should be removed from cache, with the others remaining
+        Assert.assertNull(QueryProcessor.instance.getPrepared(preparedSelect1.statementId));
+        Assert.assertNotNull(QueryProcessor.instance.getPrepared(preparedSelect2.statementId));
+        Assert.assertNull(QueryProcessor.instance.getPrepared(preparedInsert1.statementId));
+        Assert.assertNotNull(QueryProcessor.instance.getPrepared(preparedInsert2.statementId));
+
+        execute("CREATE FUNCTION " + fSin + " ( input double ) " +
+                "RETURNS NULL ON NULL INPUT " +
+                "RETURNS double " +
+                "LANGUAGE java " +
+                "AS 'return Double.valueOf(Math.sin(input));'");
+
+        Assert.assertEquals(1, Functions.find(fSinName).size());
+
+        preparedSelect1= QueryProcessor.prepare(
+                                         String.format("SELECT key, %s(d) FROM %s.%s", fSin, KEYSPACE, currentTable()),
+                                         ClientState.forInternalCalls(), false);
+        preparedInsert1 = QueryProcessor.prepare(
+                                         String.format("INSERT INTO %s.%s (key, d) VALUES (?, %s(?))", KEYSPACE, currentTable(), fSin),
+                                         ClientState.forInternalCalls(), false);
+        Assert.assertNotNull(QueryProcessor.instance.getPrepared(preparedSelect1.statementId));
+        Assert.assertNotNull(QueryProcessor.instance.getPrepared(preparedInsert1.statementId));
+
+        dropPerTestKeyspace();
+
+        // again, only the 2 statements referencing the function should be removed from cache
+        // this time because the statements select from tables in KEYSPACE, only the function
+        // is scoped to KEYSPACE_PER_TEST
+        Assert.assertNull(QueryProcessor.instance.getPrepared(preparedSelect1.statementId));
+        Assert.assertNotNull(QueryProcessor.instance.getPrepared(preparedSelect2.statementId));
+        Assert.assertNull(QueryProcessor.instance.getPrepared(preparedInsert1.statementId));
+        Assert.assertNotNull(QueryProcessor.instance.getPrepared(preparedInsert2.statementId));
+    }
+
+    @Test
+    public void testDropFunctionDropsPreparedStatementsWithDelayedValues() throws Throwable
+    {
+        // test that dropping a function removes stmts which use
+        // it to provide a DelayedValue collection from the
+        // cache in QueryProcessor
+        checkDelayedValuesCorrectlyIdentifyFunctionsInUse(false);
+    }
+
+    @Test
+    public void testDropKeyspaceContainingFunctionDropsPreparedStatementsWithDelayedValues() throws Throwable
+    {
+        // test that dropping a function removes stmts which use
+        // it to provide a DelayedValue collection from the
+        // cache in QueryProcessor
+        checkDelayedValuesCorrectlyIdentifyFunctionsInUse(true);
+    }
+
+    private ResultMessage.Prepared prepareStatementWithDelayedValue(CollectionType.Kind kind, String function)
+    {
+        String collectionType;
+        String literalArgs;
+        switch (kind)
+        {
+            case LIST:
+                collectionType = "list<double>";
+                literalArgs = String.format("[%s(0.0)]", function);
+                break;
+            case SET:
+                collectionType = "set<double>";
+                literalArgs = String.format("{%s(0.0)}", function);
+                break;
+            case MAP:
+                collectionType = "map<double, double>";
+                literalArgs = String.format("{%s(0.0):0.0}", function);
+                break;
+            default:
+                Assert.fail("Unsupported collection type " + kind);
+                collectionType = null;
+                literalArgs = null;
+        }
+
+        createTable("CREATE TABLE %s (" +
+                    " key int PRIMARY KEY," +
+                    " val " + collectionType + ')');
+
+        ResultMessage.Prepared prepared = QueryProcessor.prepare(
+                                                                String.format("INSERT INTO %s.%s (key, val) VALUES (?, %s)",
+                                                                             KEYSPACE,
+                                                                             currentTable(),
+                                                                             literalArgs),
+                                                                ClientState.forInternalCalls(), false);
+        Assert.assertNotNull(QueryProcessor.instance.getPrepared(prepared.statementId));
+        return prepared;
+    }
+
+    private ResultMessage.Prepared prepareStatementWithDelayedValueTuple(String function)
+    {
+        createTable("CREATE TABLE %s (" +
+                    " key int PRIMARY KEY," +
+                    " val tuple<double> )");
+
+        ResultMessage.Prepared prepared = QueryProcessor.prepare(
+                                                                String.format("INSERT INTO %s.%s (key, val) VALUES (?, (%s(0.0)))",
+                                                                             KEYSPACE,
+                                                                             currentTable(),
+                                                                             function),
+                                                                ClientState.forInternalCalls(), false);
+        Assert.assertNotNull(QueryProcessor.instance.getPrepared(prepared.statementId));
+        return prepared;
+    }
+
+    public void checkDelayedValuesCorrectlyIdentifyFunctionsInUse(boolean dropKeyspace) throws Throwable
+    {
+        // prepare a statement which doesn't use any function for a control
+        createTable("CREATE TABLE %s (" +
+                    " key int PRIMARY KEY," +
+                    " val double)");
+        ResultMessage.Prepared control = QueryProcessor.prepare(
+                                                               String.format("INSERT INTO %s.%s (key, val) VALUES (?, ?)",
+                                                                            KEYSPACE,
+                                                                            currentTable()),
+                                                               ClientState.forInternalCalls(), false);
+        Assert.assertNotNull(QueryProcessor.instance.getPrepared(control.statementId));
+
+        // a function that we'll drop and verify that statements which use it to
+        // provide a DelayedValue are removed from the cache in QueryProcessor
+        String function = createFunction(KEYSPACE_PER_TEST, "double",
+                                        "CREATE FUNCTION %s ( input double ) " +
+                                        "CALLED ON NULL INPUT " +
+                                        "RETURNS double " +
+                                        "LANGUAGE javascript " +
+                                        "AS 'input'");
+        Assert.assertEquals(1, Functions.find(parseFunctionName(function)).size());
+
+        List<ResultMessage.Prepared> prepared = new ArrayList<>();
+        // prepare statements which use the function to provide a DelayedValue
+        prepared.add(prepareStatementWithDelayedValue(CollectionType.Kind.LIST, function));
+        prepared.add(prepareStatementWithDelayedValue(CollectionType.Kind.SET, function));
+        prepared.add(prepareStatementWithDelayedValue(CollectionType.Kind.MAP, function));
+        prepared.add(prepareStatementWithDelayedValueTuple(function));
+
+        // what to drop - the function is scoped to the per-test keyspace, but the prepared statements
+        // select from the per-fixture keyspace. So if we drop the per-test keyspace, the function
+        // should be removed along with the statements that reference it. The control statement should
+        // remain present in the cache. Likewise, if we actually drop the function itself the control
+        // statement should not be removed, but the others should be
+        if (dropKeyspace)
+            dropPerTestKeyspace();
+        else
+            execute("DROP FUNCTION " + function);
+
+        Assert.assertNotNull(QueryProcessor.instance.getPrepared(control.statementId));
+        for (ResultMessage.Prepared removed : prepared)
+            Assert.assertNull(QueryProcessor.instance.getPrepared(removed.statementId));
+    }
+
+    @Test
+    public void testFunctionCreationAndDrop() throws Throwable
+    {
+        createTable("CREATE TABLE %s (key int PRIMARY KEY, d double)");
+
+        execute("INSERT INTO %s(key, d) VALUES (?, ?)", 1, 1d);
+        execute("INSERT INTO %s(key, d) VALUES (?, ?)", 2, 2d);
+        execute("INSERT INTO %s(key, d) VALUES (?, ?)", 3, 3d);
+
+        // simple creation
+        String fSin = createFunction(KEYSPACE_PER_TEST, "double",
+                                     "CREATE FUNCTION %s ( input double ) " +
+                                     "CALLED ON NULL INPUT " +
+                                     "RETURNS double " +
+                                     "LANGUAGE java " +
+                                     "AS 'return Math.sin(input);'");
+        // check we can't recreate the same function
+        assertInvalidMessage("already exists",
+                             "CREATE FUNCTION " + fSin + " ( input double ) " +
+                             "CALLED ON NULL INPUT " +
+                             "RETURNS double " +
+                             "LANGUAGE java AS 'return Double.valueOf(Math.sin(input.doubleValue()));'");
+
+        // but that it doesn't comply with "IF NOT EXISTS"
+        execute("CREATE FUNCTION IF NOT EXISTS " + fSin + " ( input double ) " +
+                "CALLED ON NULL INPUT " +
+                "RETURNS double " +
+                "LANGUAGE java AS 'return Double.valueOf(Math.sin(input.doubleValue()));'");
+
+        // Validate that it works as expected
+        assertRows(execute("SELECT key, " + fSin + "(d) FROM %s"),
+            row(1, Math.sin(1d)),
+            row(2, Math.sin(2d)),
+            row(3, Math.sin(3d))
+        );
+
+        // Replace the method with incompatible return type
+        assertInvalidMessage("the new return type text is not compatible with the return type double of existing function",
+                             "CREATE OR REPLACE FUNCTION " + fSin + " ( input double ) " +
+                             "CALLED ON NULL INPUT " +
+                             "RETURNS text " +
+                             "LANGUAGE java AS 'return Double.valueOf(42d);'");
+
+        // proper replacement
+        execute("CREATE OR REPLACE FUNCTION " + fSin + " ( input double ) " +
+                "CALLED ON NULL INPUT " +
+                "RETURNS double " +
+                "LANGUAGE java AS 'return Double.valueOf(42d);'");
+
+        // Validate the method as been replaced
+        assertRows(execute("SELECT key, " + fSin + "(d) FROM %s"),
+            row(1, 42.0),
+            row(2, 42.0),
+            row(3, 42.0)
+        );
+
+        // same function but other keyspace
+        String fSin2 = createFunction(KEYSPACE, "double",
+                                      "CREATE FUNCTION %s ( input double ) " +
+                                      "RETURNS NULL ON NULL INPUT " +
+                                      "RETURNS double " +
+                                      "LANGUAGE java " +
+                                      "AS 'return Math.sin(input);'");
+        assertRows(execute("SELECT key, " + fSin2 + "(d) FROM %s"),
+            row(1, Math.sin(1d)),
+            row(2, Math.sin(2d)),
+            row(3, Math.sin(3d))
+        );
+
+        // Drop
+        execute("DROP FUNCTION " + fSin);
+        execute("DROP FUNCTION " + fSin2);
+
+        // Drop unexisting function
+        assertInvalidMessage("Cannot drop non existing function", "DROP FUNCTION " + fSin);
+        // but don't complain with "IF EXISTS"
+        execute("DROP FUNCTION IF EXISTS " + fSin);
+
+        // can't drop native functions
+        assertInvalidMessage("system keyspace is not user-modifiable", "DROP FUNCTION totimestamp");
+        assertInvalidMessage("system keyspace is not user-modifiable", "DROP FUNCTION uuid");
+
+        // sin() no longer exists
+        assertInvalidMessage("Unknown function", "SELECT key, sin(d) FROM %s");
+    }
+
+    @Test
+    public void testFunctionExecution() throws Throwable
+    {
+        createTable("CREATE TABLE %s (v text PRIMARY KEY)");
+
+        execute("INSERT INTO %s(v) VALUES (?)", "aaa");
+
+        String fRepeat = createFunction(KEYSPACE_PER_TEST, "text,int",
+                                        "CREATE FUNCTION %s(v text, n int) " +
+                                        "RETURNS NULL ON NULL INPUT " +
+                                        "RETURNS text " +
+                                        "LANGUAGE java " +
+                                        "AS 'StringBuilder sb = new StringBuilder();\n" +
+                                        "    for (int i = 0; i < n; i++)\n" +
+                                        "        sb.append(v);\n" +
+                                        "    return sb.toString();'");
+
+        assertRows(execute("SELECT v FROM %s WHERE v=" + fRepeat + "(?, ?)", "a", 3), row("aaa"));
+        assertEmpty(execute("SELECT v FROM %s WHERE v=" + fRepeat + "(?, ?)", "a", 2));
+    }
+
+    @Test
+    public void testFunctionOverloading() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k text PRIMARY KEY, v int)");
+
+        execute("INSERT INTO %s(k, v) VALUES (?, ?)", "f2", 1);
+
+        String fOverload = createFunction(KEYSPACE_PER_TEST, "varchar",
+                                          "CREATE FUNCTION %s ( input varchar ) " +
+                                          "RETURNS NULL ON NULL INPUT " +
+                                          "RETURNS text " +
+                                          "LANGUAGE java " +
+                                          "AS 'return \"f1\";'");
+        createFunctionOverload(fOverload,
+                               "int",
+                               "CREATE OR REPLACE FUNCTION %s(i int) " +
+                               "RETURNS NULL ON NULL INPUT " +
+                               "RETURNS text " +
+                               "LANGUAGE java " +
+                               "AS 'return \"f2\";'");
+        createFunctionOverload(fOverload,
+                               "text,text",
+                               "CREATE OR REPLACE FUNCTION %s(v1 text, v2 text) " +
+                               "RETURNS NULL ON NULL INPUT " +
+                               "RETURNS text " +
+                               "LANGUAGE java " +
+                               "AS 'return \"f3\";'");
+        createFunctionOverload(fOverload,
+                               "ascii",
+                               "CREATE OR REPLACE FUNCTION %s(v ascii) " +
+                               "RETURNS NULL ON NULL INPUT " +
+                               "RETURNS text " +
+                               "LANGUAGE java " +
+                               "AS 'return \"f1\";'");
+
+        // text == varchar, so this should be considered as a duplicate
+        assertInvalidMessage("already exists",
+                             "CREATE FUNCTION " + fOverload + "(v varchar) " +
+                             "RETURNS NULL ON NULL INPUT " +
+                             "RETURNS text " +
+                             "LANGUAGE java AS 'return \"f1\";'");
+
+        assertRows(execute("SELECT " + fOverload + "(k), " + fOverload + "(v), " + fOverload + "(k, k) FROM %s"),
+            row("f1", "f2", "f3")
+        );
+
+        forcePreparedValues();
+        // This shouldn't work if we use preparation since there no way to know which overload to use
+        assertInvalidMessage("Ambiguous call to function", "SELECT v FROM %s WHERE k = " + fOverload + "(?)", "foo");
+        stopForcingPreparedValues();
+
+        // but those should since we specifically cast
+        assertEmpty(execute("SELECT v FROM %s WHERE k = " + fOverload + "((text)?)", "foo"));
+        assertRows(execute("SELECT v FROM %s WHERE k = " + fOverload + "((int)?)", 3), row(1));
+        assertEmpty(execute("SELECT v FROM %s WHERE k = " + fOverload + "((ascii)?)", "foo"));
+        // And since varchar == text, this should work too
+        assertEmpty(execute("SELECT v FROM %s WHERE k = " + fOverload + "((varchar)?)", "foo"));
+
+        // no such functions exist...
+        assertInvalidMessage("non existing function", "DROP FUNCTION " + fOverload + "(boolean)");
+        assertInvalidMessage("non existing function", "DROP FUNCTION " + fOverload + "(bigint)");
+
+        // 'overloaded' has multiple overloads - so it has to fail (CASSANDRA-7812)
+        assertInvalidMessage("matches multiple function definitions", "DROP FUNCTION " + fOverload);
+        execute("DROP FUNCTION " + fOverload + "(varchar)");
+        assertInvalidMessage("none of its type signatures match", "SELECT v FROM %s WHERE k = " + fOverload + "((text)?)", "foo");
+        execute("DROP FUNCTION " + fOverload + "(text, text)");
+        assertInvalidMessage("none of its type signatures match", "SELECT v FROM %s WHERE k = " + fOverload + "((text)?,(text)?)", "foo", "bar");
+        execute("DROP FUNCTION " + fOverload + "(ascii)");
+        assertInvalidMessage("cannot be passed as argument 0 of function", "SELECT v FROM %s WHERE k = " + fOverload + "((ascii)?)", "foo");
+        // single-int-overload must still work
+        assertRows(execute("SELECT v FROM %s WHERE k = " + fOverload + "((int)?)", 3), row(1));
+        // overloaded has just one overload now - so the following DROP FUNCTION is not ambigious (CASSANDRA-7812)
+        execute("DROP FUNCTION " + fOverload);
+    }
+
+    @Test
+    public void testCreateOrReplaceJavaFunction() throws Throwable
+    {
+        createTable("CREATE TABLE %s (key int primary key, val double)");
+        execute("INSERT INTO %s (key, val) VALUES (?, ?)", 1, 1d);
+        execute("INSERT INTO %s (key, val) VALUES (?, ?)", 2, 2d);
+        execute("INSERT INTO %s (key, val) VALUES (?, ?)", 3, 3d);
+
+        String fName = createFunction(KEYSPACE_PER_TEST, "double",
+                "CREATE FUNCTION %s( input double ) " +
+                "CALLED ON NULL INPUT " +
+                "RETURNS double " +
+                "LANGUAGE java " +
+                "AS '\n" +
+                "  // parameter val is of type java.lang.Double\n" +
+                "  /* return type is of type java.lang.Double */\n" +
+                "  if (input == null) {\n" +
+                "    return null;\n" +
+                "  }\n" +
+                "  return Math.sin( input );\n" +
+                "';");
+
+        // just check created function
+        assertRows(execute("SELECT key, val, " + fName + "(val) FROM %s"),
+                   row(1, 1d, Math.sin(1d)),
+                   row(2, 2d, Math.sin(2d)),
+                   row(3, 3d, Math.sin(3d))
+        );
+
+        execute("CREATE OR REPLACE FUNCTION " + fName + "( input double ) " +
+                "CALLED ON NULL INPUT " +
+                "RETURNS double " +
+                "LANGUAGE java\n" +
+                "AS '\n" +
+                "  return input;\n" +
+                "';");
+
+        // check if replaced function returns correct result
+        assertRows(execute("SELECT key, val, " + fName + "(val) FROM %s"),
+                   row(1, 1d, 1d),
+                   row(2, 2d, 2d),
+                   row(3, 3d, 3d)
+        );
+    }
+
+    @Test
+    public void testJavaFunctionNoParameters() throws Throwable
+    {
+        createTable("CREATE TABLE %s (key int primary key, val double)");
+
+        String functionBody = "\n  return 1L;\n";
+
+        String fName = createFunction(KEYSPACE, "",
+                                      "CREATE OR REPLACE FUNCTION %s() " +
+                                      "RETURNS NULL ON NULL INPUT " +
+                                      "RETURNS bigint " +
+                                      "LANGUAGE JAVA\n" +
+                                      "AS '" +functionBody + "';");
+
+        assertRows(execute("SELECT language, body FROM system.schema_functions WHERE keyspace_name=? AND function_name=?",
+                           KEYSPACE, parseFunctionName(fName).name),
+                   row("java", functionBody));
+
+        execute("INSERT INTO %s (key, val) VALUES (?, ?)", 1, 1d);
+        execute("INSERT INTO %s (key, val) VALUES (?, ?)", 2, 2d);
+        execute("INSERT INTO %s (key, val) VALUES (?, ?)", 3, 3d);
+        assertRows(execute("SELECT key, val, " + fName + "() FROM %s"),
+                   row(1, 1d, 1L),
+                   row(2, 2d, 1L),
+                   row(3, 3d, 1L)
+        );
+    }
+
+    @Test
+    public void testJavaFunctionInvalidBodies() throws Throwable
+    {
+        try
+        {
+            execute("CREATE OR REPLACE FUNCTION " + KEYSPACE + ".jfinv() " +
+                    "RETURNS NULL ON NULL INPUT " +
+                    "RETURNS bigint " +
+                    "LANGUAGE JAVA\n" +
+                    "AS '\n" +
+                    "foobarbaz" +
+                    "\n';");
+            Assert.fail();
+        }
+        catch (InvalidRequestException e)
+        {
+            Assert.assertTrue(e.getMessage(), e.getMessage().contains("Java source compilation failed"));
+            Assert.assertTrue(e.getMessage(), e.getMessage().contains("insert \";\" to complete BlockStatements"));
+        }
+
+        try
+        {
+            execute("CREATE OR REPLACE FUNCTION " + KEYSPACE + ".jfinv() " +
+                    "RETURNS NULL ON NULL INPUT " +
+                    "RETURNS bigint " +
+                    "LANGUAGE JAVA\n" +
+                    "AS '\n" +
+                    "foobarbaz;" +
+                    "\n';");
+            Assert.fail();
+        }
+        catch (InvalidRequestException e)
+        {
+            Assert.assertTrue(e.getMessage(), e.getMessage().contains("Java source compilation failed"));
+            Assert.assertTrue(e.getMessage(), e.getMessage().contains("foobarbaz cannot be resolved to a type"));
+        }
+    }
+
+    @Test
+    public void testJavaFunctionInvalidReturn() throws Throwable
+    {
+        assertInvalidMessage("system keyspace is not user-modifiable",
+                             "CREATE OR REPLACE FUNCTION jfir(val double) " +
+                             "RETURNS NULL ON NULL INPUT " +
+                             "RETURNS double " +
+                             "LANGUAGE JAVA\n" +
+                             "AS 'return 1L;';");
+    }
+
+    @Test
+    public void testJavaFunctionArgumentTypeMismatch() throws Throwable
+    {
+        createTable("CREATE TABLE %s (key int primary key, val bigint)");
+
+        String fName = createFunction(KEYSPACE, "double",
+                                      "CREATE OR REPLACE FUNCTION %s(val double)" +
+                                      "RETURNS NULL ON NULL INPUT " +
+                                      "RETURNS double " +
+                                      "LANGUAGE JAVA " +
+                                      "AS 'return Double.valueOf(val);';");
+
+        execute("INSERT INTO %s (key, val) VALUES (?, ?)", 1, 1L);
+        execute("INSERT INTO %s (key, val) VALUES (?, ?)", 2, 2L);
+        execute("INSERT INTO %s (key, val) VALUES (?, ?)", 3, 3L);
+        assertInvalidMessage("val cannot be passed as argument 0 of function",
+                             "SELECT key, val, " + fName + "(val) FROM %s");
+    }
+
+    @Test
+    public void testJavaFunction() throws Throwable
+    {
+        createTable("CREATE TABLE %s (key int primary key, val double)");
+
+        String functionBody = '\n' +
+                              "  // parameter val is of type java.lang.Double\n" +
+                              "  /* return type is of type java.lang.Double */\n" +
+                              "  if (val == null) {\n" +
+                              "    return null;\n" +
+                              "  }\n" +
+                              "  return Math.sin(val);\n";
+
+        String fName = createFunction(KEYSPACE, "double",
+                                      "CREATE OR REPLACE FUNCTION %s(val double) " +
+                                      "CALLED ON NULL INPUT " +
+                                      "RETURNS double " +
+                                      "LANGUAGE JAVA " +
+                                      "AS '" + functionBody + "';");
+
+        FunctionName fNameName = parseFunctionName(fName);
+
+        assertRows(execute("SELECT language, body FROM system.schema_functions WHERE keyspace_name=? AND function_name=?",
+                           fNameName.keyspace, fNameName.name),
+                   row("java", functionBody));
+
+        execute("INSERT INTO %s (key, val) VALUES (?, ?)", 1, 1d);
+        execute("INSERT INTO %s (key, val) VALUES (?, ?)", 2, 2d);
+        execute("INSERT INTO %s (key, val) VALUES (?, ?)", 3, 3d);
+        assertRows(execute("SELECT key, val, " + fName + "(val) FROM %s"),
+                   row(1, 1d, Math.sin(1d)),
+                   row(2, 2d, Math.sin(2d)),
+                   row(3, 3d, Math.sin(3d))
+        );
+    }
+
+    @Test
+    public void testFunctionInTargetKeyspace() throws Throwable
+    {
+        createTable("CREATE TABLE %s (key int primary key, val double)");
+
+        execute("CREATE TABLE " + KEYSPACE_PER_TEST + ".second_tab (key int primary key, val double)");
+
+        String fName = createFunction(KEYSPACE_PER_TEST, "double",
+                                      "CREATE OR REPLACE FUNCTION %s(val double) " +
+                                      "RETURNS NULL ON NULL INPUT " +
+                                      "RETURNS double " +
+                                      "LANGUAGE JAVA " +
+                                      "AS 'return Double.valueOf(val);';");
+
+        execute("INSERT INTO %s (key, val) VALUES (?, ?)", 1, 1d);
+        execute("INSERT INTO %s (key, val) VALUES (?, ?)", 2, 2d);
+        execute("INSERT INTO %s (key, val) VALUES (?, ?)", 3, 3d);
+        assertInvalidMessage("Unknown function",
+                             "SELECT key, val, " + parseFunctionName(fName).name + "(val) FROM %s");
+
+        execute("INSERT INTO " + KEYSPACE_PER_TEST + ".second_tab (key, val) VALUES (?, ?)", 1, 1d);
+        execute("INSERT INTO " + KEYSPACE_PER_TEST + ".second_tab (key, val) VALUES (?, ?)", 2, 2d);
+        execute("INSERT INTO " + KEYSPACE_PER_TEST + ".second_tab (key, val) VALUES (?, ?)", 3, 3d);
+        assertRows(execute("SELECT key, val, " + fName + "(val) FROM " + KEYSPACE_PER_TEST + ".second_tab"),
+                   row(1, 1d, 1d),
+                   row(2, 2d, 2d),
+                   row(3, 3d, 3d)
+        );
+    }
+
+    @Test
+    public void testFunctionWithReservedName() throws Throwable
+    {
+        execute("CREATE TABLE " + KEYSPACE_PER_TEST + ".second_tab (key int primary key, val double)");
+
+        String fName = createFunction(KEYSPACE_PER_TEST, "",
+                                      "CREATE OR REPLACE FUNCTION %s() " +
+                                      "RETURNS NULL ON NULL INPUT " +
+                                      "RETURNS timestamp " +
+                                      "LANGUAGE JAVA " +
+                                      "AS 'return null;';");
+
+        execute("INSERT INTO " + KEYSPACE_PER_TEST + ".second_tab (key, val) VALUES (?, ?)", 1, 1d);
+        execute("INSERT INTO " + KEYSPACE_PER_TEST + ".second_tab (key, val) VALUES (?, ?)", 2, 2d);
+        execute("INSERT INTO " + KEYSPACE_PER_TEST + ".second_tab (key, val) VALUES (?, ?)", 3, 3d);
+
+        // ensure that system now() is executed
+        UntypedResultSet rows = execute("SELECT key, val, now() FROM " + KEYSPACE_PER_TEST + ".second_tab");
+        Assert.assertEquals(3, rows.size());
+        UntypedResultSet.Row row = rows.iterator().next();
+        Date ts = row.getTimestamp(row.getColumns().get(2).name.toString());
+        Assert.assertNotNull(ts);
+
+        // ensure that KEYSPACE_PER_TEST's now() is executed
+        rows = execute("SELECT key, val, " + fName + "() FROM " + KEYSPACE_PER_TEST + ".second_tab");
+        Assert.assertEquals(3, rows.size());
+        row = rows.iterator().next();
+        Assert.assertFalse(row.has(row.getColumns().get(2).name.toString()));
+    }
+
+    @Test
+    public void testFunctionInSystemKS() throws Throwable
+    {
+        execute("CREATE OR REPLACE FUNCTION " + KEYSPACE + ".totimestamp(val timeuuid) " +
+                "RETURNS NULL ON NULL INPUT " +
+                "RETURNS timestamp " +
+                "LANGUAGE JAVA\n" +
+
+                "AS 'return null;';");
+
+        assertInvalidMessage("system keyspace is not user-modifiable",
+                             "CREATE OR REPLACE FUNCTION system.jnft(val double) " +
+                             "RETURNS NULL ON NULL INPUT " +
+                             "RETURNS double " +
+                             "LANGUAGE JAVA\n" +
+                             "AS 'return null;';");
+        assertInvalidMessage("system keyspace is not user-modifiable",
+                             "CREATE OR REPLACE FUNCTION system.totimestamp(val timeuuid) " +
+                             "RETURNS NULL ON NULL INPUT " +
+                             "RETURNS timestamp " +
+                             "LANGUAGE JAVA\n" +
+
+                             "AS 'return null;';");
+        assertInvalidMessage("system keyspace is not user-modifiable",
+                             "DROP FUNCTION system.now");
+
+        // KS for executeInternal() is system
+        assertInvalidMessage("system keyspace is not user-modifiable",
+                             "CREATE OR REPLACE FUNCTION jnft(val double) " +
+                             "RETURNS NULL ON NULL INPUT " +
+                             "RETURNS double " +
+                             "LANGUAGE JAVA\n" +
+                             "AS 'return null;';");
+        assertInvalidMessage("system keyspace is not user-modifiable",
+                             "CREATE OR REPLACE FUNCTION totimestamp(val timeuuid) " +
+                             "RETURNS NULL ON NULL INPUT " +
+                             "RETURNS timestamp " +
+                             "LANGUAGE JAVA\n" +
+                             "AS 'return null;';");
+        assertInvalidMessage("system keyspace is not user-modifiable",
+                             "DROP FUNCTION now");
+    }
+
+    @Test
+    public void testFunctionNonExistingKeyspace() throws Throwable
+    {
+        assertInvalidMessage("to non existing keyspace",
+                             "CREATE OR REPLACE FUNCTION this_ks_does_not_exist.jnft(val double) " +
+                             "RETURNS NULL ON NULL INPUT " +
+                             "RETURNS double " +
+                             "LANGUAGE JAVA\n" +
+                             "AS 'return null;';");
+    }
+
+    @Test
+    public void testFunctionAfterOnDropKeyspace() throws Throwable
+    {
+        dropPerTestKeyspace();
+
+        assertInvalidMessage("to non existing keyspace",
+                             "CREATE OR REPLACE FUNCTION " + KEYSPACE_PER_TEST + ".jnft(val double) " +
+                             "RETURNS NULL ON NULL INPUT " +
+                             "RETURNS double " +
+                             "LANGUAGE JAVA\n" +
+                             "AS 'return null;';");
+    }
+
+    @Test
+    public void testJavaKeyspaceFunction() throws Throwable
+    {
+        createTable("CREATE TABLE %s (key int primary key, val double)");
+
+        String functionBody = '\n' +
+                              "  // parameter val is of type java.lang.Double\n" +
+                              "  /* return type is of type java.lang.Double */\n" +
+                              "  if (val == null) {\n" +
+                              "    return null;\n" +
+                              "  }\n" +
+                              "  return Math.sin( val );\n";
+
+        String fName = createFunction(KEYSPACE_PER_TEST, "double",
+                                     "CREATE OR REPLACE FUNCTION %s(val double) " +
+                                     "CALLED ON NULL INPUT " +
+                                     "RETURNS double " +
+                                     "LANGUAGE JAVA " +
+                                     "AS '" + functionBody + "';");
+
+        FunctionName fNameName = parseFunctionName(fName);
+
+        assertRows(execute("SELECT language, body FROM system.schema_functions WHERE keyspace_name=? AND function_name=?",
+                           fNameName.keyspace, fNameName.name),
+                   row("java", functionBody));
+
+        execute("INSERT INTO %s (key, val) VALUES (?, ?)", 1, 1d);
+        execute("INSERT INTO %s (key, val) VALUES (?, ?)", 2, 2d);
+        execute("INSERT INTO %s (key, val) VALUES (?, ?)", 3, 3d);
+        assertRows(execute("SELECT key, val, " + fName + "(val) FROM %s"),
+                   row(1, 1d, Math.sin(1d)),
+                   row(2, 2d, Math.sin(2d)),
+                   row(3, 3d, Math.sin(3d))
+        );
+    }
+
+    @Test
+    public void testJavaRuntimeException() throws Throwable
+    {
+        createTable("CREATE TABLE %s (key int primary key, val double)");
+
+        String functionBody = '\n' +
+                              "  throw new RuntimeException(\"oh no!\");\n";
+
+        String fName = createFunction(KEYSPACE_PER_TEST, "double",
+                                      "CREATE OR REPLACE FUNCTION %s(val double) " +
+                                      "RETURNS NULL ON NULL INPUT " +
+                                      "RETURNS double " +
+                                      "LANGUAGE JAVA\n" +
+                                      "AS '" + functionBody + "';");
+
+        FunctionName fNameName = parseFunctionName(fName);
+
+        assertRows(execute("SELECT language, body FROM system.schema_functions WHERE keyspace_name=? AND function_name=?",
+                           fNameName.keyspace, fNameName.name),
+                   row("java", functionBody));
+
+        execute("INSERT INTO %s (key, val) VALUES (?, ?)", 1, 1d);
+        execute("INSERT INTO %s (key, val) VALUES (?, ?)", 2, 2d);
+        execute("INSERT INTO %s (key, val) VALUES (?, ?)", 3, 3d);
+
+        // function throws a RuntimeException which is wrapped by FunctionExecutionException
+        assertInvalidThrowMessage("java.lang.RuntimeException: oh no", FunctionExecutionException.class,
+                                  "SELECT key, val, " + fName + "(val) FROM %s");
+    }
+
+    @Test
+    public void testJavaDollarQuotedFunction() throws Throwable
+    {
+        String functionBody = '\n' +
+                              "  // parameter val is of type java.lang.Double\n" +
+                              "  /* return type is of type java.lang.Double */\n" +
+                              "  if (input == null) {\n" +
+                              "    return null;\n" +
+                              "  }\n" +
+                              "  return \"'\"+Math.sin(input)+'\\\'';\n";
+
+        String fName = createFunction(KEYSPACE_PER_TEST, "double",
+                                      "CREATE FUNCTION %s( input double ) " +
+                                      "CALLED ON NULL INPUT " +
+                                      "RETURNS text " +
+                                      "LANGUAGE java\n" +
+                                      "AS $$" + functionBody + "$$;");
+
+        FunctionName fNameName = parseFunctionName(fName);
+
+        assertRows(execute("SELECT language, body FROM system.schema_functions WHERE keyspace_name=? AND function_name=?",
+                           fNameName.keyspace, fNameName.name),
+                   row("java", functionBody));
+    }
+
+    @Test
+    public void testJavaSimpleCollections() throws Throwable
+    {
+        createTable("CREATE TABLE %s (key int primary key, lst list<double>, st set<text>, mp map<int, boolean>)");
+
+        String fList = createFunction(KEYSPACE_PER_TEST, "list<double>",
+                                     "CREATE FUNCTION %s( lst list<double> ) " +
+                                     "RETURNS NULL ON NULL INPUT " +
+                                     "RETURNS list<double> " +
+                                     "LANGUAGE java\n" +
+                                     "AS $$return lst;$$;");
+        String fSet = createFunction(KEYSPACE_PER_TEST, "set<text>",
+                                     "CREATE FUNCTION %s( st set<text> ) " +
+                                     "RETURNS NULL ON NULL INPUT " +
+                                     "RETURNS set<text> " +
+                                     "LANGUAGE java\n" +
+                                     "AS $$return st;$$;");
+        String fMap = createFunction(KEYSPACE_PER_TEST, "map<int, boolean>",
+                                     "CREATE FUNCTION %s( mp map<int, boolean> ) " +
+                                     "RETURNS NULL ON NULL INPUT " +
+                                     "RETURNS map<int, boolean> " +
+                                     "LANGUAGE java\n" +
+                                     "AS $$return mp;$$;");
+
+        List<Double> list = Arrays.asList(1d, 2d, 3d);
+        Set<String> set = new TreeSet<>(Arrays.asList("one", "three", "two"));
+        Map<Integer, Boolean> map = new TreeMap<>();
+        map.put(1, true);
+        map.put(2, false);
+        map.put(3, true);
+
+        execute("INSERT INTO %s (key, lst, st, mp) VALUES (1, ?, ?, ?)", list, set, map);
+
+        assertRows(execute("SELECT " + fList + "(lst), " + fSet + "(st), " + fMap + "(mp) FROM %s WHERE key = 1"),
+                   row(list, set, map));
+
+        // same test - but via native protocol
+        for (int version = Server.VERSION_2; version <= maxProtocolVersion; version++)
+            assertRowsNet(version,
+                          executeNet(version, "SELECT " + fList + "(lst), " + fSet + "(st), " + fMap + "(mp) FROM %s WHERE key = 1"),
+                          row(list, set, map));
+    }
+
+    @Test
+    public void testWrongKeyspace() throws Throwable
+    {
+        String typeName = createType("CREATE TYPE %s (txt text, i int)");
+        String type = KEYSPACE + '.' + typeName;
+
+        assertInvalidMessage(String.format("Statement on keyspace %s cannot refer to a user type in keyspace %s; user types can only be used in the keyspace they are defined in",
+                                           KEYSPACE_PER_TEST, KEYSPACE),
+                             "CREATE FUNCTION " + KEYSPACE_PER_TEST + ".test_wrong_ks( val int ) " +
+                             "CALLED ON NULL INPUT " +
+                             "RETURNS " + type + " " +
+                             "LANGUAGE java\n" +
+                             "AS $$return val;$$;");
+
+        assertInvalidMessage(String.format("Statement on keyspace %s cannot refer to a user type in keyspace %s; user types can only be used in the keyspace they are defined in",
+                                           KEYSPACE_PER_TEST, KEYSPACE),
+                             "CREATE FUNCTION " + KEYSPACE_PER_TEST + ".test_wrong_ks( val " + type + " ) " +
+                             "CALLED ON NULL INPUT " +
+                             "RETURNS int " +
+                             "LANGUAGE java\n" +
+                             "AS $$return val;$$;");
+    }
+
+    @Test
+    public void testComplexNullValues() throws Throwable
+    {
+        String type = KEYSPACE + '.' + createType("CREATE TYPE %s (txt text, i int)");
+
+        createTable("CREATE TABLE %s (key int primary key, lst list<double>, st set<text>, mp map<int, boolean>," +
+                    "tup frozen<tuple<double, text, int, boolean>>, udt frozen<" + type + ">)");
+
+        String fList = createFunction(KEYSPACE, "list<double>",
+                                      "CREATE FUNCTION %s( coll list<double> ) " +
+                                      "CALLED ON NULL INPUT " +
+                                      "RETURNS list<double> " +
+                                      "LANGUAGE java\n" +
+                                      "AS $$return coll;$$;");
+        String fSet = createFunction(KEYSPACE, "set<text>",
+                                     "CREATE FUNCTION %s( coll set<text> ) " +
+                                     "CALLED ON NULL INPUT " +
+                                     "RETURNS set<text> " +
+                                     "LANGUAGE java\n" +
+                                     "AS $$return coll;$$;");
+        String fMap = createFunction(KEYSPACE, "map<int, boolean>",
+                                     "CREATE FUNCTION %s( coll map<int, boolean> ) " +
+                                     "CALLED ON NULL INPUT " +
+                                     "RETURNS map<int, boolean> " +
+                                     "LANGUAGE java\n" +
+                                     "AS $$return coll;$$;");
+        String fTup = createFunction(KEYSPACE, "tuple<double, text, int, boolean>",
+                                     "CREATE FUNCTION %s( val tuple<double, text, int, boolean> ) " +
+                                     "CALLED ON NULL INPUT " +
+                                     "RETURNS tuple<double, text, int, boolean> " +
+                                     "LANGUAGE java\n" +
+                                     "AS $$return val;$$;");
+        String fUdt = createFunction(KEYSPACE, type,
+                                     "CREATE FUNCTION %s( val " + type + " ) " +
+                                     "CALLED ON NULL INPUT " +
+                                     "RETURNS " + type + " " +
+                                     "LANGUAGE java\n" +
+                                     "AS $$return val;$$;");
+        List<Double> list = Arrays.asList(1d, 2d, 3d);
+        Set<String> set = new TreeSet<>(Arrays.asList("one", "three", "two"));
+        Map<Integer, Boolean> map = new TreeMap<>();
+        map.put(1, true);
+        map.put(2, false);
+        map.put(3, true);
+        Object t = tuple(1d, "one", 42, false);
+
+        execute("INSERT INTO %s (key, lst, st, mp, tup, udt) VALUES (1, ?, ?, ?, ?, {txt: 'one', i:1})", list, set, map, t);
+        execute("INSERT INTO %s (key, lst, st, mp, tup, udt) VALUES (2, ?, ?, ?, ?, null)", null, null, null, null);
+
+        execute("SELECT " +
+                fList + "(lst), " +
+                fSet + "(st), " +
+                fMap + "(mp), " +
+                fTup + "(tup), " +
+                fUdt + "(udt) FROM %s WHERE key = 1");
+        UntypedResultSet.Row row = execute("SELECT " +
+                                           fList + "(lst) as l, " +
+                                           fSet + "(st) as s, " +
+                                           fMap + "(mp) as m, " +
+                                           fTup + "(tup) as t, " +
+                                           fUdt + "(udt) as u " +
+                                           "FROM %s WHERE key = 1").one();
+        Assert.assertNotNull(row.getBytes("l"));
+        Assert.assertNotNull(row.getBytes("s"));
+        Assert.assertNotNull(row.getBytes("m"));
+        Assert.assertNotNull(row.getBytes("t"));
+        Assert.assertNotNull(row.getBytes("u"));
+        row = execute("SELECT " +
+                      fList + "(lst) as l, " +
+                      fSet + "(st) as s, " +
+                      fMap + "(mp) as m, " +
+                      fTup + "(tup) as t, " +
+                      fUdt + "(udt) as u " +
+                      "FROM %s WHERE key = 2").one();
+        Assert.assertNull(row.getBytes("l"));
+        Assert.assertNull(row.getBytes("s"));
+        Assert.assertNull(row.getBytes("m"));
+        Assert.assertNull(row.getBytes("t"));
+        Assert.assertNull(row.getBytes("u"));
+
+        for (int version = Server.VERSION_2; version <= maxProtocolVersion; version++)
+        {
+            Row r = executeNet(version, "SELECT " +
+                                        fList + "(lst) as l, " +
+                                        fSet + "(st) as s, " +
+                                        fMap + "(mp) as m, " +
+                                        fTup + "(tup) as t, " +
+                                        fUdt + "(udt) as u " +
+                                        "FROM %s WHERE key = 1").one();
+            Assert.assertNotNull(r.getBytesUnsafe("l"));
+            Assert.assertNotNull(r.getBytesUnsafe("s"));
+            Assert.assertNotNull(r.getBytesUnsafe("m"));
+            Assert.assertNotNull(r.getBytesUnsafe("t"));
+            Assert.assertNotNull(r.getBytesUnsafe("u"));
+            r = executeNet(version, "SELECT " +
+                                    fList + "(lst) as l, " +
+                                    fSet + "(st) as s, " +
+                                    fMap + "(mp) as m, " +
+                                    fTup + "(tup) as t, " +
+                                    fUdt + "(udt) as u " +
+                                    "FROM %s WHERE key = 2").one();
+            Assert.assertNull(r.getBytesUnsafe("l"));
+            Assert.assertNull(r.getBytesUnsafe("s"));
+            Assert.assertNull(r.getBytesUnsafe("m"));
+            Assert.assertNull(r.getBytesUnsafe("t"));
+            Assert.assertNull(r.getBytesUnsafe("u"));
+        }
+    }
+
+    @Test
+    public void testJavaTupleType() throws Throwable
+    {
+        createTable("CREATE TABLE %s (key int primary key, tup frozen<tuple<double, text, int, boolean>>)");
+
+        String fName = createFunction(KEYSPACE, "tuple<double, text, int, boolean>",
+                                     "CREATE FUNCTION %s( tup tuple<double, text, int, boolean> ) " +
+                                     "RETURNS NULL ON NULL INPUT " +
+                                     "RETURNS tuple<double, text, int, boolean> " +
+                                     "LANGUAGE java\n" +
+                                     "AS $$return tup;$$;");
+
+        Object t = tuple(1d, "foo", 2, true);
+
+        execute("INSERT INTO %s (key, tup) VALUES (1, ?)", t);
+
+        assertRows(execute("SELECT tup FROM %s WHERE key = 1"),
+                   row(t));
+
+        assertRows(execute("SELECT " + fName + "(tup) FROM %s WHERE key = 1"),
+                   row(t));
+    }
+
+    @Test
+    public void testJavaTupleTypeCollection() throws Throwable
+    {
+        String tupleTypeDef = "tuple<double, list<double>, set<text>, map<int, boolean>>";
+
+        createTable("CREATE TABLE %s (key int primary key, tup frozen<" + tupleTypeDef + ">)");
+
+        String fTup0 = createFunction(KEYSPACE_PER_TEST, tupleTypeDef,
+                "CREATE FUNCTION %s( tup " + tupleTypeDef + " ) " +
+                "CALLED ON NULL INPUT " +
+                "RETURNS " + tupleTypeDef + ' ' +
+                "LANGUAGE java\n" +
+                "AS $$return " +
+                "       tup;$$;");
+        String fTup1 = createFunction(KEYSPACE_PER_TEST, tupleTypeDef,
+                "CREATE FUNCTION %s( tup " + tupleTypeDef + " ) " +
+                "CALLED ON NULL INPUT " +
+                "RETURNS double " +
+                "LANGUAGE java\n" +
+                "AS $$return " +
+                "       Double.valueOf(tup.getDouble(0));$$;");
+        String fTup2 = createFunction(KEYSPACE_PER_TEST, tupleTypeDef,
+                                      "CREATE FUNCTION %s( tup " + tupleTypeDef + " ) " +
+                                      "RETURNS NULL ON NULL INPUT " +
+                                      "RETURNS list<double> " +
+                                      "LANGUAGE java\n" +
+                                      "AS $$return " +
+                                      "       tup.getList(1, Double.class);$$;");
+        String fTup3 = createFunction(KEYSPACE_PER_TEST, tupleTypeDef,
+                "CREATE FUNCTION %s( tup " + tupleTypeDef + " ) " +
+                "RETURNS NULL ON NULL INPUT " +
+                "RETURNS set<text> " +
+                "LANGUAGE java\n" +
+                "AS $$return " +
+                "       tup.getSet(2, String.class);$$;");
+        String fTup4 = createFunction(KEYSPACE_PER_TEST, tupleTypeDef,
+                "CREATE FUNCTION %s( tup " + tupleTypeDef + " ) " +
+                "RETURNS NULL ON NULL INPUT " +
+                "RETURNS map<int, boolean> " +
+                "LANGUAGE java\n" +
+                "AS $$return " +
+                "       tup.getMap(3, Integer.class, Boolean.class);$$;");
+
+        List<Double> list = Arrays.asList(1d, 2d, 3d);
+        Set<String> set = new TreeSet<>(Arrays.asList("one", "three", "two"));
+        Map<Integer, Boolean> map = new TreeMap<>();
+        map.put(1, true);
+        map.put(2, false);
+        map.put(3, true);
+
+        Object t = tuple(1d, list, set, map);
+
+        execute("INSERT INTO %s (key, tup) VALUES (1, ?)", t);
+
+        assertRows(execute("SELECT " + fTup0 + "(tup) FROM %s WHERE key = 1"),
+                   row(t));
+        assertRows(execute("SELECT " + fTup1 + "(tup) FROM %s WHERE key = 1"),
+                   row(1d));
+        assertRows(execute("SELECT " + fTup2 + "(tup) FROM %s WHERE key = 1"),
+                   row(list));
+        assertRows(execute("SELECT " + fTup3 + "(tup) FROM %s WHERE key = 1"),
+                   row(set));
+        assertRows(execute("SELECT " + fTup4 + "(tup) FROM %s WHERE key = 1"),
+                   row(map));
+
+        TupleType tType = TupleType.of(DataType.cdouble(),
+                                       DataType.list(DataType.cdouble()),
+                                       DataType.set(DataType.text()),
+                                       DataType.map(DataType.cint(), DataType.cboolean()));
+        TupleValue tup = tType.newValue(1d, list, set, map);
+        for (int version = Server.VERSION_2; version <= maxProtocolVersion; version++)
+        {
+            assertRowsNet(version,
+                          executeNet(version, "SELECT " + fTup0 + "(tup) FROM %s WHERE key = 1"),
+                          row(tup));
+            assertRowsNet(version,
+                          executeNet(version, "SELECT " + fTup1 + "(tup) FROM %s WHERE key = 1"),
+                          row(1d));
+            assertRowsNet(version,
+                          executeNet(version, "SELECT " + fTup2 + "(tup) FROM %s WHERE key = 1"),
+                          row(list));
+            assertRowsNet(version,
+                          executeNet(version, "SELECT " + fTup3 + "(tup) FROM %s WHERE key = 1"),
+                          row(set));
+            assertRowsNet(version,
+                          executeNet(version, "SELECT " + fTup4 + "(tup) FROM %s WHERE key = 1"),
+                          row(map));
+        }
+    }
+
+    @Test
+    public void testJavaUserTypeWithUse() throws Throwable
+    {
+        String type = createType("CREATE TYPE %s (txt text, i int)");
+        createTable("CREATE TABLE %s (key int primary key, udt frozen<" + KEYSPACE + '.' + type + ">)");
+        execute("INSERT INTO %s (key, udt) VALUES (1, {txt: 'one', i:1})");
+
+        for (int version = Server.VERSION_2; version <= maxProtocolVersion; version++)
+        {
+            executeNet(version, "USE " + KEYSPACE);
+
+            executeNet(version,
+                       "CREATE FUNCTION f_use1( udt " + type + " ) " +
+                       "RETURNS NULL ON NULL INPUT " +
+                       "RETURNS " + type + " " +
+                       "LANGUAGE java " +
+                       "AS $$return " +
+                       "     udt;$$;");
+            try
+            {
+                List<Row> rowsNet = executeNet(version, "SELECT f_use1(udt) FROM %s WHERE key = 1").all();
+                Assert.assertEquals(1, rowsNet.size());
+                UDTValue udtVal = rowsNet.get(0).getUDTValue(0);
+                Assert.assertEquals("one", udtVal.getString("txt"));
+                Assert.assertEquals(1, udtVal.getInt("i"));
+            }
+            finally
+            {
+                executeNet(version, "DROP FUNCTION f_use1");
+            }
+        }
+    }
+
+    @Test
+    public void testJavaUserType() throws Throwable
+    {
+        String type = KEYSPACE + '.' + createType("CREATE TYPE %s (txt text, i int)");
+
+        createTable("CREATE TABLE %s (key int primary key, udt frozen<" + type + ">)");
+
+        String fUdt0 = createFunction(KEYSPACE, type,
+                                      "CREATE FUNCTION %s( udt " + type + " ) " +
+                                      "RETURNS NULL ON NULL INPUT " +
+                                      "RETURNS " + type + " " +
+                                      "LANGUAGE java " +
+                                      "AS $$return " +
+                                      "     udt;$$;");
+        String fUdt1 = createFunction(KEYSPACE, type,
+                                      "CREATE FUNCTION %s( udt " + type + ") " +
+                                      "RETURNS NULL ON NULL INPUT " +
+                                      "RETURNS text " +
+                                      "LANGUAGE java " +
+                                      "AS $$return " +
+                                      "     udt.getString(\"txt\");$$;");
+        String fUdt2 = createFunction(KEYSPACE, type,
+                                      "CREATE FUNCTION %s( udt " + type + ") " +
+                                      "CALLED ON NULL INPUT " +
+                                      "RETURNS int " +
+                                      "LANGUAGE java " +
+                                      "AS $$return " +
+                                      "     Integer.valueOf(udt.getInt(\"i\"));$$;");
+
+        execute("INSERT INTO %s (key, udt) VALUES (1, {txt: 'one', i:1})");
+
+        UntypedResultSet rows = execute("SELECT " + fUdt0 + "(udt) FROM %s WHERE key = 1");
+        Assert.assertEquals(1, rows.size());
+        assertRows(execute("SELECT " + fUdt1 + "(udt) FROM %s WHERE key = 1"),
+                   row("one"));
+        assertRows(execute("SELECT " + fUdt2 + "(udt) FROM %s WHERE key = 1"),
+                   row(1));
+
+        for (int version = Server.VERSION_2; version <= maxProtocolVersion; version++)
+        {
+            List<Row> rowsNet = executeNet(version, "SELECT " + fUdt0 + "(udt) FROM %s WHERE key = 1").all();
+            Assert.assertEquals(1, rowsNet.size());
+            UDTValue udtVal = rowsNet.get(0).getUDTValue(0);
+            Assert.assertEquals("one", udtVal.getString("txt"));
+            Assert.assertEquals(1, udtVal.getInt("i"));
+            assertRowsNet(version,
+                          executeNet(version, "SELECT " + fUdt1 + "(udt) FROM %s WHERE key = 1"),
+                          row("one"));
+            assertRowsNet(version,
+                          executeNet(version, "SELECT " + fUdt2 + "(udt) FROM %s WHERE key = 1"),
+                          row(1));
+        }
+    }
+
+    @Test
+    public void testUserTypeDrop() throws Throwable
+    {
+        String type = KEYSPACE + '.' + createType("CREATE TYPE %s (txt text, i int)");
+
+        createTable("CREATE TABLE %s (key int primary key, udt frozen<" + type + ">)");
+
+        String fName = createFunction(KEYSPACE, type,
+                                      "CREATE FUNCTION %s( udt " + type + " ) " +
+                                      "CALLED ON NULL INPUT " +
+                                      "RETURNS int " +
+                                      "LANGUAGE java " +
+                                      "AS $$return " +
+                                      "     Integer.valueOf(udt.getInt(\"i\"));$$;");
+
+        FunctionName fNameName = parseFunctionName(fName);
+
+        Assert.assertEquals(1, Functions.find(fNameName).size());
+
+        ResultMessage.Prepared prepared = QueryProcessor.prepare(String.format("SELECT key, %s(udt) FROM %s.%s", fName, KEYSPACE, currentTable()),
+                                                                 ClientState.forInternalCalls(), false);
+        Assert.assertNotNull(QueryProcessor.instance.getPrepared(prepared.statementId));
+
+        // UT still referenced by table
+        assertInvalidMessage("Cannot drop user type", "DROP TYPE " + type);
+
+        execute("DROP TABLE %s");
+
+        // UT still referenced by UDF
+        assertInvalidMessage("as it is still used by function", "DROP TYPE " + type);
+
+        Assert.assertNull(QueryProcessor.instance.getPrepared(prepared.statementId));
+
+        // function stays
+        Assert.assertEquals(1, Functions.find(fNameName).size());
+    }
+
+    @Test
+    public void testJavaUserTypeRenameField() throws Throwable
+    {
+        String type = KEYSPACE + '.' + createType("CREATE TYPE %s (txt text, i int)");
+
+        createTable("CREATE TABLE %s (key int primary key, udt frozen<" + type + ">)");
+
+        String fName = createFunction(KEYSPACE, type,
+                                      "CREATE FUNCTION %s( udt " + type + " ) " +
+                                      "RETURNS NULL ON NULL INPUT " +
+                                      "RETURNS text " +
+                                      "LANGUAGE java\n" +
+                                      "AS $$return udt.getString(\"txt\");$$;");
+
+        execute("INSERT INTO %s (key, udt) VALUES (1, {txt: 'one', i:1})");
+
+        assertRows(execute("SELECT " + fName + "(udt) FROM %s WHERE key = 1"),
+                   row("one"));
+
+        execute("ALTER TYPE " + type + " RENAME txt TO str");
+
+        assertInvalidMessage("txt is not a field defined in this UDT",
+                             "SELECT " + fName + "(udt) FROM %s WHERE key = 1");
+
+        execute("ALTER TYPE " + type + " RENAME str TO txt");
+
+        assertRows(execute("SELECT " + fName + "(udt) FROM %s WHERE key = 1"),
+                   row("one"));
+    }
+
+    @Test
+    public void testJavaUserTypeAddFieldWithReplace() throws Throwable
+    {
+        String type = KEYSPACE + '.' + createType("CREATE TYPE %s (txt text, i int)");
+
+        createTable("CREATE TABLE %s (key int primary key, udt frozen<" + type + ">)");
+
+        String fName1replace = createFunction(KEYSPACE, type,
+                                              "CREATE FUNCTION %s( udt " + type + ") " +
+                                              "RETURNS NULL ON NULL INPUT " +
+                                              "RETURNS text " +
+                                              "LANGUAGE java\n" +
+                                              "AS $$return udt.getString(\"txt\");$$;");
+        String fName2replace = createFunction(KEYSPACE, type,
+                                              "CREATE FUNCTION %s( udt " + type + " ) " +
+                                              "CALLED ON NULL INPUT " +
+                                              "RETURNS int " +
+                                              "LANGUAGE java\n" +
+                                              "AS $$return Integer.valueOf(udt.getInt(\"i\"));$$;");
+        String fName3replace = createFunction(KEYSPACE, type,
+                                              "CREATE FUNCTION %s( udt " + type + " ) " +
+                                              "CALLED ON NULL INPUT " +
+                                              "RETURNS double " +
+                                              "LANGUAGE java\n" +
+                                              "AS $$return Double.valueOf(udt.getDouble(\"added\"));$$;");
+        String fName4replace = createFunction(KEYSPACE, type,
+                                              "CREATE FUNCTION %s( udt " + type + " ) " +
+                                              "RETURNS NULL ON NULL INPUT " +
+                                              "RETURNS " + type + " " +
+                                              "LANGUAGE java\n" +
+                                              "AS $$return udt;$$;");
+
+        String fName1noReplace = createFunction(KEYSPACE, type,
+                                              "CREATE FUNCTION %s( udt " + type + " ) " +
+                                              "RETURNS NULL ON NULL INPUT " +
+                                              "RETURNS text " +
+                                              "LANGUAGE java\n" +
+                                              "AS $$return udt.getString(\"txt\");$$;");
+        String fName2noReplace = createFunction(KEYSPACE, type,
+                                              "CREATE FUNCTION %s( udt " + type + " ) " +
+                                              "CALLED ON NULL INPUT " +
+                                              "RETURNS int " +
+                                              "LANGUAGE java\n" +
+                                              "AS $$return Integer.valueOf(udt.getInt(\"i\"));$$;");
+        String fName3noReplace = createFunction(KEYSPACE, type,
+                                                "CREATE FUNCTION %s( udt " + type + " ) " +
+                                                "CALLED ON NULL INPUT " +
+                                                "RETURNS double " +
+                                                "LANGUAGE java\n" +
+                                                "AS $$return Double.valueOf(udt.getDouble(\"added\"));$$;");
+        String fName4noReplace = createFunction(KEYSPACE, type,
+                                                "CREATE FUNCTION %s( udt " + type + " ) " +
+                                                "RETURNS NULL ON NULL INPUT " +
+                                                "RETURNS " + type + " " +
+                                                "LANGUAGE java\n" +
+                                                "AS $$return udt;$$;");
+
+        execute("INSERT INTO %s (key, udt) VALUES (1, {txt: 'one', i:1})");
+
+        assertRows(execute("SELECT " + fName1replace + "(udt) FROM %s WHERE key = 1"),
+                   row("one"));
+        assertRows(execute("SELECT " + fName2replace + "(udt) FROM %s WHERE key = 1"),
+                   row(1));
+
+        // add field
+
+        execute("ALTER TYPE " + type + " ADD added double");
+
+        execute("INSERT INTO %s (key, udt) VALUES (2, {txt: 'two', i:2, added: 2})");
+
+        // note: type references of functions remain at the state _before_ the type mutation
+        // means we need to recreate the functions
+
+        execute(String.format("CREATE OR REPLACE FUNCTION %s( udt %s ) " +
+                              "RETURNS NULL ON NULL INPUT " +
+                              "RETURNS text " +
+                              "LANGUAGE java\n" +
+                              "AS $$return " +
+                              "     udt.getString(\"txt\");$$;",
+                              fName1replace, type));
+        Assert.assertEquals(1, Functions.find(parseFunctionName(fName1replace)).size());
+        execute(String.format("CREATE OR REPLACE FUNCTION %s( udt %s ) " +
+                              "CALLED ON NULL INPUT " +
+                              "RETURNS int " +
+                              "LANGUAGE java\n" +
+                              "AS $$return " +
+                              "     Integer.valueOf(udt.getInt(\"i\"));$$;",
+                              fName2replace, type));
+        Assert.assertEquals(1, Functions.find(parseFunctionName(fName2replace)).size());
+        execute(String.format("CREATE OR REPLACE FUNCTION %s( udt %s ) " +
+                              "CALLED ON NULL INPUT " +
+                              "RETURNS double " +
+                              "LANGUAGE java\n" +
+                              "AS $$return " +
+                              "     Double.valueOf(udt.getDouble(\"added\"));$$;",
+                              fName3replace, type));
+        Assert.assertEquals(1, Functions.find(parseFunctionName(fName3replace)).size());
+        execute(String.format("CREATE OR REPLACE FUNCTION %s( udt %s ) " +
+                              "RETURNS NULL ON NULL INPUT " +
+                              "RETURNS %s " +
+                              "LANGUAGE java\n" +
+                              "AS $$return " +
+                              "     udt;$$;",
+                              fName4replace, type, type));
+        Assert.assertEquals(1, Functions.find(parseFunctionName(fName4replace)).size());
+
+        assertRows(execute("SELECT " + fName1replace + "(udt) FROM %s WHERE key = 2"),
+                   row("two"));
+        assertRows(execute("SELECT " + fName2replace + "(udt) FROM %s WHERE key = 2"),
+                   row(2));
+        assertRows(execute("SELECT " + fName3replace + "(udt) FROM %s WHERE key = 2"),
+                   row(2d));
+        assertRows(execute("SELECT " + fName3replace + "(udt) FROM %s WHERE key = 1"),
+                   row(0d));
+
+        // un-replaced functions will work since the user type has changed
+        // and the UDF has exchanged the user type reference
+
+        assertRows(execute("SELECT " + fName1noReplace + "(udt) FROM %s WHERE key = 2"),
+                   row("two"));
+        assertRows(execute("SELECT " + fName2noReplace + "(udt) FROM %s WHERE key = 2"),
+                   row(2));
+        assertRows(execute("SELECT " + fName3noReplace + "(udt) FROM %s WHERE key = 2"),
+                   row(2d));
+        assertRows(execute("SELECT " + fName3noReplace + "(udt) FROM %s WHERE key = 1"),
+                   row(0d));
+
+        execute("DROP FUNCTION " + fName1replace);
+        execute("DROP FUNCTION " + fName2replace);
+        execute("DROP FUNCTION " + fName3replace);
+        execute("DROP FUNCTION " + fName4replace);
+        execute("DROP FUNCTION " + fName1noReplace);
+        execute("DROP FUNCTION " + fName2noReplace);
+        execute("DROP FUNCTION " + fName3noReplace);
+        execute("DROP FUNCTION " + fName4noReplace);
+    }
+
+    @Test
+    public void testJavaUTCollections() throws Throwable
+    {
+        String type = KEYSPACE + '.' + createType("CREATE TYPE %s (txt text, i int)");
+
+        createTable(String.format("CREATE TABLE %%s " +
+                                  "(key int primary key, lst list<frozen<%s>>, st set<frozen<%s>>, mp map<int, frozen<%s>>)",
+                                  type, type, type));
+
+        String fName1 = createFunction(KEYSPACE, "list<frozen<" + type + ">>",
+                              "CREATE FUNCTION %s( lst list<frozen<" + type + ">> ) " +
+                              "RETURNS NULL ON NULL INPUT " +
+                              "RETURNS text " +
+                              "LANGUAGE java\n" +
+                              "AS $$" +
+                              "     com.datastax.driver.core.UDTValue udtVal = (com.datastax.driver.core.UDTValue)lst.get(1);" +
+                              "     return udtVal.getString(\"txt\");$$;");
+        String fName2 = createFunction(KEYSPACE, "set<frozen<" + type + ">>",
+                                       "CREATE FUNCTION %s( st set<frozen<" + type + ">> ) " +
+                                       "RETURNS NULL ON NULL INPUT " +
+                                       "RETURNS text " +
+                                       "LANGUAGE java\n" +
+                                       "AS $$" +
+                                       "     com.datastax.driver.core.UDTValue udtVal = (com.datastax.driver.core.UDTValue)st.iterator().next();" +
+                                       "     return udtVal.getString(\"txt\");$$;");
+        String fName3 = createFunction(KEYSPACE, "map<int, frozen<" + type + ">>",
+                              "CREATE FUNCTION %s( mp map<int, frozen<" + type + ">> ) " +
+                              "RETURNS NULL ON NULL INPUT " +
+                              "RETURNS text " +
+                              "LANGUAGE java\n" +
+                              "AS $$" +
+                              "     com.datastax.driver.core.UDTValue udtVal = (com.datastax.driver.core.UDTValue)mp.get(Integer.valueOf(3));" +
+                              "     return udtVal.getString(\"txt\");$$;");
+
+        execute("INSERT INTO %s (key, lst, st, mp) values (1, " +
+                "[ {txt: 'one', i:1}, {txt: 'three', i:1}, {txt: 'one', i:1} ] , " +
+                "{ {txt: 'one', i:1}, {txt: 'three', i:3}, {txt: 'two', i:2} }, " +
+                "{ 1: {txt: 'one', i:1}, 2: {txt: 'one', i:3}, 3: {txt: 'two', i:2} })");
+
+        assertRows(execute("SELECT " + fName1 + "(lst), " + fName2 + "(st), " + fName3 + "(mp) FROM %s WHERE key = 1"),
+                   row("three", "one", "two"));
+
+        for (int version = Server.VERSION_2; version <= maxProtocolVersion; version++)
+            assertRowsNet(version,
+                          executeNet(version, "SELECT " + fName1 + "(lst), " + fName2 + "(st), " + fName3 + "(mp) FROM %s WHERE key = 1"),
+                          row("three", "one", "two"));
+    }
+
+    @Test
+    public void testJavascriptSimpleCollections() throws Throwable
+    {
+        createTable("CREATE TABLE %s (key int primary key, lst list<double>, st set<text>, mp map<int, boolean>)");
+
+        String fName1 = createFunction(KEYSPACE_PER_TEST, "list<double>",
+                "CREATE FUNCTION %s( lst list<double> ) " +
+                "RETURNS NULL ON NULL INPUT " +
+                "RETURNS list<double> " +
+                "LANGUAGE javascript\n" +
+                "AS 'lst;';");
+        String fName2 = createFunction(KEYSPACE_PER_TEST, "set<text>",
+                "CREATE FUNCTION %s( st set<text> ) " +
+                "RETURNS NULL ON NULL INPUT " +
+                "RETURNS set<text> " +
+                "LANGUAGE javascript\n" +
+                "AS 'st;';");
+        String fName3 = createFunction(KEYSPACE_PER_TEST, "map<int, boolean>",
+                "CREATE FUNCTION %s( mp map<int, boolean> ) " +
+                "RETURNS NULL ON NULL INPUT " +
+                "RETURNS map<int, boolean> " +
+                "LANGUAGE javascript\n" +
+                "AS 'mp;';");
+
+        List<Double> list = Arrays.asList(1d, 2d, 3d);
+        Set<String> set = new TreeSet<>(Arrays.asList("one", "three", "two"));
+        Map<Integer, Boolean> map = new TreeMap<>();
+        map.put(1, true);
+        map.put(2, false);
+        map.put(3, true);
+
+        execute("INSERT INTO %s (key, lst, st, mp) VALUES (1, ?, ?, ?)", list, set, map);
+
+        assertRows(execute("SELECT lst, st, mp FROM %s WHERE key = 1"),
+                   row(list, set, map));
+
+        assertRows(execute("SELECT " + fName1 + "(lst), " + fName2 + "(st), " + fName3 + "(mp) FROM %s WHERE key = 1"),
+                   row(list, set, map));
+
+        for (int version = Server.VERSION_2; version <= maxProtocolVersion; version++)
+            assertRowsNet(version,
+                          executeNet(version, "SELECT " + fName1 + "(lst), " + fName2 + "(st), " + fName3 + "(mp) FROM %s WHERE key = 1"),
+                          row(list, set, map));
+    }
+
+    @Test
+    public void testJavascriptTupleType() throws Throwable
+    {
+        createTable("CREATE TABLE %s (key int primary key, tup frozen<tuple<double, text, int, boolean>>)");
+
+        String fName = createFunction(KEYSPACE_PER_TEST, "tuple<double, text, int, boolean>",
+                "CREATE FUNCTION %s( tup tuple<double, text, int, boolean> ) " +
+                "RETURNS NULL ON NULL INPUT " +
+                "RETURNS tuple<double, text, int, boolean> " +
+                "LANGUAGE javascript\n" +
+                "AS $$tup;$$;");
+
+        Object t = tuple(1d, "foo", 2, true);
+
+        execute("INSERT INTO %s (key, tup) VALUES (1, ?)", t);
+
+        assertRows(execute("SELECT tup FROM %s WHERE key = 1"),
+                   row(t));
+
+        assertRows(execute("SELECT " + fName + "(tup) FROM %s WHERE key = 1"),
+                   row(t));
+    }
+
+    @Test
+    public void testJavascriptTupleTypeCollection() throws Throwable
+    {
+        String tupleTypeDef = "tuple<double, list<double>, set<text>, map<int, boolean>>";
+        createTable("CREATE TABLE %s (key int primary key, tup frozen<" + tupleTypeDef + ">)");
+
+        String fTup1 = createFunction(KEYSPACE_PER_TEST, tupleTypeDef,
+                "CREATE FUNCTION %s( tup " + tupleTypeDef + " ) " +
+                "RETURNS NULL ON NULL INPUT " +
+                "RETURNS tuple<double, list<double>, set<text>, map<int, boolean>> " +
+                "LANGUAGE javascript\n" +
+                "AS $$" +
+                "       tup;$$;");
+        String fTup2 = createFunction(KEYSPACE_PER_TEST, tupleTypeDef,
+                "CREATE FUNCTION %s( tup " + tupleTypeDef + " ) " +
+                "RETURNS NULL ON NULL INPUT " +
+                "RETURNS double " +
+                "LANGUAGE javascript\n" +
+                "AS $$" +
+                "       tup.getDouble(0);$$;");
+        String fTup3 = createFunction(KEYSPACE_PER_TEST, tupleTypeDef,
+                "CREATE FUNCTION %s( tup " + tupleTypeDef + " ) " +
+                "RETURNS NULL ON NULL INPUT " +
+                "RETURNS list<double> " +
+                "LANGUAGE javascript\n" +
+                "AS $$" +
+                "       tup.getList(1, java.lang.Class.forName(\"java.lang.Double\"));$$;");
+        String fTup4 = createFunction(KEYSPACE_PER_TEST, tupleTypeDef,
+                "CREATE FUNCTION %s( tup " + tupleTypeDef + " ) " +
+                "RETURNS NULL ON NULL INPUT " +
+                "RETURNS set<text> " +
+                "LANGUAGE javascript\n" +
+                "AS $$" +
+                "       tup.getSet(2, java.lang.Class.forName(\"java.lang.String\"));$$;");
+        String fTup5 = createFunction(KEYSPACE_PER_TEST, tupleTypeDef,
+                "CREATE FUNCTION %s( tup " + tupleTypeDef + " ) " +
+                "RETURNS NULL ON NULL INPUT " +
+                "RETURNS map<int, boolean> " +
+                "LANGUAGE javascript\n" +
+                "AS $$" +
+                "       tup.getMap(3, java.lang.Class.forName(\"java.lang.Integer\"), java.lang.Class.forName(\"java.lang.Boolean\"));$$;");
+
+        List<Double> list = Arrays.asList(1d, 2d, 3d);
+        Set<String> set = new TreeSet<>(Arrays.asList("one", "three", "two"));
+        Map<Integer, Boolean> map = new TreeMap<>();
+        map.put(1, true);
+        map.put(2, false);
+        map.put(3, true);
+
+        Object t = tuple(1d, list, set, map);
+
+        execute("INSERT INTO %s (key, tup) VALUES (1, ?)", t);
+
+        assertRows(execute("SELECT " + fTup1 + "(tup) FROM %s WHERE key = 1"),
+                   row(t));
+        assertRows(execute("SELECT " + fTup2 + "(tup) FROM %s WHERE key = 1"),
+                   row(1d));
+        assertRows(execute("SELECT " + fTup3 + "(tup) FROM %s WHERE key = 1"),
+                   row(list));
+        assertRows(execute("SELECT " + fTup4 + "(tup) FROM %s WHERE key = 1"),
+                   row(set));
+        assertRows(execute("SELECT " + fTup5 + "(tup) FROM %s WHERE key = 1"),
+                   row(map));
+
+        // same test - but via native protocol
+        TupleType tType = TupleType.of(DataType.cdouble(),
+                                       DataType.list(DataType.cdouble()),
+                                       DataType.set(DataType.text()),
+                                       DataType.map(DataType.cint(), DataType.cboolean()));
+        TupleValue tup = tType.newValue(1d, list, set, map);
+        for (int version = Server.VERSION_2; version <= maxProtocolVersion; version++)
+        {
+            assertRowsNet(version,
+                          executeNet(version, "SELECT " + fTup1 + "(tup) FROM %s WHERE key = 1"),
+                          row(tup));
+            assertRowsNet(version,
+                          executeNet(version, "SELECT " + fTup2 + "(tup) FROM %s WHERE key = 1"),
+                          row(1d));
+            assertRowsNet(version,
+                          executeNet(version, "SELECT " + fTup3 + "(tup) FROM %s WHERE key = 1"),
+                          row(list));
+            assertRowsNet(version,
+                          executeNet(version, "SELECT " + fTup4 + "(tup) FROM %s WHERE key = 1"),
+                          row(set));
+            assertRowsNet(version,
+                          executeNet(version, "SELECT " + fTup5 + "(tup) FROM %s WHERE key = 1"),
+                          row(map));
+        }
+    }
+
+    @Test
+    public void testJavascriptUserType() throws Throwable
+    {
+        String type = createType("CREATE TYPE %s (txt text, i int)");
+
+        createTable("CREATE TABLE %s (key int primary key, udt frozen<" + type + ">)");
+
+        String fUdt1 = createFunction(KEYSPACE, type,
+                              "CREATE FUNCTION %s( udt " + type + " ) " +
+                              "RETURNS NULL ON NULL INPUT " +
+                              "RETURNS " + type + " " +
+                              "LANGUAGE javascript\n" +
+                              "AS $$" +
+                              "     udt;$$;");
+        String fUdt2 = createFunction(KEYSPACE, type,
+                              "CREATE FUNCTION %s( udt " + type + " ) " +
+                              "RETURNS NULL ON NULL INPUT " +
+                              "RETURNS text " +
+                              "LANGUAGE javascript\n" +
+                              "AS $$" +
+                              "     udt.getString(\"txt\");$$;");
+        String fUdt3 = createFunction(KEYSPACE, type,
+                                      "CREATE FUNCTION %s( udt " + type + " ) " +
+                                      "RETURNS NULL ON NULL INPUT " +
+                                      "RETURNS int " +
+                                      "LANGUAGE javascript\n" +
+                                      "AS $$" +
+                                      "     udt.getInt(\"i\");$$;");
+
+        execute("INSERT INTO %s (key, udt) VALUES (1, {txt: 'one', i:1})");
+
+        UntypedResultSet rows = execute("SELECT " + fUdt1 + "(udt) FROM %s WHERE key = 1");
+        Assert.assertEquals(1, rows.size());
+        assertRows(execute("SELECT " + fUdt2 + "(udt) FROM %s WHERE key = 1"),
+                   row("one"));
+        assertRows(execute("SELECT " + fUdt3 + "(udt) FROM %s WHERE key = 1"),
+                   row(1));
+    }
+
+    @Test
+    public void testJavascriptUTCollections() throws Throwable
+    {
+        String type = createType("CREATE TYPE %s (txt text, i int)");
+
+        createTable(String.format("CREATE TABLE %%s " +
+                                  "(key int primary key, lst list<frozen<%s>>, st set<frozen<%s>>, mp map<int, frozen<%s>>)",
+                                  type, type, type));
+
+        String fName = createFunction(KEYSPACE, "list<frozen<" + type + ">>",
+                                      "CREATE FUNCTION %s( lst list<frozen<" + type + ">> ) " +
+                                      "RETURNS NULL ON NULL INPUT " +
+                                      "RETURNS text " +
+                                      "LANGUAGE javascript\n" +
+                                      "AS $$" +
+                                      "        lst.get(1).getString(\"txt\");$$;");
+        createFunctionOverload(fName, "set<frozen<" + type + ">>",
+                               "CREATE FUNCTION %s( st set<frozen<" + type + ">> ) " +
+                               "R

<TRUNCATED>

[13/32] cassandra git commit: 2.2 commit for CASSANDRA-9160

Posted by jm...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/01115f72/test/unit/org/apache/cassandra/cql3/validation/operations/SelectMultiColumnRelationTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/SelectMultiColumnRelationTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/SelectMultiColumnRelationTest.java
new file mode 100644
index 0000000..552e39e
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/SelectMultiColumnRelationTest.java
@@ -0,0 +1,962 @@
+/*
+ * 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.validation.operations;
+
+import org.junit.Test;
+
+import org.apache.cassandra.cql3.CQLTester;
+
+import static org.junit.Assert.assertEquals;
+
+public class SelectMultiColumnRelationTest extends CQLTester
+{
+    @Test
+    public void testSingleClusteringInvalidQueries() throws Throwable
+    {
+        for (String compactOption : new String[] { "", " WITH COMPACT STORAGE" })
+        {
+            createTable("CREATE TABLE %s (a int, b int, c int, PRIMARY KEY (a, b))" + compactOption);
+
+            assertInvalidSyntax("SELECT * FROM %s WHERE () = (?, ?)", 1, 2);
+            assertInvalidMessage("b cannot be restricted by more than one relation if it includes an Equal",
+                                 "SELECT * FROM %s WHERE a = 0 AND (b) = (?) AND (b) > (?)", 0, 0);
+            assertInvalidMessage("More than one restriction was found for the start bound on b",
+                                 "SELECT * FROM %s WHERE a = 0 AND (b) > (?) AND (b) > (?)", 0, 1);
+            assertInvalidMessage("Multi-column relations can only be applied to clustering columns but was applied to: a",
+                                 "SELECT * FROM %s WHERE (a, b) = (?, ?)", 0, 0);
+        }
+    }
+
+    @Test
+    public void testMultiClusteringInvalidQueries() throws Throwable
+    {
+        for (String compactOption : new String[] { "", " WITH COMPACT STORAGE" })
+        {
+            createTable("CREATE TABLE %s (a int, b int, c int, d int, PRIMARY KEY (a, b, c, d))" + compactOption);
+
+            assertInvalidSyntax("SELECT * FROM %s WHERE a = 0 AND (b, c) > ()");
+            assertInvalidMessage("Expected 2 elements in value tuple, but got 3: (?, ?, ?)",
+                                 "SELECT * FROM %s WHERE a = 0 AND (b, c) > (?, ?, ?)", 1, 2, 3);
+            assertInvalidMessage("Invalid null value in condition for column c",
+                                 "SELECT * FROM %s WHERE a = 0 AND (b, c) > (?, ?)", 1, null);
+
+            // Wrong order of columns
+            assertInvalidMessage("Clustering columns must appear in the PRIMARY KEY order in multi-column relations: (d, c, b) = (?, ?, ?)",
+                                 "SELECT * FROM %s WHERE a = 0 AND (d, c, b) = (?, ?, ?)", 0, 0, 0);
+            assertInvalidMessage("Clustering columns must appear in the PRIMARY KEY order in multi-column relations: (d, c, b) > (?, ?, ?)",
+                                 "SELECT * FROM %s WHERE a = 0 AND (d, c, b) > (?, ?, ?)", 0, 0, 0);
+
+            // Wrong number of values
+            assertInvalidMessage("Expected 3 elements in value tuple, but got 2: (?, ?)",
+                                 "SELECT * FROM %s WHERE a=0 AND (b, c, d) IN ((?, ?))", 0, 1);
+            assertInvalidMessage("Expected 3 elements in value tuple, but got 5: (?, ?, ?, ?, ?)",
+                                 "SELECT * FROM %s WHERE a=0 AND (b, c, d) IN ((?, ?, ?, ?, ?))", 0, 1, 2, 3, 4);
+
+            // Missing first clustering column
+            assertInvalidMessage("PRIMARY KEY column \"c\" cannot be restricted as preceding column \"b\" is not restricted",
+                                 "SELECT * FROM %s WHERE a = 0 AND (c, d) = (?, ?)", 0, 0);
+            assertInvalidMessage("PRIMARY KEY column \"c\" cannot be restricted as preceding column \"b\" is not restricted",
+                                 "SELECT * FROM %s WHERE a = 0 AND (c, d) > (?, ?)", 0, 0);
+
+            // Nulls
+            assertInvalidMessage("Invalid null value in condition for columns: [b, c, d]",
+                                 "SELECT * FROM %s WHERE a = 0 AND (b, c, d) IN ((?, ?, ?))", 1, 2, null);
+
+            // Wrong type for 'd'
+            assertInvalid("SELECT * FROM %s WHERE a = 0 AND (b, c, d) = (?, ?, ?)", 1, 2, "foobar");
+            assertInvalid("SELECT * FROM %s WHERE a = 0 AND b = (?, ?, ?)", 1, 2, 3);
+
+            // Mix single and tuple inequalities
+            assertInvalidMessage("Mixing single column relations and multi column relations on clustering columns is not allowed",
+                                 "SELECT * FROM %s WHERE a = 0 AND (b, c, d) > (?, ?, ?) AND b < ?", 0, 1, 0, 1);
+            assertInvalidMessage("Mixing single column relations and multi column relations on clustering columns is not allowed",
+                                 "SELECT * FROM %s WHERE a = 0 AND (b, c, d) > (?, ?, ?) AND c < ?", 0, 1, 0, 1);
+            assertInvalidMessage("Mixing single column relations and multi column relations on clustering columns is not allowed",
+                                 "SELECT * FROM %s WHERE a = 0 AND b > ? AND (b, c, d) < (?, ?, ?)", 1, 1, 1, 0);
+            assertInvalidMessage("Mixing single column relations and multi column relations on clustering columns is not allowed",
+                                 "SELECT * FROM %s WHERE a = 0 AND c > ? AND (b, c, d) < (?, ?, ?)", 1, 1, 1, 0);
+
+            assertInvalidMessage("Multi-column relations can only be applied to clustering columns but was applied to: a",
+                                 "SELECT * FROM %s WHERE (a, b, c, d) IN ((?, ?, ?, ?))", 0, 1, 2, 3);
+            assertInvalidMessage("PRIMARY KEY column \"c\" cannot be restricted as preceding column \"b\" is not restricted",
+                                 "SELECT * FROM %s WHERE (c, d) IN ((?, ?))", 0, 1);
+
+            assertInvalidMessage("Clustering column \"c\" cannot be restricted (preceding column \"b\" is restricted by a non-EQ relation)",
+                                 "SELECT * FROM %s WHERE a = ? AND b > ?  AND (c, d) IN ((?, ?))", 0, 0, 0, 0);
+
+            assertInvalidMessage("Clustering column \"c\" cannot be restricted (preceding column \"b\" is restricted by a non-EQ relation)",
+                                 "SELECT * FROM %s WHERE a = ? AND b > ?  AND (c, d) > (?, ?)", 0, 0, 0, 0);
+            assertInvalidMessage("PRIMARY KEY column \"c\" cannot be restricted (preceding column \"b\" is restricted by a non-EQ relation)",
+                                 "SELECT * FROM %s WHERE a = ? AND (c, d) > (?, ?) AND b > ?  ", 0, 0, 0, 0);
+            assertInvalidMessage("Column \"c\" cannot be restricted by two tuple-notation inequalities not starting with the same column",
+                                 "SELECT * FROM %s WHERE a = ? AND (b, c) > (?, ?) AND (b) < (?) AND (c) < (?)", 0, 0, 0, 0, 0);
+            assertInvalidMessage("Column \"c\" cannot be restricted by two tuple-notation inequalities not starting with the same column",
+                                 "SELECT * FROM %s WHERE a = ? AND (c) < (?) AND (b, c) > (?, ?) AND (b) < (?)", 0, 0, 0, 0, 0);
+            assertInvalidMessage("Clustering column \"c\" cannot be restricted (preceding column \"b\" is restricted by a non-EQ relation)",
+                                 "SELECT * FROM %s WHERE a = ? AND (b) < (?) AND (c) < (?) AND (b, c) > (?, ?)", 0, 0, 0, 0, 0);
+
+            assertInvalidMessage("Column \"c\" cannot be restricted by two tuple-notation inequalities not starting with the same column",
+                                 "SELECT * FROM %s WHERE a = ? AND (b, c) > (?, ?) AND (c) < (?)", 0, 0, 0, 0);
+        }
+    }
+
+    @Test
+    public void testMultiAndSingleColumnRelationMix() throws Throwable
+    {
+        for (String compactOption : new String[]{"", " WITH COMPACT STORAGE"})
+        {
+            createTable("CREATE TABLE %s (a int, b int, c int, d int, PRIMARY KEY (a, b, c, d))" + compactOption);
+
+            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 0, 0);
+            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 1, 0);
+            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 1, 1);
+
+            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 0, 0);
+            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 1, 0);
+            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 1, 1);
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? and b = ? and (c, d) = (?, ?)", 0, 1, 0, 0),
+                       row(0, 1, 0, 0));
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? and b IN (?, ?) and (c, d) = (?, ?)", 0, 0, 1, 0, 0),
+                       row(0, 0, 0, 0),
+                       row(0, 1, 0, 0));
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? and b = ? and (c) IN ((?))", 0, 1, 0),
+                       row(0, 1, 0, 0));
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? and b IN (?, ?) and (c) IN ((?))", 0, 0, 1, 0),
+                       row(0, 0, 0, 0),
+                       row(0, 1, 0, 0));
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? and b = ? and (c) IN ((?), (?))", 0, 1, 0, 1),
+                       row(0, 1, 0, 0),
+                       row(0, 1, 1, 0),
+                       row(0, 1, 1, 1));
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? and b = ? and (c, d) IN ((?, ?))", 0, 1, 0, 0),
+                       row(0, 1, 0, 0));
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? and b = ? and (c, d) IN ((?, ?), (?, ?))", 0, 1, 0, 0, 1, 1),
+                       row(0, 1, 0, 0),
+                       row(0, 1, 1, 1));
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? and b IN (?, ?) and (c, d) IN ((?, ?), (?, ?))", 0, 0, 1, 0, 0, 1, 1),
+                       row(0, 0, 0, 0),
+                       row(0, 0, 1, 1),
+                       row(0, 1, 0, 0),
+                       row(0, 1, 1, 1));
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? and b = ? and (c, d) > (?, ?)", 0, 1, 0, 0),
+                       row(0, 1, 1, 0),
+                       row(0, 1, 1, 1));
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? and b IN (?, ?) and (c, d) > (?, ?)", 0, 0, 1, 0, 0),
+                       row(0, 0, 1, 0),
+                       row(0, 0, 1, 1),
+                       row(0, 1, 1, 0),
+                       row(0, 1, 1, 1));
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? and b = ? and (c, d) > (?, ?) and (c) <= (?) ", 0, 1, 0, 0, 1),
+                       row(0, 1, 1, 0),
+                       row(0, 1, 1, 1));
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? and b = ? and (c, d) >= (?, ?) and (c, d) < (?, ?)", 0, 1, 0, 0, 1, 1),
+                       row(0, 1, 0, 0),
+                       row(0, 1, 1, 0));
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b, c) = (?, ?) and d = ?", 0, 0, 1, 0),
+                       row(0, 0, 1, 0));
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b, c) IN ((?, ?), (?, ?)) and d = ?", 0, 0, 1, 0, 0, 0),
+                       row(0, 0, 0, 0),
+                       row(0, 0, 1, 0));
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? and b = ? and (c) = (?) and d = ?", 0, 0, 1, 0),
+                       row(0, 0, 1, 0));
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b, c) = (?, ?) and d IN (?, ?)", 0, 0, 1, 0, 2),
+                       row(0, 0, 1, 0));
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? and b = ? and (c) = (?) and d IN (?, ?)", 0, 0, 1, 0, 2),
+                       row(0, 0, 1, 0));
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b, c) = (?, ?) and d >= ?", 0, 0, 1, 0),
+                       row(0, 0, 1, 0),
+                       row(0, 0, 1, 1));
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? and d < 1 and (b, c) = (?, ?) and d >= ?", 0, 0, 1, 0),
+                       row(0, 0, 1, 0));
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? and d < 1 and (b, c) IN ((?, ?), (?, ?)) and d >= ?", 0, 0, 1, 0, 0, 0),
+                       row(0, 0, 0, 0),
+                       row(0, 0, 1, 0));
+        }
+    }
+
+    @Test
+    public void testSeveralMultiColumnRelation() throws Throwable
+    {
+        for (String compactOption : new String[]{"", " WITH COMPACT STORAGE"})
+        {
+            createTable("CREATE TABLE %s (a int, b int, c int, d int, PRIMARY KEY (a, b, c, d))" + compactOption);
+
+            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 0, 0);
+            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 1, 0);
+            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 1, 1);
+
+            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 0, 0);
+            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 1, 0);
+            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 1, 1);
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b) = (?) and (c, d) = (?, ?)", 0, 1, 0, 0),
+                       row(0, 1, 0, 0));
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b) IN ((?), (?)) and (c, d) = (?, ?)", 0, 0, 1, 0, 0),
+                       row(0, 0, 0, 0),
+                       row(0, 1, 0, 0));
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b) = (?) and (c) IN ((?))", 0, 1, 0),
+                       row(0, 1, 0, 0));
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b) IN ((?),(?)) and (c) IN ((?))", 0, 0, 1, 0),
+                       row(0, 0, 0, 0),
+                       row(0, 1, 0, 0));
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b) = (?) and (c) IN ((?), (?))", 0, 1, 0, 1),
+                       row(0, 1, 0, 0),
+                       row(0, 1, 1, 0),
+                       row(0, 1, 1, 1));
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b) = (?) and (c, d) IN ((?, ?))", 0, 1, 0, 0),
+                       row(0, 1, 0, 0));
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b) = (?) and (c, d) IN ((?, ?), (?, ?))", 0, 1, 0, 0, 1, 1),
+                       row(0, 1, 0, 0),
+                       row(0, 1, 1, 1));
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b) IN ((?), (?)) and (c, d) IN ((?, ?), (?, ?))", 0, 0, 1, 0, 0, 1, 1),
+                       row(0, 0, 0, 0),
+                       row(0, 0, 1, 1),
+                       row(0, 1, 0, 0),
+                       row(0, 1, 1, 1));
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b) = (?) and (c, d) > (?, ?)", 0, 1, 0, 0),
+                       row(0, 1, 1, 0),
+                       row(0, 1, 1, 1));
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b) IN ((?),(?)) and (c, d) > (?, ?)", 0, 0, 1, 0, 0),
+                       row(0, 0, 1, 0),
+                       row(0, 0, 1, 1),
+                       row(0, 1, 1, 0),
+                       row(0, 1, 1, 1));
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b) = (?) and (c, d) > (?, ?) and (c) <= (?) ", 0, 1, 0, 0, 1),
+                       row(0, 1, 1, 0),
+                       row(0, 1, 1, 1));
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b) = (?) and (c, d) >= (?, ?) and (c, d) < (?, ?)", 0, 1, 0, 0, 1, 1),
+                       row(0, 1, 0, 0),
+                       row(0, 1, 1, 0));
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b, c) = (?, ?) and d = ?", 0, 0, 1, 0),
+                       row(0, 0, 1, 0));
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b, c) IN ((?, ?), (?, ?)) and d = ?", 0, 0, 1, 0, 0, 0),
+                       row(0, 0, 0, 0),
+                       row(0, 0, 1, 0));
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? and (d) < (1) and (b, c) = (?, ?) and (d) >= (?)", 0, 0, 1, 0),
+                       row(0, 0, 1, 0));
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? and (d) < (1) and (b, c) IN ((?, ?), (?, ?)) and (d) >= (?)", 0, 0, 1, 0, 0, 0),
+                       row(0, 0, 0, 0),
+                       row(0, 0, 1, 0));
+        }
+    }
+
+    @Test
+    public void testSinglePartitionInvalidQueries() throws Throwable
+    {
+        for (String compactOption : new String[]{"", " WITH COMPACT STORAGE"})
+        {
+            createTable("CREATE TABLE %s (a int PRIMARY KEY, b int)" + compactOption);
+            assertInvalidMessage("Multi-column relations can only be applied to clustering columns but was applied to: a",
+                                 "SELECT * FROM %s WHERE (a) > (?)", 0);
+            assertInvalidMessage("Multi-column relations can only be applied to clustering columns but was applied to: a",
+                                 "SELECT * FROM %s WHERE (a) = (?)", 0);
+            assertInvalidMessage("Multi-column relations can only be applied to clustering columns but was applied to: b",
+                                 "SELECT * FROM %s WHERE (b) = (?)", 0);
+        }
+    }
+
+    @Test
+    public void testSingleClustering() throws Throwable
+    {
+        for (String compactOption : new String[]{"", " WITH COMPACT STORAGE"})
+        {
+            createTable("CREATE TABLE %s (a int, b int, c int, PRIMARY KEY (a, b))" + compactOption);
+
+            execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, 0, 0);
+            execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, 1, 0);
+            execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, 2, 0);
+
+            // Equalities
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) = (?)", 0, 1),
+                    row(0, 1, 0)
+            );
+
+            // Same but check the whole tuple can be prepared
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) = ?", 0, tuple(1)),
+                    row(0, 1, 0)
+            );
+
+            assertEmpty(execute("SELECT * FROM %s WHERE a = ? AND (b) = (?)", 0, 3));
+
+            // Inequalities
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) > (?)", 0, 0),
+                    row(0, 1, 0),
+                    row(0, 2, 0)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) >= (?)", 0, 1),
+                    row(0, 1, 0),
+                    row(0, 2, 0)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) < (?)", 0, 2),
+                    row(0, 0, 0),
+                    row(0, 1, 0)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) <= (?)", 0, 1),
+                    row(0, 0, 0),
+                    row(0, 1, 0)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) > (?) AND (b) < (?)", 0, 0, 2),
+                    row(0, 1, 0)
+            );
+        }
+    }
+
+    @Test
+    public void testNonEqualsRelation() throws Throwable
+    {
+        for (String compactOption : new String[]{"", " WITH COMPACT STORAGE"})
+        {
+            createTable("CREATE TABLE %s (a int PRIMARY KEY, b int)" + compactOption);
+            assertInvalidMessage("Unsupported \"!=\" relation: (b) != (0)",
+                    "SELECT * FROM %s WHERE a = 0 AND (b) != (0)");
+        }
+    }
+
+    @Test
+    public void testMultipleClustering() throws Throwable
+    {
+        for (String compactOption : new String[]{"", " WITH COMPACT STORAGE"})
+        {
+            createTable("CREATE TABLE %s (a int, b int, c int, d int, PRIMARY KEY (a, b, c, d))" + compactOption);
+
+            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 0, 0);
+            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 1, 0);
+            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 1, 1);
+
+            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 0, 0);
+            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 1, 0);
+            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 1, 1);
+
+            // Empty query
+            assertEmpty(execute("SELECT * FROM %s WHERE a = 0 AND (b, c, d) IN ()"));
+
+            // Equalities
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) = (?)", 0, 1),
+                    row(0, 1, 0, 0),
+                    row(0, 1, 1, 0),
+                    row(0, 1, 1, 1)
+            );
+
+            // Same with whole tuple prepared
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) = ?", 0, tuple(1)),
+                    row(0, 1, 0, 0),
+                    row(0, 1, 1, 0),
+                    row(0, 1, 1, 1)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c) = (?, ?)", 0, 1, 1),
+                    row(0, 1, 1, 0),
+                    row(0, 1, 1, 1)
+            );
+
+            // Same with whole tuple prepared
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c) = ?", 0, tuple(1, 1)),
+                    row(0, 1, 1, 0),
+                    row(0, 1, 1, 1)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) = (?, ?, ?)", 0, 1, 1, 1),
+                    row(0, 1, 1, 1)
+            );
+
+            // Same with whole tuple prepared
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) = ?", 0, tuple(1, 1, 1)),
+                    row(0, 1, 1, 1)
+            );
+
+            // Inequalities
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) > (?)", 0, 0),
+                    row(0, 1, 0, 0),
+                    row(0, 1, 1, 0),
+                    row(0, 1, 1, 1)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) >= (?)", 0, 0),
+                    row(0, 0, 0, 0),
+                    row(0, 0, 1, 0),
+                    row(0, 0, 1, 1),
+                    row(0, 1, 0, 0),
+                    row(0, 1, 1, 0),
+                    row(0, 1, 1, 1)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c) > (?, ?)", 0, 1, 0),
+                    row(0, 1, 1, 0),
+                    row(0, 1, 1, 1)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c) >= (?, ?)", 0, 1, 0),
+                    row(0, 1, 0, 0),
+                    row(0, 1, 1, 0),
+                    row(0, 1, 1, 1)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) > (?, ?, ?)", 0, 1, 1, 0),
+                    row(0, 1, 1, 1)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) >= (?, ?, ?)", 0, 1, 1, 0),
+                    row(0, 1, 1, 0),
+                    row(0, 1, 1, 1)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) < (?)", 0, 1),
+                    row(0, 0, 0, 0),
+                    row(0, 0, 1, 0),
+                    row(0, 0, 1, 1)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) <= (?)", 0, 1),
+                    row(0, 0, 0, 0),
+                    row(0, 0, 1, 0),
+                    row(0, 0, 1, 1),
+                    row(0, 1, 0, 0),
+                    row(0, 1, 1, 0),
+                    row(0, 1, 1, 1)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c) < (?, ?)", 0, 0, 1),
+                    row(0, 0, 0, 0)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c) <= (?, ?)", 0, 0, 1),
+                    row(0, 0, 0, 0),
+                    row(0, 0, 1, 0),
+                    row(0, 0, 1, 1)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) < (?, ?, ?)", 0, 0, 1, 1),
+                    row(0, 0, 0, 0),
+                    row(0, 0, 1, 0)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) <= (?, ?, ?)", 0, 0, 1, 1),
+                    row(0, 0, 0, 0),
+                    row(0, 0, 1, 0),
+                    row(0, 0, 1, 1)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) > (?, ?, ?) AND (b) < (?)", 0, 0, 1, 0, 1),
+                    row(0, 0, 1, 1)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) > (?, ?, ?) AND (b, c) < (?, ?)", 0, 0, 1, 1, 1, 1),
+                    row(0, 1, 0, 0)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) > (?, ?, ?) AND (b, c, d) < (?, ?, ?)", 0, 0, 1, 1, 1, 1, 0),
+                    row(0, 1, 0, 0)
+            );
+
+            // Same with whole tuple prepared
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) > ? AND (b, c, d) < ?", 0, tuple(0, 1, 1), tuple(1, 1, 0)),
+                    row(0, 1, 0, 0)
+            );
+
+            // reversed
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) > (?) ORDER BY b DESC, c DESC, d DESC", 0, 0),
+                    row(0, 1, 1, 1),
+                    row(0, 1, 1, 0),
+                    row(0, 1, 0, 0)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) >= (?) ORDER BY b DESC, c DESC, d DESC", 0, 0),
+                    row(0, 1, 1, 1),
+                    row(0, 1, 1, 0),
+                    row(0, 1, 0, 0),
+                    row(0, 0, 1, 1),
+                    row(0, 0, 1, 0),
+                    row(0, 0, 0, 0)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c) > (?, ?) ORDER BY b DESC, c DESC, d DESC", 0, 1, 0),
+                    row(0, 1, 1, 1),
+                    row(0, 1, 1, 0)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c) >= (?, ?) ORDER BY b DESC, c DESC, d DESC", 0, 1, 0),
+                    row(0, 1, 1, 1),
+                    row(0, 1, 1, 0),
+                    row(0, 1, 0, 0)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) > (?, ?, ?) ORDER BY b DESC, c DESC, d DESC", 0, 1, 1, 0),
+                    row(0, 1, 1, 1)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) >= (?, ?, ?) ORDER BY b DESC, c DESC, d DESC", 0, 1, 1, 0),
+                    row(0, 1, 1, 1),
+                    row(0, 1, 1, 0)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) < (?) ORDER BY b DESC, c DESC, d DESC", 0, 1),
+                    row(0, 0, 1, 1),
+                    row(0, 0, 1, 0),
+                    row(0, 0, 0, 0)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) <= (?) ORDER BY b DESC, c DESC, d DESC", 0, 1),
+                    row(0, 1, 1, 1),
+                    row(0, 1, 1, 0),
+                    row(0, 1, 0, 0),
+                    row(0, 0, 1, 1),
+                    row(0, 0, 1, 0),
+                    row(0, 0, 0, 0)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c) < (?, ?) ORDER BY b DESC, c DESC, d DESC", 0, 0, 1),
+                    row(0, 0, 0, 0)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c) <= (?, ?) ORDER BY b DESC, c DESC, d DESC", 0, 0, 1),
+                    row(0, 0, 1, 1),
+                    row(0, 0, 1, 0),
+                    row(0, 0, 0, 0)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) < (?, ?, ?) ORDER BY b DESC, c DESC, d DESC", 0, 0, 1, 1),
+                    row(0, 0, 1, 0),
+                    row(0, 0, 0, 0)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) <= (?, ?, ?) ORDER BY b DESC, c DESC, d DESC", 0, 0, 1, 1),
+                    row(0, 0, 1, 1),
+                    row(0, 0, 1, 0),
+                    row(0, 0, 0, 0)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) > (?, ?, ?) AND (b) < (?) ORDER BY b DESC, c DESC, d DESC", 0, 0, 1, 0, 1),
+                    row(0, 0, 1, 1)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) > (?, ?, ?) AND (b, c) < (?, ?) ORDER BY b DESC, c DESC, d DESC", 0, 0, 1, 1, 1, 1),
+                    row(0, 1, 0, 0)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) > (?, ?, ?) AND (b, c, d) < (?, ?, ?) ORDER BY b DESC, c DESC, d DESC", 0, 0, 1, 1, 1, 1, 0),
+                    row(0, 1, 0, 0)
+            );
+
+            // IN
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) IN ((?, ?, ?), (?, ?, ?))", 0, 0, 1, 0, 0, 1, 1),
+                    row(0, 0, 1, 0),
+                    row(0, 0, 1, 1)
+            );
+
+            // same query but with whole tuple prepared
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) IN (?, ?)", 0, tuple(0, 1, 0), tuple(0, 1, 1)),
+                    row(0, 0, 1, 0),
+                    row(0, 0, 1, 1)
+            );
+
+            // same query but with whole IN list prepared
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) IN ?", 0, list(tuple(0, 1, 0), tuple(0, 1, 1))),
+                    row(0, 0, 1, 0),
+                    row(0, 0, 1, 1)
+            );
+
+            // same query, but reversed order for the IN values
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) IN (?, ?)", 0, tuple(0, 1, 1), tuple(0, 1, 0)),
+                    row(0, 0, 1, 0),
+                    row(0, 0, 1, 1)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b, c) IN ((?, ?))", 0, 0, 1),
+                    row(0, 0, 1, 0),
+                    row(0, 0, 1, 1)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b) IN ((?))", 0, 0),
+                    row(0, 0, 0, 0),
+                    row(0, 0, 1, 0),
+                    row(0, 0, 1, 1)
+            );
+
+            assertEmpty(execute("SELECT * FROM %s WHERE a = ? and (b) IN ()", 0));
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c) IN ((?, ?)) ORDER BY b DESC, c DESC, d DESC", 0, 0, 1),
+                    row(0, 0, 1, 1),
+                    row(0, 0, 1, 0)
+            );
+
+            assertEmpty(execute("SELECT * FROM %s WHERE a = ? AND (b, c) IN () ORDER BY b DESC, c DESC, d DESC", 0));
+
+            // IN on both partition key and clustering key
+            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 1, 0, 0, 0);
+            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 1, 0, 1, 0);
+            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 1, 0, 1, 1);
+
+            assertRows(execute("SELECT * FROM %s WHERE a IN (?, ?) AND (b, c, d) IN (?, ?)", 0, 1, tuple(0, 1, 0), tuple(0, 1, 1)),
+                    row(0, 0, 1, 0),
+                    row(0, 0, 1, 1),
+                    row(1, 0, 1, 0),
+                    row(1, 0, 1, 1)
+            );
+
+            // same but with whole IN lists prepared
+            assertRows(execute("SELECT * FROM %s WHERE a IN ? AND (b, c, d) IN ?", list(0, 1), list(tuple(0, 1, 0), tuple(0, 1, 1))),
+                    row(0, 0, 1, 0),
+                    row(0, 0, 1, 1),
+                    row(1, 0, 1, 0),
+                    row(1, 0, 1, 1)
+            );
+
+            // same query, but reversed order for the IN values
+            assertRows(execute("SELECT * FROM %s WHERE a IN (?, ?) AND (b, c, d) IN (?, ?)", 1, 0, tuple(0, 1, 1), tuple(0, 1, 0)),
+                    row(0, 0, 1, 0),
+                    row(0, 0, 1, 1),
+                    row(1, 0, 1, 0),
+                    row(1, 0, 1, 1)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a IN (?, ?) and (b, c) IN ((?, ?))", 0, 1, 0, 1),
+                    row(0, 0, 1, 0),
+                    row(0, 0, 1, 1),
+                    row(1, 0, 1, 0),
+                    row(1, 0, 1, 1)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a IN (?, ?) and (b) IN ((?))", 0, 1, 0),
+                    row(0, 0, 0, 0),
+                    row(0, 0, 1, 0),
+                    row(0, 0, 1, 1),
+                    row(1, 0, 0, 0),
+                    row(1, 0, 1, 0),
+                    row(1, 0, 1, 1)
+            );
+        }
+    }
+
+    @Test
+    public void testMultipleClusteringReversedComponents() throws Throwable
+    {
+        for (String compactOption : new String[]{"", " COMPACT STORAGE AND"})
+        {
+            createTable("CREATE TABLE %s (a int, b int, c int, d int, PRIMARY KEY (a, b, c, d)) WITH" + compactOption + " CLUSTERING ORDER BY (b DESC, c ASC, d DESC)");
+
+            // b and d are reversed in the clustering order
+            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 0, 0);
+            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 1, 1);
+            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 1, 0);
+
+            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 0, 0);
+            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 1, 1);
+            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 1, 0);
+
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) > (?)", 0, 0),
+                    row(0, 1, 0, 0),
+                    row(0, 1, 1, 1),
+                    row(0, 1, 1, 0)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) >= (?)", 0, 0),
+                    row(0, 1, 0, 0),
+                    row(0, 1, 1, 1),
+                    row(0, 1, 1, 0),
+                    row(0, 0, 0, 0),
+                    row(0, 0, 1, 1),
+                    row(0, 0, 1, 0)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) < (?)", 0, 1),
+                    row(0, 0, 0, 0),
+                    row(0, 0, 1, 1),
+                    row(0, 0, 1, 0)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) <= (?)", 0, 1),
+                    row(0, 1, 0, 0),
+                    row(0, 1, 1, 1),
+                    row(0, 1, 1, 0),
+                    row(0, 0, 0, 0),
+                    row(0, 0, 1, 1),
+                    row(0, 0, 1, 0)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a=? AND (b, c, d) IN ((?, ?, ?), (?, ?, ?))", 0, 1, 1, 1, 0, 1, 1),
+                    row(0, 1, 1, 1),
+                    row(0, 0, 1, 1)
+            );
+
+            // same query, but reversed order for the IN values
+            assertRows(execute("SELECT * FROM %s WHERE a=? AND (b, c, d) IN ((?, ?, ?), (?, ?, ?))", 0, 0, 1, 1, 1, 1, 1),
+                    row(0, 1, 1, 1),
+                    row(0, 0, 1, 1)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) IN (?, ?, ?, ?, ?, ?)",
+                            0, tuple(1, 0, 0), tuple(1, 1, 1), tuple(1, 1, 0), tuple(0, 0, 0), tuple(0, 1, 1), tuple(0, 1, 0)),
+                    row(0, 1, 0, 0),
+                    row(0, 1, 1, 1),
+                    row(0, 1, 1, 0),
+                    row(0, 0, 0, 0),
+                    row(0, 0, 1, 1),
+                    row(0, 0, 1, 0)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c) IN (?)", 0, tuple(0, 1)),
+                    row(0, 0, 1, 1),
+                    row(0, 0, 1, 0)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c) IN (?)", 0, tuple(0, 0)),
+                    row(0, 0, 0, 0)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) IN ((?))", 0, 0),
+                    row(0, 0, 0, 0),
+                    row(0, 0, 1, 1),
+                    row(0, 0, 1, 0)
+            );
+
+            // preserve pre-6875 behavior (even though the query result is technically incorrect)
+            assertEmpty(execute("SELECT * FROM %s WHERE a = ? AND (b, c) > (?, ?)", 0, 1, 0));
+        }
+    }
+
+    @Test
+    public void testMultipleClusteringWithIndex() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int, b int, c int, d int, e int, PRIMARY KEY (a, b, c, d))");
+        createIndex("CREATE INDEX ON %s (b)");
+        createIndex("CREATE INDEX ON %s (e)");
+        execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 0, 0, 0, 0);
+        execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 0, 1, 0, 1);
+        execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 0, 1, 1, 2);
+        execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 1, 0, 0, 0);
+        execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 1, 1, 0, 1);
+        execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 1, 1, 1, 2);
+        execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 2, 0, 0, 0);
+        assertRows(execute("SELECT * FROM %s WHERE (b) = (?)", 1),
+                   row(0, 1, 0, 0, 0),
+                   row(0, 1, 1, 0, 1),
+                   row(0, 1, 1, 1, 2));
+
+        assertInvalidMessage("Cannot execute this query as it might involve data filtering",
+                             "SELECT * FROM %s WHERE (b, c) = (?, ?)", 1, 1);
+        assertRows(execute("SELECT * FROM %s WHERE (b, c) = (?, ?) ALLOW FILTERING", 1, 1),
+                   row(0, 1, 1, 0, 1),
+                   row(0, 1, 1, 1, 2));
+
+        assertInvalidMessage("Cannot execute this query as it might involve data filtering",
+                             "SELECT * FROM %s WHERE (b, c) = (?, ?) AND e = ?", 1, 1, 2);
+        assertRows(execute("SELECT * FROM %s WHERE (b, c) = (?, ?) AND e = ? ALLOW FILTERING", 1, 1, 2),
+                   row(0, 1, 1, 1, 2));
+
+        assertInvalidMessage("Cannot execute this query as it might involve data filtering",
+                             "SELECT * FROM %s WHERE (b) IN ((?)) AND e = ?", 1, 2);
+        assertRows(execute("SELECT * FROM %s WHERE (b) IN ((?)) AND e = ? ALLOW FILTERING", 1, 2),
+                   row(0, 1, 1, 1, 2));
+
+        assertInvalidMessage("Cannot execute this query as it might involve data filtering",
+                             "SELECT * FROM %s WHERE (b) IN ((?), (?)) AND e = ?", 0, 1, 2);
+        assertRows(execute("SELECT * FROM %s WHERE (b) IN ((?), (?)) AND e = ? ALLOW FILTERING", 0, 1, 2),
+                   row(0, 0, 1, 1, 2),
+                   row(0, 1, 1, 1, 2));
+
+        assertInvalidMessage("Cannot execute this query as it might involve data filtering",
+                             "SELECT * FROM %s WHERE (b, c) IN ((?, ?)) AND e = ?", 0, 1, 2);
+        assertRows(execute("SELECT * FROM %s WHERE (b, c) IN ((?, ?)) AND e = ? ALLOW FILTERING", 0, 1, 2),
+                   row(0, 0, 1, 1, 2));
+
+        assertInvalidMessage("Cannot execute this query as it might involve data filtering",
+                             "SELECT * FROM %s WHERE (b, c) IN ((?, ?), (?, ?)) AND e = ?", 0, 1, 1, 1, 2);
+        assertRows(execute("SELECT * FROM %s WHERE (b, c) IN ((?, ?), (?, ?)) AND e = ? ALLOW FILTERING", 0, 1, 1, 1, 2),
+                   row(0, 0, 1, 1, 2),
+                   row(0, 1, 1, 1, 2));
+
+        assertInvalidMessage("Cannot execute this query as it might involve data filtering",
+                             "SELECT * FROM %s WHERE (b) >= (?) AND e = ?", 1, 2);
+        assertRows(execute("SELECT * FROM %s WHERE (b) >= (?) AND e = ? ALLOW FILTERING", 1, 2),
+                   row(0, 1, 1, 1, 2));
+
+        assertInvalidMessage("Cannot execute this query as it might involve data filtering",
+                             "SELECT * FROM %s WHERE (b, c) >= (?, ?) AND e = ?", 1, 1, 2);
+        assertRows(execute("SELECT * FROM %s WHERE (b, c) >= (?, ?) AND e = ? ALLOW FILTERING", 1, 1, 2),
+                   row(0, 1, 1, 1, 2));
+    }
+
+    @Test
+    public void testMultiplePartitionKeyAndMultiClusteringWithIndex() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int, b int, c int, d int, e int, f int, PRIMARY KEY ((a, b), c, d, e))");
+        createIndex("CREATE INDEX ON %s (c)");
+        createIndex("CREATE INDEX ON %s (f)");
+
+        execute("INSERT INTO %s (a, b, c, d, e, f) VALUES (?, ?, ?, ?, ?, ?)", 0, 0, 0, 0, 0, 0);
+        execute("INSERT INTO %s (a, b, c, d, e, f) VALUES (?, ?, ?, ?, ?, ?)", 0, 0, 0, 1, 0, 1);
+        execute("INSERT INTO %s (a, b, c, d, e, f) VALUES (?, ?, ?, ?, ?, ?)", 0, 0, 0, 1, 1, 2);
+
+        execute("INSERT INTO %s (a, b, c, d, e, f) VALUES (?, ?, ?, ?, ?, ?)", 0, 0, 1, 0, 0, 3);
+        execute("INSERT INTO %s (a, b, c, d, e, f) VALUES (?, ?, ?, ?, ?, ?)", 0, 0, 1, 1, 0, 4);
+        execute("INSERT INTO %s (a, b, c, d, e, f) VALUES (?, ?, ?, ?, ?, ?)", 0, 0, 1, 1, 1, 5);
+
+        execute("INSERT INTO %s (a, b, c, d, e, f) VALUES (?, ?, ?, ?, ?, ?)", 0, 0, 2, 0, 0, 5);
+
+        assertInvalidMessage("Cannot execute this query as it might involve data filtering",
+                             "SELECT * FROM %s WHERE a = ? AND (c) = (?)");
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND (c) = (?) ALLOW FILTERING", 0, 1),
+                   row(0, 0, 1, 0, 0, 3),
+                   row(0, 0, 1, 1, 0, 4),
+                   row(0, 0, 1, 1, 1, 5));
+
+        assertInvalidMessage("Cannot execute this query as it might involve data filtering",
+                             "SELECT * FROM %s WHERE a = ? AND (c, d) = (?, ?)", 0, 1, 1);
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND (c, d) = (?, ?) ALLOW FILTERING", 0, 1, 1),
+                   row(0, 0, 1, 1, 0, 4),
+                   row(0, 0, 1, 1, 1, 5));
+
+        assertInvalidMessage("Partition key parts: b must be restricted as other parts are",
+                             "SELECT * FROM %s WHERE a = ? AND (c, d) IN ((?, ?)) ALLOW FILTERING", 0, 1, 1);
+
+        assertInvalidMessage("Partition key parts: b must be restricted as other parts are",
+                             "SELECT * FROM %s WHERE a = ? AND (c, d) >= (?, ?) ALLOW FILTERING", 0, 1, 1);
+
+        assertInvalidMessage("Cannot execute this query as it might involve data filtering",
+                             "SELECT * FROM %s WHERE a = ? AND (c) IN ((?)) AND f = ?", 0, 1, 5);
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND (c) IN ((?)) AND f = ? ALLOW FILTERING", 0, 1, 5),
+                   row(0, 0, 1, 1, 1, 5));
+
+        assertInvalidMessage("Cannot execute this query as it might involve data filtering",
+                             "SELECT * FROM %s WHERE a = ? AND (c) IN ((?), (?)) AND f = ?", 0, 1, 2, 5);
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND (c) IN ((?), (?)) AND f = ? ALLOW FILTERING", 0, 1, 2, 5),
+                   row(0, 0, 1, 1, 1, 5),
+                   row(0, 0, 2, 0, 0, 5));
+
+        assertInvalidMessage("Cannot execute this query as it might involve data filtering",
+                             "SELECT * FROM %s WHERE a = ? AND (c, d) IN ((?, ?)) AND f = ?", 0, 1, 0, 3);
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND (c, d) IN ((?, ?)) AND f = ? ALLOW FILTERING", 0, 1, 0, 3),
+                   row(0, 0, 1, 0, 0, 3));
+
+        assertInvalidMessage("Cannot execute this query as it might involve data filtering",
+                             "SELECT * FROM %s WHERE a = ? AND (c) >= (?) AND f = ?", 0, 1, 5);
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND (c) >= (?) AND f = ? ALLOW FILTERING", 0, 1, 5),
+                   row(0, 0, 1, 1, 1, 5),
+                   row(0, 0, 2, 0, 0, 5));
+
+        assertInvalidMessage("Cannot execute this query as it might involve data filtering",
+                             "SELECT * FROM %s WHERE a = ? AND (c, d) >= (?, ?) AND f = ?", 0, 1, 1, 5);
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND (c, d) >= (?, ?) AND f = ? ALLOW FILTERING", 0, 1, 1, 5),
+                   row(0, 0, 1, 1, 1, 5),
+                   row(0, 0, 2, 0, 0, 5));
+    }
+
+    @Test
+    public void testINWithDuplicateValue() throws Throwable
+    {
+        for (String compactOption : new String[] { "", " WITH COMPACT STORAGE" })
+        {
+            createTable("CREATE TABLE %s (k1 int, k2 int, v int, PRIMARY KEY (k1, k2))" + compactOption);
+            execute("INSERT INTO %s (k1,  k2, v) VALUES (?, ?, ?)", 1, 1, 1);
+
+            assertRows(execute("SELECT * FROM %s WHERE k1 IN (?, ?) AND (k2) IN ((?), (?))", 1, 1, 1, 2),
+                       row(1, 1, 1));
+            assertRows(execute("SELECT * FROM %s WHERE k1 = ? AND (k2) IN ((?), (?))", 1, 1, 1),
+                       row(1, 1, 1));
+        }
+    }
+
+    @Test
+    public void testWithUnsetValues() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int, i int, j int, s text, PRIMARY KEY(k,i,j))");
+        createIndex("CREATE INDEX s_index ON %s (s)");
+
+        assertInvalidMessage("Invalid unset value for tuple field number 0",
+                             "SELECT * from %s WHERE (i, j) = (?,?) ALLOW FILTERING", unset(), 1);
+        assertInvalidMessage("Invalid unset value for tuple field number 0",
+                             "SELECT * from %s WHERE (i, j) IN ((?,?)) ALLOW FILTERING", unset(), 1);
+        assertInvalidMessage("Invalid unset value for tuple field number 1",
+                             "SELECT * from %s WHERE (i, j) > (1,?) ALLOW FILTERING", unset());
+        assertInvalidMessage("Invalid unset value for tuple (i,j)",
+                             "SELECT * from %s WHERE (i, j) = ? ALLOW FILTERING", unset());
+        assertInvalidMessage("Invalid unset value for tuple (j)",
+                             "SELECT * from %s WHERE i = ? AND (j) > ? ALLOW FILTERING", 1, unset());
+        assertInvalidMessage("Invalid unset value for tuple (i,j)",
+                             "SELECT * from %s WHERE (i, j) IN (?, ?) ALLOW FILTERING", unset(), tuple(1, 1));
+        assertInvalidMessage("Invalid unset value for in(i,j)",
+                             "SELECT * from %s WHERE (i, j) IN ? ALLOW FILTERING", unset());
+    }
+
+    /**
+     * Check select on tuple relations, see CASSANDRA-8613
+     * migrated from cql_tests.py:TestCQL.simple_tuple_query_test()
+     */
+    @Test
+    public void testSimpleTupleQuery() throws Throwable
+    {
+        createTable("create table %s (a int, b int, c int, d int , e int, PRIMARY KEY (a, b, c, d, e))");
+
+        execute("INSERT INTO %s (a, b, c, d, e) VALUES (0, 2, 0, 0, 0)");
+        execute("INSERT INTO %s (a, b, c, d, e) VALUES (0, 1, 0, 0, 0)");
+        execute("INSERT INTO %s (a, b, c, d, e) VALUES (0, 0, 0, 0, 0)");
+        execute("INSERT INTO %s (a, b, c, d, e) VALUES (0, 0, 1, 1, 1)");
+        execute("INSERT INTO %s (a, b, c, d, e) VALUES (0, 0, 2, 2, 2)");
+        execute("INSERT INTO %s (a, b, c, d, e) VALUES (0, 0, 3, 3, 3)");
+        execute("INSERT INTO %s (a, b, c, d, e) VALUES (0, 0, 1, 1, 1)");
+
+        assertRows(execute("SELECT * FROM %s WHERE b=0 AND (c, d, e) > (1, 1, 1) ALLOW FILTERING"),
+                   row(0, 0, 2, 2, 2),
+                   row(0, 0, 3, 3, 3));
+    }
+ }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/01115f72/test/unit/org/apache/cassandra/cql3/validation/operations/SelectOrderByTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/SelectOrderByTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/SelectOrderByTest.java
new file mode 100644
index 0000000..e788095
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/SelectOrderByTest.java
@@ -0,0 +1,504 @@
+/*
+ * 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.validation.operations;
+
+import org.junit.Test;
+
+import org.apache.cassandra.cql3.CQLTester;
+
+import static org.junit.Assert.assertTrue;
+
+public class SelectOrderByTest extends CQLTester
+{
+    @Test
+    public void testNormalSelectionOrderSingleClustering() throws Throwable
+    {
+        for (String descOption : new String[]{"", " WITH CLUSTERING ORDER BY (b DESC)"})
+        {
+            createTable("CREATE TABLE %s (a int, b int, c int, PRIMARY KEY (a, b))" + descOption);
+            execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, 0, 0);
+            execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, 1, 1);
+            execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, 2, 2);
+
+            assertRows(execute("SELECT * FROM %s WHERE a=? ORDER BY b ASC", 0),
+                    row(0, 0, 0),
+                    row(0, 1, 1),
+                    row(0, 2, 2)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a=? ORDER BY b DESC", 0),
+                    row(0, 2, 2),
+                    row(0, 1, 1),
+                    row(0, 0, 0)
+            );
+
+            // order by the only column in the selection
+            assertRows(execute("SELECT b FROM %s WHERE a=? ORDER BY b ASC", 0),
+                    row(0), row(1), row(2));
+
+            assertRows(execute("SELECT b FROM %s WHERE a=? ORDER BY b DESC", 0),
+                    row(2), row(1), row(0));
+
+            // order by a column not in the selection
+            assertRows(execute("SELECT c FROM %s WHERE a=? ORDER BY b ASC", 0),
+                    row(0), row(1), row(2));
+
+            assertRows(execute("SELECT c FROM %s WHERE a=? ORDER BY b DESC", 0),
+                    row(2), row(1), row(0));
+        }
+    }
+
+    @Test
+    public void testFunctionSelectionOrderSingleClustering() throws Throwable
+    {
+        for (String descOption : new String[]{"", " WITH CLUSTERING ORDER BY (b DESC)"})
+        {
+            createTable("CREATE TABLE %s (a int, b int, c int, PRIMARY KEY (a, b))" + descOption);
+            execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, 0, 0);
+            execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, 1, 1);
+            execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, 2, 2);
+
+            // order by the only column in the selection
+            assertRows(execute("SELECT blobAsInt(intAsBlob(b)) FROM %s WHERE a=? ORDER BY b ASC", 0),
+                    row(0), row(1), row(2));
+
+            assertRows(execute("SELECT blobAsInt(intAsBlob(b)) FROM %s WHERE a=? ORDER BY b DESC", 0),
+                    row(2), row(1), row(0));
+
+            // order by a column not in the selection
+            assertRows(execute("SELECT blobAsInt(intAsBlob(c)) FROM %s WHERE a=? ORDER BY b ASC", 0),
+                    row(0), row(1), row(2));
+
+            assertRows(execute("SELECT blobAsInt(intAsBlob(c)) FROM %s WHERE a=? ORDER BY b DESC", 0),
+                    row(2), row(1), row(0));
+
+            assertInvalid("SELECT * FROM %s WHERE a=? ORDER BY c ASC", 0);
+            assertInvalid("SELECT * FROM %s WHERE a=? ORDER BY c DESC", 0);
+        }
+    }
+
+    @Test
+    public void testFieldSelectionOrderSingleClustering() throws Throwable
+    {
+        String type = createType("CREATE TYPE %s (a int)");
+
+        for (String descOption : new String[]{"", " WITH CLUSTERING ORDER BY (b DESC)"})
+        {
+            createTable("CREATE TABLE %s (a int, b int, c frozen<" + type + "   >, PRIMARY KEY (a, b))" + descOption);
+            execute("INSERT INTO %s (a, b, c) VALUES (?, ?, {a: ?})", 0, 0, 0);
+            execute("INSERT INTO %s (a, b, c) VALUES (?, ?, {a: ?})", 0, 1, 1);
+            execute("INSERT INTO %s (a, b, c) VALUES (?, ?, {a: ?})", 0, 2, 2);
+
+            // order by a column not in the selection
+            assertRows(execute("SELECT c.a FROM %s WHERE a=? ORDER BY b ASC", 0),
+                    row(0), row(1), row(2));
+
+            assertRows(execute("SELECT c.a FROM %s WHERE a=? ORDER BY b DESC", 0),
+                    row(2), row(1), row(0));
+
+            assertRows(execute("SELECT blobAsInt(intAsBlob(c.a)) FROM %s WHERE a=? ORDER BY b DESC", 0),
+                       row(2), row(1), row(0));
+            dropTable("DROP TABLE %s");
+        }
+    }
+
+    @Test
+    public void testNormalSelectionOrderMultipleClustering() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int, b int, c int, d int, PRIMARY KEY (a, b, c))");
+        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 0, 0);
+        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 1, 1);
+        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 2, 2);
+        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 0, 3);
+        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 1, 4);
+        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 2, 5);
+
+        assertRows(execute("SELECT * FROM %s WHERE a=? ORDER BY b ASC", 0),
+                row(0, 0, 0, 0),
+                row(0, 0, 1, 1),
+                row(0, 0, 2, 2),
+                row(0, 1, 0, 3),
+                row(0, 1, 1, 4),
+                row(0, 1, 2, 5)
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE a=? ORDER BY b DESC", 0),
+                row(0, 1, 2, 5),
+                row(0, 1, 1, 4),
+                row(0, 1, 0, 3),
+                row(0, 0, 2, 2),
+                row(0, 0, 1, 1),
+                row(0, 0, 0, 0)
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE a=? ORDER BY b DESC, c DESC", 0),
+                row(0, 1, 2, 5),
+                row(0, 1, 1, 4),
+                row(0, 1, 0, 3),
+                row(0, 0, 2, 2),
+                row(0, 0, 1, 1),
+                row(0, 0, 0, 0)
+        );
+
+        assertInvalid("SELECT * FROM %s WHERE a=? ORDER BY c ASC", 0);
+        assertInvalid("SELECT * FROM %s WHERE a=? ORDER BY c DESC", 0);
+        assertInvalid("SELECT * FROM %s WHERE a=? ORDER BY b ASC, c DESC", 0);
+        assertInvalid("SELECT * FROM %s WHERE a=? ORDER BY b DESC, c ASC", 0);
+        assertInvalid("SELECT * FROM %s WHERE a=? ORDER BY d ASC", 0);
+
+        // select and order by b
+        assertRows(execute("SELECT b FROM %s WHERE a=? ORDER BY b ASC", 0),
+                row(0), row(0), row(0), row(1), row(1), row(1));
+        assertRows(execute("SELECT b FROM %s WHERE a=? ORDER BY b DESC", 0),
+                row(1), row(1), row(1), row(0), row(0), row(0));
+
+        // select c, order by b
+        assertRows(execute("SELECT c FROM %s WHERE a=? ORDER BY b ASC", 0),
+                row(0), row(1), row(2), row(0), row(1), row(2));
+        assertRows(execute("SELECT c FROM %s WHERE a=? ORDER BY b DESC", 0),
+                row(2), row(1), row(0), row(2), row(1), row(0));
+
+        // select c, order by b, c
+        assertRows(execute("SELECT c FROM %s WHERE a=? ORDER BY b ASC, c ASC", 0),
+                row(0), row(1), row(2), row(0), row(1), row(2));
+        assertRows(execute("SELECT c FROM %s WHERE a=? ORDER BY b DESC, c DESC", 0),
+                row(2), row(1), row(0), row(2), row(1), row(0));
+
+        // select d, order by b, c
+        assertRows(execute("SELECT d FROM %s WHERE a=? ORDER BY b ASC, c ASC", 0),
+                row(0), row(1), row(2), row(3), row(4), row(5));
+        assertRows(execute("SELECT d FROM %s WHERE a=? ORDER BY b DESC, c DESC", 0),
+                row(5), row(4), row(3), row(2), row(1), row(0));
+    }
+
+    @Test
+    public void testFunctionSelectionOrderMultipleClustering() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int, b int, c int, d int, PRIMARY KEY (a, b, c))");
+        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 0, 0);
+        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 1, 1);
+        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 2, 2);
+        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 0, 3);
+        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 1, 4);
+        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 2, 5);
+
+        assertInvalid("SELECT blobAsInt(intAsBlob(b)) FROM %s WHERE a=? ORDER BY c ASC", 0);
+        assertInvalid("SELECT blobAsInt(intAsBlob(b)) FROM %s WHERE a=? ORDER BY c DESC", 0);
+        assertInvalid("SELECT blobAsInt(intAsBlob(b)) FROM %s WHERE a=? ORDER BY b ASC, c DESC", 0);
+        assertInvalid("SELECT blobAsInt(intAsBlob(b)) FROM %s WHERE a=? ORDER BY b DESC, c ASC", 0);
+        assertInvalid("SELECT blobAsInt(intAsBlob(b)) FROM %s WHERE a=? ORDER BY d ASC", 0);
+
+        // select and order by b
+        assertRows(execute("SELECT blobAsInt(intAsBlob(b)) FROM %s WHERE a=? ORDER BY b ASC", 0),
+                   row(0), row(0), row(0), row(1), row(1), row(1));
+        assertRows(execute("SELECT blobAsInt(intAsBlob(b)) FROM %s WHERE a=? ORDER BY b DESC", 0),
+                   row(1), row(1), row(1), row(0), row(0), row(0));
+
+        assertRows(execute("SELECT b, blobAsInt(intAsBlob(b)) FROM %s WHERE a=? ORDER BY b ASC", 0),
+                row(0, 0), row(0, 0), row(0, 0), row(1, 1), row(1, 1), row(1, 1));
+        assertRows(execute("SELECT b, blobAsInt(intAsBlob(b)) FROM %s WHERE a=? ORDER BY b DESC", 0),
+                row(1, 1), row(1, 1), row(1, 1), row(0, 0), row(0, 0), row(0, 0));
+
+        // select c, order by b
+        assertRows(execute("SELECT blobAsInt(intAsBlob(c)) FROM %s WHERE a=? ORDER BY b ASC", 0),
+                row(0), row(1), row(2), row(0), row(1), row(2));
+        assertRows(execute("SELECT blobAsInt(intAsBlob(c)) FROM %s WHERE a=? ORDER BY b DESC", 0),
+                row(2), row(1), row(0), row(2), row(1), row(0));
+
+        // select c, order by b, c
+        assertRows(execute("SELECT blobAsInt(intAsBlob(c)) FROM %s WHERE a=? ORDER BY b ASC, c ASC", 0),
+                row(0), row(1), row(2), row(0), row(1), row(2));
+        assertRows(execute("SELECT blobAsInt(intAsBlob(c)) FROM %s WHERE a=? ORDER BY b DESC, c DESC", 0),
+                row(2), row(1), row(0), row(2), row(1), row(0));
+
+        // select d, order by b, c
+        assertRows(execute("SELECT blobAsInt(intAsBlob(d)) FROM %s WHERE a=? ORDER BY b ASC, c ASC", 0),
+                row(0), row(1), row(2), row(3), row(4), row(5));
+        assertRows(execute("SELECT blobAsInt(intAsBlob(d)) FROM %s WHERE a=? ORDER BY b DESC, c DESC", 0),
+                row(5), row(4), row(3), row(2), row(1), row(0));
+
+    }
+
+    /**
+     * Check ORDER BY support in SELECT statement
+     * migrated from cql_tests.py:TestCQL.order_by_test()
+     */
+    @Test
+    public void testSimpleOrderBy() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int, c int, v int, PRIMARY KEY (k, c)) WITH COMPACT STORAGE");
+
+        for (int i = 0; i < 10; i++)
+            execute("INSERT INTO %s (k, c, v) VALUES (0, ?, ?)", i, i);
+
+        assertRows(execute("SELECT v FROM %s WHERE k = 0 ORDER BY c DESC"),
+                   row(9), row(8), row(7), row(6), row(5), row(4), row(3), row(2), row(1), row(0));
+
+        createTable("CREATE TABLE %s (k int, c1 int, c2 int, v int, PRIMARY KEY (k, c1, c2)) WITH COMPACT STORAGE");
+
+        for (int i = 0; i < 4; i++)
+            for (int j = 0; j < 2; j++)
+                execute("INSERT INTO %s (k, c1, c2, v) VALUES (0, ?, ?, ?)", i, j, i * 2 + j);
+
+        assertInvalid("SELECT v FROM %s WHERE k = 0 ORDER BY c DESC");
+        assertInvalid("SELECT v FROM %s WHERE k = 0 ORDER BY c2 DESC");
+        assertInvalid("SELECT v FROM %s WHERE k = 0 ORDER BY k DESC");
+
+        assertRows(execute("SELECT v FROM %s WHERE k = 0 ORDER BY c1 DESC"),
+                   row(7), row(6), row(5), row(4), row(3), row(2), row(1), row(0));
+
+        assertRows(execute("SELECT v FROM %s WHERE k = 0 ORDER BY c1"),
+                   row(0), row(1), row(2), row(3), row(4), row(5), row(6), row(7));
+    }
+
+    /**
+     * More ORDER BY checks (#4160)
+     * migrated from cql_tests.py:TestCQL.more_order_by_test()
+     */
+    @Test
+    public void testMoreOrderBy() throws Throwable
+    {
+        createTable("CREATE TABLE %s (row text, number int, string text, PRIMARY KEY(row, number)) WITH COMPACT STORAGE ");
+
+        execute("INSERT INTO %s (row, number, string) VALUES ('row', 1, 'one')");
+        execute("INSERT INTO %s (row, number, string) VALUES ('row', 2, 'two')");
+        execute("INSERT INTO %s (row, number, string) VALUES ('row', 3, 'three')");
+        execute("INSERT INTO %s (row, number, string) VALUES ('row', 4, 'four')");
+
+        assertRows(execute("SELECT number FROM %s WHERE row='row' AND number < 3 ORDER BY number ASC"),
+                   row(1), row(2));
+
+        assertRows(execute("SELECT number FROM %s WHERE row='row' AND number >= 3 ORDER BY number ASC"),
+                   row(3), row(4));
+
+        assertRows(execute("SELECT number FROM %s WHERE row='row' AND number < 3 ORDER BY number DESC"),
+                   row(2), row(1));
+
+        assertRows(execute("SELECT number FROM %s WHERE row='row' AND number >= 3 ORDER BY number DESC"),
+                   row(4), row(3));
+
+        assertRows(execute("SELECT number FROM %s WHERE row='row' AND number > 3 ORDER BY number DESC"),
+                   row(4));
+
+        assertRows(execute("SELECT number FROM %s WHERE row='row' AND number <= 3 ORDER BY number DESC"),
+                   row(3), row(2), row(1));
+    }
+
+    /**
+     * Check we don't allow order by on row key (#4246)
+     * migrated from cql_tests.py:TestCQL.order_by_validation_test()
+     */
+    @Test
+    public void testInvalidOrderBy() throws Throwable
+    {
+        createTable("CREATE TABLE %s( k1 int, k2 int, v int, PRIMARY KEY (k1, k2))");
+
+        execute("INSERT INTO %s (k1, k2, v) VALUES (?, ?, ?)", 0, 0, 0);
+        execute("INSERT INTO %s (k1, k2, v) VALUES (?, ?, ?)", 1, 1, 1);
+        execute("INSERT INTO %s (k1, k2, v) VALUES (?, ?, ?)", 2, 2, 2);
+
+        assertInvalid("SELECT * FROM %s ORDER BY k2");
+    }
+
+    /**
+     * Check that order-by works with IN (#4327)
+     * migrated from cql_tests.py:TestCQL.order_by_with_in_test()
+     */
+    @Test
+    public void testOrderByForInClause() throws Throwable
+    {
+        createTable("CREATE TABLE %s (my_id varchar, col1 int, value varchar, PRIMARY KEY (my_id, col1))");
+
+        execute("INSERT INTO %s (my_id, col1, value) VALUES ( 'key1', 1, 'a')");
+        execute("INSERT INTO %s (my_id, col1, value) VALUES ( 'key2', 3, 'c')");
+        execute("INSERT INTO %s (my_id, col1, value) VALUES ( 'key3', 2, 'b')");
+        execute("INSERT INTO %s (my_id, col1, value) VALUES ( 'key4', 4, 'd')");
+
+        assertRows(execute("SELECT col1 FROM %s WHERE my_id in('key1', 'key2', 'key3') ORDER BY col1"),
+                   row(1), row(2), row(3));
+
+        assertRows(execute("SELECT col1, my_id FROM %s WHERE my_id in('key1', 'key2', 'key3') ORDER BY col1"),
+                   row(1, "key1"), row(2, "key3"), row(3, "key2"));
+
+        assertRows(execute("SELECT my_id, col1 FROM %s WHERE my_id in('key1', 'key2', 'key3') ORDER BY col1"),
+                   row("key1", 1), row("key3", 2), row("key2", 3));
+    }
+
+    /**
+     * Test reversed comparators
+     * migrated from cql_tests.py:TestCQL.reversed_comparator_test()
+     */
+    @Test
+    public void testReversedComparator() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int, c int, v int, PRIMARY KEY (k, c)) WITH CLUSTERING ORDER BY (c DESC);");
+
+        for(int i =0; i < 10; i++)
+            execute("INSERT INTO %s (k, c, v) VALUES (0, ?, ?)", i, i);
+
+        assertRows(execute("SELECT c, v FROM %s WHERE k = 0 ORDER BY c ASC"),
+                   row(0, 0), row(1, 1), row(2, 2), row(3, 3), row(4, 4),
+                   row(5, 5), row(6, 6), row(7, 7), row(8, 8), row(9, 9));
+
+        assertRows(execute("SELECT c, v FROM %s WHERE k = 0 ORDER BY c DESC"),
+                   row(9, 9), row(8, 8), row(7, 7), row(6, 6), row(5, 5),
+                   row(4, 4), row(3, 3), row(2, 2), row(1, 1), row(0, 0));
+
+        createTable("CREATE TABLE %s (k int, c1 int, c2 int, v text, PRIMARY KEY (k, c1, c2)) WITH CLUSTERING ORDER BY (c1 ASC, c2 DESC)");
+
+        for(int i = 0; i < 10; i++)
+            for(int j = 0; j < 10; j++)
+                execute("INSERT INTO %s (k, c1, c2, v) VALUES (0, ?, ?, ?)", i, j, String.format("%d%d", i, j));
+
+        assertInvalid("SELECT c1, c2, v FROM %s WHERE k = 0 ORDER BY c1 ASC, c2 ASC");
+        assertInvalid("SELECT c1, c2, v FROM %s WHERE k = 0 ORDER BY c1 DESC, c2 DESC");
+
+        Object[][] expectedRows = new Object[100][];
+        for(int i = 0; i < 10; i++)
+            for(int j = 9; j >= 0; j--)
+                expectedRows[i * 10 + (9 - j)] = row(i, j, String.format("%d%d", i, j));
+
+        assertRows(execute("SELECT c1, c2, v FROM %s WHERE k = 0 ORDER BY c1 ASC"),
+                   expectedRows);
+
+        assertRows(execute("SELECT c1, c2, v FROM %s WHERE k = 0 ORDER BY c1 ASC, c2 DESC"),
+                   expectedRows);
+
+        for(int i = 9; i >= 0; i--)
+            for(int j = 0; j < 10; j++)
+                expectedRows[(9 - i) * 10 + j] = row(i, j, String.format("%d%d", i, j));
+
+        assertRows(execute("SELECT c1, c2, v FROM %s WHERE k = 0 ORDER BY c1 DESC, c2 ASC"),
+                   expectedRows);
+
+        assertInvalid("SELECT c1, c2, v FROM %s WHERE k = 0 ORDER BY c2 DESC, c1 ASC");
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.multiordering_test()
+     */
+    @Test
+    public void testMultiordering() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k text, c1 int, c2 int, PRIMARY KEY (k, c1, c2) ) WITH CLUSTERING ORDER BY (c1 ASC, c2 DESC)");
+
+        for (int i = 0; i < 2; i++)
+            for (int j = 0; j < 2; j++)
+                execute("INSERT INTO %s (k, c1, c2) VALUES ('foo', ?, ?)", i, j);
+
+        assertRows(execute("SELECT c1, c2 FROM %s WHERE k = 'foo'"),
+                   row(0, 1), row(0, 0), row(1, 1), row(1, 0));
+
+        assertRows(execute("SELECT c1, c2 FROM %s WHERE k = 'foo' ORDER BY c1 ASC, c2 DESC"),
+                   row(0, 1), row(0, 0), row(1, 1), row(1, 0));
+
+        assertRows(execute("SELECT c1, c2 FROM %s WHERE k = 'foo' ORDER BY c1 DESC, c2 ASC"),
+                   row(1, 0), row(1, 1), row(0, 0), row(0, 1));
+
+        assertInvalid("SELECT c1, c2 FROM %s WHERE k = 'foo' ORDER BY c2 DESC");
+        assertInvalid("SELECT c1, c2 FROM %s WHERE k = 'foo' ORDER BY c2 ASC");
+        assertInvalid("SELECT c1, c2 FROM %s WHERE k = 'foo' ORDER BY c1 ASC, c2 ASC");
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.in_with_desc_order_test()
+     */
+    @Test
+    public void testSelectInStatementWithDesc() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int, c1 int, c2 int, PRIMARY KEY (k, c1, c2))");
+        execute("INSERT INTO %s(k, c1, c2) VALUES (0, 0, 0)");
+        execute("INSERT INTO %s(k, c1, c2) VALUES (0, 0, 1)");
+        execute("INSERT INTO %s(k, c1, c2) VALUES (0, 0, 2)");
+
+        assertRows(execute("SELECT * FROM %s WHERE k=0 AND c1 = 0 AND c2 IN (2, 0) ORDER BY c1 DESC"),
+                   row(0, 0, 2),
+                   row(0, 0, 0));
+    }
+
+    /**
+     * Test that columns don't need to be selected for ORDER BY when there is a IN (#4911),
+     * migrated from cql_tests.py:TestCQL.in_order_by_without_selecting_test()
+     */
+    @Test
+    public void testInOrderByWithoutSelecting() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int, c1 int, c2 int, v int, PRIMARY KEY (k, c1, c2))");
+
+        execute("INSERT INTO %s (k, c1, c2, v) VALUES (0, 0, 0, 0)");
+        execute("INSERT INTO %s (k, c1, c2, v) VALUES (0, 0, 1, 1)");
+        execute("INSERT INTO %s (k, c1, c2, v) VALUES (0, 0, 2, 2)");
+        execute("INSERT INTO %s (k, c1, c2, v) VALUES (1, 1, 0, 3)");
+        execute("INSERT INTO %s (k, c1, c2, v) VALUES (1, 1, 1, 4)");
+        execute("INSERT INTO %s (k, c1, c2, v) VALUES (1, 1, 2, 5)");
+
+        assertRows(execute("SELECT * FROM %s WHERE k=0 AND c1 = 0 AND c2 IN (2, 0)"),
+                   row(0, 0, 0, 0),
+                   row(0, 0, 2, 2));
+        assertRows(execute("SELECT * FROM %s WHERE k=0 AND c1 = 0 AND c2 IN (2, 0) ORDER BY c1 ASC, c2 ASC"),
+                   row(0, 0, 0, 0),
+                   row(0, 0, 2, 2));
+
+        // check that we don 't need to select the column on which we order
+        assertRows(execute("SELECT v FROM %s WHERE k=0 AND c1 = 0 AND c2 IN (2, 0)"),
+                   row(0),
+                   row(2));
+        assertRows(execute("SELECT v FROM %s WHERE k=0 AND c1 = 0 AND c2 IN (2, 0) ORDER BY c1 ASC"),
+                   row(0),
+                   row(2));
+        assertRows(execute("SELECT v FROM %s WHERE k=0 AND c1 = 0 AND c2 IN (2, 0) ORDER BY c1 DESC"),
+                   row(2),
+                   row(0));
+
+        assertRows(execute("SELECT v FROM %s WHERE k IN (1, 0)"),
+                   row(0),
+                   row(1),
+                   row(2),
+                   row(3),
+                   row(4),
+                   row(5));
+
+        assertRows(execute("SELECT v FROM %s WHERE k IN (1, 0) ORDER BY c1 ASC"),
+                   row(0),
+                   row(1),
+                   row(2),
+                   row(3),
+                   row(4),
+                   row(5));
+
+        // we should also be able to use functions in the select clause (additional test for CASSANDRA - 8286)
+        Object[][] results = getRows(execute("SELECT writetime(v) FROM %s WHERE k IN (1, 0) ORDER BY c1 ASC"));
+
+        // since we don 't know the write times, just assert that the order matches the order we expect
+        assertTrue(isFirstIntSorted(results));
+    }
+
+    private boolean isFirstIntSorted(Object[][] rows)
+    {
+        for (int i = 1; i < rows.length; i++)
+        {
+            Long prev = (Long)rows[i-1][0];
+            Long curr = (Long)rows[i][0];
+
+            if (prev > curr)
+                return false;
+        }
+
+        return true;
+    }
+}


[08/32] cassandra git commit: Migrate CQL tests from dtest to unit tests

Posted by jm...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/f797bfa4/test/unit/org/apache/cassandra/cql3/CreateIndexStatementTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/CreateIndexStatementTest.java b/test/unit/org/apache/cassandra/cql3/CreateIndexStatementTest.java
deleted file mode 100644
index 18e1be5..0000000
--- a/test/unit/org/apache/cassandra/cql3/CreateIndexStatementTest.java
+++ /dev/null
@@ -1,101 +0,0 @@
-/*
- * 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;
-
-import java.util.Locale;
-
-import org.apache.commons.lang.StringUtils;
-
-import org.junit.Test;
-
-public class CreateIndexStatementTest extends CQLTester
-{
-    @Test
-    public void testCreateAndDropIndex() throws Throwable
-    {
-        testCreateAndDropIndex("test", false);
-        testCreateAndDropIndex("test2", true);
-    }
-
-    @Test
-    public void testCreateAndDropIndexWithQuotedIdentifier() throws Throwable
-    {
-        testCreateAndDropIndex("\"quoted_ident\"", false);
-        testCreateAndDropIndex("\"quoted_ident2\"", true);
-    }
-
-    @Test
-    public void testCreateAndDropIndexWithCamelCaseIdentifier() throws Throwable
-    {
-        testCreateAndDropIndex("CamelCase", false);
-        testCreateAndDropIndex("CamelCase2", true);
-    }
-
-    /**
-     * Test creating and dropping an index with the specified name.
-     *
-     * @param indexName the index name
-     * @param addKeyspaceOnDrop add the keyspace name in the drop statement
-     * @throws Throwable if an error occurs
-     */
-    private void testCreateAndDropIndex(String indexName, boolean addKeyspaceOnDrop) throws Throwable
-    {
-        execute("USE system");
-        assertInvalidMessage("Index '" + removeQuotes(indexName.toLowerCase(Locale.US)) + "' could not be found", "DROP INDEX " + indexName + ";");
-
-        createTable("CREATE TABLE %s (a int primary key, b int);");
-        createIndex("CREATE INDEX " + indexName + " ON %s(b);");
-        createIndex("CREATE INDEX IF NOT EXISTS " + indexName + " ON %s(b);");
-
-        assertInvalidMessage("Index already exists", "CREATE INDEX " + indexName + " ON %s(b)");
-
-        execute("INSERT INTO %s (a, b) values (?, ?);", 0, 0);
-        execute("INSERT INTO %s (a, b) values (?, ?);", 1, 1);
-        execute("INSERT INTO %s (a, b) values (?, ?);", 2, 2);
-        execute("INSERT INTO %s (a, b) values (?, ?);", 3, 1);
-
-        assertRows(execute("SELECT * FROM %s where b = ?", 1), row(1, 1), row(3, 1));
-        assertInvalidMessage("Index '" + removeQuotes(indexName.toLowerCase(Locale.US)) + "' could not be found in any of the tables of keyspace 'system'", "DROP INDEX " + indexName);
-
-        if (addKeyspaceOnDrop)
-        {
-            dropIndex("DROP INDEX " + KEYSPACE + "." + indexName);
-        }
-        else
-        {
-            execute("USE " + KEYSPACE);
-            dropIndex("DROP INDEX " + indexName);
-        }
-
-        assertInvalidMessage("No secondary indexes on the restricted columns support the provided operators",
-                             "SELECT * FROM %s where b = ?", 1);
-        dropIndex("DROP INDEX IF EXISTS " + indexName);
-        assertInvalidMessage("Index '" + removeQuotes(indexName.toLowerCase(Locale.US)) + "' could not be found", "DROP INDEX " + indexName);
-    }
-
-    /**
-     * Removes the quotes from the specified index name.
-     *
-     * @param indexName the index name from which the quotes must be removed.
-     * @return the unquoted index name.
-     */
-    private static String removeQuotes(String indexName)
-    {
-        return StringUtils.remove(indexName, '\"');
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f797bfa4/test/unit/org/apache/cassandra/cql3/CreateTableTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/CreateTableTest.java b/test/unit/org/apache/cassandra/cql3/CreateTableTest.java
deleted file mode 100644
index 14d2c2b..0000000
--- a/test/unit/org/apache/cassandra/cql3/CreateTableTest.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/*
- * 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;
-
-import org.junit.Test;
-
-import static junit.framework.Assert.assertFalse;
-
-public class CreateTableTest extends CQLTester
-{
-    @Test
-    public void testCQL3PartitionKeyOnlyTable()
-    {
-        createTable("CREATE TABLE %s (id text PRIMARY KEY);");
-        assertFalse(currentTableMetadata().isThriftCompatible());
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f797bfa4/test/unit/org/apache/cassandra/cql3/CreateTriggerStatementTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/CreateTriggerStatementTest.java b/test/unit/org/apache/cassandra/cql3/CreateTriggerStatementTest.java
deleted file mode 100644
index 6557c16..0000000
--- a/test/unit/org/apache/cassandra/cql3/CreateTriggerStatementTest.java
+++ /dev/null
@@ -1,121 +0,0 @@
-/*
- * 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;
-
-import java.nio.ByteBuffer;
-import java.util.Collection;
-import java.util.Collections;
-
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.Schema;
-import org.apache.cassandra.config.TriggerDefinition;
-import org.apache.cassandra.db.ColumnFamily;
-import org.apache.cassandra.db.Mutation;
-import org.apache.cassandra.triggers.ITrigger;
-import org.junit.Test;
-
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-public class CreateTriggerStatementTest extends CQLTester
-{
-    @Test
-    public void testCreateTrigger() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a int, b int, c int, PRIMARY KEY (a))");
-        execute("CREATE TRIGGER trigger_1 ON %s USING '" + TestTrigger.class.getName() + "'");
-        assertTriggerExists("trigger_1", TestTrigger.class);
-        execute("CREATE TRIGGER trigger_2 ON %s USING '" + TestTrigger.class.getName() + "'");
-        assertTriggerExists("trigger_2", TestTrigger.class);
-        assertInvalid("CREATE TRIGGER trigger_1 ON %s USING '" + TestTrigger.class.getName() + "'");
-        execute("CREATE TRIGGER \"Trigger 3\" ON %s USING '" + TestTrigger.class.getName() + "'");
-        assertTriggerExists("Trigger 3", TestTrigger.class);
-    }
-
-    @Test
-    public void testCreateTriggerIfNotExists() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a int, b int, c int, PRIMARY KEY (a, b))");
-
-        execute("CREATE TRIGGER IF NOT EXISTS trigger_1 ON %s USING '" + TestTrigger.class.getName() + "'");
-        assertTriggerExists("trigger_1", TestTrigger.class);
-
-        execute("CREATE TRIGGER IF NOT EXISTS trigger_1 ON %s USING '" + TestTrigger.class.getName() + "'");
-        assertTriggerExists("trigger_1", TestTrigger.class);
-    }
-
-    @Test
-    public void testDropTrigger() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a int, b int, c int, PRIMARY KEY (a))");
-
-        execute("CREATE TRIGGER trigger_1 ON %s USING '" + TestTrigger.class.getName() + "'");
-        assertTriggerExists("trigger_1", TestTrigger.class);
-
-        execute("DROP TRIGGER trigger_1 ON %s");
-        assertTriggerDoesNotExists("trigger_1", TestTrigger.class);
-
-        execute("CREATE TRIGGER trigger_1 ON %s USING '" + TestTrigger.class.getName() + "'");
-        assertTriggerExists("trigger_1", TestTrigger.class);
-
-        assertInvalid("DROP TRIGGER trigger_2 ON %s");
-        
-        execute("CREATE TRIGGER \"Trigger 3\" ON %s USING '" + TestTrigger.class.getName() + "'");
-        assertTriggerExists("Trigger 3", TestTrigger.class);
-
-        execute("DROP TRIGGER \"Trigger 3\" ON %s");
-        assertTriggerDoesNotExists("Trigger 3", TestTrigger.class);
-    }
-
-    @Test
-    public void testDropTriggerIfExists() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a int, b int, c int, PRIMARY KEY (a))");
-
-        execute("DROP TRIGGER IF EXISTS trigger_1 ON %s");
-        assertTriggerDoesNotExists("trigger_1", TestTrigger.class);
-
-        execute("CREATE TRIGGER trigger_1 ON %s USING '" + TestTrigger.class.getName() + "'");
-        assertTriggerExists("trigger_1", TestTrigger.class);
-
-        execute("DROP TRIGGER IF EXISTS trigger_1 ON %s");
-        assertTriggerDoesNotExists("trigger_1", TestTrigger.class);
-    }
-
-    private void assertTriggerExists(String name, Class<?> clazz)
-    {
-        CFMetaData cfm = Schema.instance.getCFMetaData(keyspace(), currentTable()).copy();
-        assertTrue("the trigger does not exist", cfm.containsTriggerDefinition(TriggerDefinition.create(name,
-                clazz.getName())));
-    }
-
-    private void assertTriggerDoesNotExists(String name, Class<?> clazz)
-    {
-        CFMetaData cfm = Schema.instance.getCFMetaData(keyspace(), currentTable()).copy();
-        assertFalse("the trigger exists", cfm.containsTriggerDefinition(TriggerDefinition.create(name,
-                clazz.getName())));
-    }
-
-    public static class TestTrigger implements ITrigger
-    {
-        public Collection<Mutation> augment(ByteBuffer key, ColumnFamily update)
-        {
-            return Collections.emptyList();
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f797bfa4/test/unit/org/apache/cassandra/cql3/FrozenCollectionsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/FrozenCollectionsTest.java b/test/unit/org/apache/cassandra/cql3/FrozenCollectionsTest.java
deleted file mode 100644
index bf347f0..0000000
--- a/test/unit/org/apache/cassandra/cql3/FrozenCollectionsTest.java
+++ /dev/null
@@ -1,1101 +0,0 @@
-/*
- * 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;
-
-import org.apache.cassandra.db.marshal.*;
-import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.exceptions.InvalidRequestException;
-import org.apache.cassandra.exceptions.SyntaxException;
-import org.apache.commons.lang3.StringUtils;
-import org.junit.Assert;
-import org.junit.Test;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-
-import static org.junit.Assert.assertEquals;
-
-public class FrozenCollectionsTest extends CQLTester
-{
-    @Test
-    public void testPartitionKeyUsage() throws Throwable
-    {
-        createTable("CREATE TABLE %s (k frozen<set<int>> PRIMARY KEY, v int)");
-
-        execute("INSERT INTO %s (k, v) VALUES (?, ?)", set(), 1);
-        execute("INSERT INTO %s (k, v) VALUES (?, ?)", set(1, 2, 3), 1);
-        execute("INSERT INTO %s (k, v) VALUES (?, ?)", set(4, 5, 6), 0);
-        execute("INSERT INTO %s (k, v) VALUES (?, ?)", set(7, 8, 9), 0);
-
-        // overwrite with an update
-        execute("UPDATE %s SET v=? WHERE k=?", 0, set());
-        execute("UPDATE %s SET v=? WHERE k=?", 0, set(1, 2, 3));
-
-        assertRows(execute("SELECT * FROM %s"),
-            row(set(), 0),
-            row(set(1, 2, 3), 0),
-            row(set(4, 5, 6), 0),
-            row(set(7, 8, 9), 0)
-        );
-
-        assertRows(execute("SELECT k FROM %s"),
-            row(set()),
-            row(set(1, 2, 3)),
-            row(set(4, 5, 6)),
-            row(set(7, 8, 9))
-        );
-
-        assertRows(execute("SELECT * FROM %s LIMIT 2"),
-                row(set(), 0),
-                row(set(1, 2, 3), 0)
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE k=?", set(4, 5, 6)),
-            row(set(4, 5, 6), 0)
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE k=?", set()),
-                row(set(), 0)
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE k IN ?", list(set(4, 5, 6), set())),
-                row(set(4, 5, 6), 0),
-                row(set(), 0)
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE token(k) >= token(?)", set(4, 5, 6)),
-                row(set(4, 5, 6), 0),
-                row(set(7, 8, 9), 0)
-        );
-
-        assertInvalid("INSERT INTO %s (k, v) VALUES (null, 0)");
-
-        execute("DELETE FROM %s WHERE k=?", set());
-        execute("DELETE FROM %s WHERE k=?", set(4, 5, 6));
-        assertRows(execute("SELECT * FROM %s"),
-            row(set(1, 2, 3), 0),
-            row(set(7, 8, 9), 0)
-        );
-    }
-
-    @Test
-    public void testNestedPartitionKeyUsage() throws Throwable
-    {
-        createTable("CREATE TABLE %s (k frozen<map<set<int>, list<int>>> PRIMARY KEY, v int)");
-
-        execute("INSERT INTO %s (k, v) VALUES (?, ?)", map(), 1);
-        execute("INSERT INTO %s (k, v) VALUES (?, ?)", map(set(), list(1, 2, 3)), 0);
-        execute("INSERT INTO %s (k, v) VALUES (?, ?)", map(set(1, 2, 3), list(1, 2, 3)), 1);
-        execute("INSERT INTO %s (k, v) VALUES (?, ?)", map(set(4, 5, 6), list(1, 2, 3)), 0);
-        execute("INSERT INTO %s (k, v) VALUES (?, ?)", map(set(7, 8, 9), list(1, 2, 3)), 0);
-
-        // overwrite with an update
-        execute("UPDATE %s SET v=? WHERE k=?", 0, map());
-        execute("UPDATE %s SET v=? WHERE k=?", 0, map(set(1, 2, 3), list(1, 2, 3)));
-
-        assertRows(execute("SELECT * FROM %s"),
-            row(map(), 0),
-            row(map(set(), list(1, 2, 3)), 0),
-            row(map(set(1, 2, 3), list(1, 2, 3)), 0),
-            row(map(set(4, 5, 6), list(1, 2, 3)), 0),
-            row(map(set(7, 8, 9), list(1, 2, 3)), 0)
-        );
-
-        assertRows(execute("SELECT k FROM %s"),
-            row(map()),
-            row(map(set(), list(1, 2, 3))),
-            row(map(set(1, 2, 3), list(1, 2, 3))),
-            row(map(set(4, 5, 6), list(1, 2, 3))),
-            row(map(set(7, 8, 9), list(1, 2, 3)))
-        );
-
-        assertRows(execute("SELECT * FROM %s LIMIT 3"),
-            row(map(), 0),
-            row(map(set(), list(1, 2, 3)), 0),
-            row(map(set(1, 2, 3), list(1, 2, 3)), 0)
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE k=?", map(set(4, 5, 6), list(1, 2, 3))),
-            row(map(set(4, 5, 6), list(1, 2, 3)), 0)
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE k=?", map()),
-                row(map(), 0)
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE k=?", map(set(), list(1, 2, 3))),
-                row(map(set(), list(1, 2, 3)), 0)
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE k IN ?", list(map(set(4, 5, 6), list(1, 2, 3)), map(), map(set(), list(1, 2, 3)))),
-            row(map(set(4, 5, 6), list(1, 2, 3)), 0),
-            row(map(), 0),
-            row(map(set(), list(1, 2, 3)), 0)
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE token(k) >= token(?)", map(set(4, 5, 6), list(1, 2, 3))),
-            row(map(set(4, 5, 6), list(1, 2, 3)), 0),
-            row(map(set(7, 8, 9), list(1, 2, 3)), 0)
-        );
-
-        execute("DELETE FROM %s WHERE k=?", map());
-        execute("DELETE FROM %s WHERE k=?", map(set(), list(1, 2, 3)));
-        execute("DELETE FROM %s WHERE k=?", map(set(4, 5, 6), list(1, 2, 3)));
-        assertRows(execute("SELECT * FROM %s"),
-            row(map(set(1, 2, 3), list(1, 2, 3)), 0),
-            row(map(set(7, 8, 9), list(1, 2, 3)), 0)
-        );
-
-    }
-
-    @Test
-    public void testClusteringKeyUsage() throws Throwable
-    {
-        for (String option : Arrays.asList("", " WITH COMPACT STORAGE"))
-        {
-            createTable("CREATE TABLE %s (a int, b frozen<set<int>>, c int, PRIMARY KEY (a, b))" + option);
-
-            execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, set(), 1);
-            execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, set(1, 2, 3), 1);
-            execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, set(4, 5, 6), 0);
-            execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, set(7, 8, 9), 0);
-
-            // overwrite with an update
-            execute("UPDATE %s SET c=? WHERE a=? AND b=?", 0, 0, set());
-            execute("UPDATE %s SET c=? WHERE a=? AND b=?", 0, 0, set(1, 2, 3));
-
-            assertRows(execute("SELECT * FROM %s"),
-                row(0, set(), 0),
-                row(0, set(1, 2, 3), 0),
-                row(0, set(4, 5, 6), 0),
-                row(0, set(7, 8, 9), 0)
-            );
-
-            assertRows(execute("SELECT b FROM %s"),
-                row(set()),
-                row(set(1, 2, 3)),
-                row(set(4, 5, 6)),
-                row(set(7, 8, 9))
-            );
-
-            assertRows(execute("SELECT * FROM %s LIMIT 2"),
-                row(0, set(), 0),
-                row(0, set(1, 2, 3), 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a=? AND b=?", 0, set(4, 5, 6)),
-                row(0, set(4, 5, 6), 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a=? AND b=?", 0, set()),
-                row(0, set(), 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a=? AND b IN ?", 0, list(set(4, 5, 6), set())),
-                row(0, set(), 0),
-                row(0, set(4, 5, 6), 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a=? AND b > ?", 0, set(4, 5, 6)),
-                row(0, set(7, 8, 9), 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a=? AND b >= ?", 0, set(4, 5, 6)),
-                row(0, set(4, 5, 6), 0),
-                row(0, set(7, 8, 9), 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a=? AND b < ?", 0, set(4, 5, 6)),
-                row(0, set(), 0),
-                row(0, set(1, 2, 3), 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a=? AND b <= ?", 0, set(4, 5, 6)),
-                row(0, set(), 0),
-                row(0, set(1, 2, 3), 0),
-                row(0, set(4, 5, 6), 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a=? AND b > ? AND b <= ?", 0, set(1, 2, 3), set(4, 5, 6)),
-                row(0, set(4, 5, 6), 0)
-            );
-
-            execute("DELETE FROM %s WHERE a=? AND b=?", 0, set());
-            execute("DELETE FROM %s WHERE a=? AND b=?", 0, set(4, 5, 6));
-            assertRows(execute("SELECT * FROM %s"),
-                row(0, set(1, 2, 3), 0),
-                row(0, set(7, 8, 9), 0)
-            );
-        }
-    }
-
-    @Test
-    public void testNestedClusteringKeyUsage() throws Throwable
-    {
-        for (String option : Arrays.asList("", " WITH COMPACT STORAGE"))
-        {
-            createTable("CREATE TABLE %s (a int, b frozen<map<set<int>, list<int>>>, c frozen<set<int>>, d int, PRIMARY KEY (a, b, c))" + option);
-
-            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, map(), set(), 0);
-            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, map(set(), list(1, 2, 3)), set(), 0);
-            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, map(set(1, 2, 3), list(1, 2, 3)), set(1, 2, 3), 0);
-            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, map(set(4, 5, 6), list(1, 2, 3)), set(1, 2, 3), 0);
-            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, map(set(7, 8, 9), list(1, 2, 3)), set(1, 2, 3), 0);
-
-            assertRows(execute("SELECT * FROM %s"),
-                row(0, map(), set(), 0),
-                row(0, map(set(), list(1, 2, 3)), set(), 0),
-                row(0, map(set(1, 2, 3), list(1, 2, 3)), set(1, 2, 3), 0),
-                row(0, map(set(4, 5, 6), list(1, 2, 3)), set(1, 2, 3), 0),
-                row(0, map(set(7, 8, 9), list(1, 2, 3)), set(1, 2, 3), 0)
-            );
-
-            assertRows(execute("SELECT b FROM %s"),
-                row(map()),
-                row(map(set(), list(1, 2, 3))),
-                row(map(set(1, 2, 3), list(1, 2, 3))),
-                row(map(set(4, 5, 6), list(1, 2, 3))),
-                row(map(set(7, 8, 9), list(1, 2, 3)))
-            );
-
-            assertRows(execute("SELECT c FROM %s"),
-                row(set()),
-                row(set()),
-                row(set(1, 2, 3)),
-                row(set(1, 2, 3)),
-                row(set(1, 2, 3))
-            );
-
-            assertRows(execute("SELECT * FROM %s LIMIT 3"),
-                row(0, map(), set(), 0),
-                row(0, map(set(), list(1, 2, 3)), set(), 0),
-                row(0, map(set(1, 2, 3), list(1, 2, 3)), set(1, 2, 3), 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a=0 ORDER BY b DESC LIMIT 4"),
-                row(0, map(set(7, 8, 9), list(1, 2, 3)), set(1, 2, 3), 0),
-                row(0, map(set(4, 5, 6), list(1, 2, 3)), set(1, 2, 3), 0),
-                row(0, map(set(1, 2, 3), list(1, 2, 3)), set(1, 2, 3), 0),
-                row(0, map(set(), list(1, 2, 3)), set(), 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a=? AND b=?", 0, map()),
-                row(0, map(), set(), 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a=? AND b=?", 0, map(set(), list(1, 2, 3))),
-                row(0, map(set(), list(1, 2, 3)), set(), 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a=? AND b=?", 0, map(set(1, 2, 3), list(1, 2, 3))),
-                row(0, map(set(1, 2, 3), list(1, 2, 3)), set(1, 2, 3), 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a=? AND b=? AND c=?", 0, map(set(), list(1, 2, 3)), set()),
-                    row(0, map(set(), list(1, 2, 3)), set(), 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a=? AND (b, c) IN ?", 0, list(tuple(map(set(4, 5, 6), list(1, 2, 3)), set(1, 2, 3)),
-                                                                                     tuple(map(), set()))),
-                row(0, map(), set(), 0),
-                row(0, map(set(4, 5, 6), list(1, 2, 3)), set(1, 2, 3), 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a=? AND b > ?", 0, map(set(4, 5, 6), list(1, 2, 3))),
-                row(0, map(set(7, 8, 9), list(1, 2, 3)), set(1, 2, 3), 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a=? AND b >= ?", 0, map(set(4, 5, 6), list(1, 2, 3))),
-                row(0, map(set(4, 5, 6), list(1, 2, 3)), set(1, 2, 3), 0),
-                row(0, map(set(7, 8, 9), list(1, 2, 3)), set(1, 2, 3), 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a=? AND b < ?", 0, map(set(4, 5, 6), list(1, 2, 3))),
-                row(0, map(), set(), 0),
-                row(0, map(set(), list(1, 2, 3)), set(), 0),
-                row(0, map(set(1, 2, 3), list(1, 2, 3)), set(1, 2, 3), 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a=? AND b <= ?", 0, map(set(4, 5, 6), list(1, 2, 3))),
-                row(0, map(), set(), 0),
-                row(0, map(set(), list(1, 2, 3)), set(), 0),
-                row(0, map(set(1, 2, 3), list(1, 2, 3)), set(1, 2, 3), 0),
-                row(0, map(set(4, 5, 6), list(1, 2, 3)), set(1, 2, 3), 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a=? AND b > ? AND b <= ?", 0, map(set(1, 2, 3), list(1, 2, 3)), map(set(4, 5, 6), list(1, 2, 3))),
-                row(0, map(set(4, 5, 6), list(1, 2, 3)), set(1, 2, 3), 0)
-            );
-
-            execute("DELETE FROM %s WHERE a=? AND b=? AND c=?", 0, map(), set());
-            assertEmpty(execute("SELECT * FROM %s WHERE a=? AND b=? AND c=?", 0, map(), set()));
-
-            execute("DELETE FROM %s WHERE a=? AND b=? AND c=?", 0, map(set(), list(1, 2, 3)), set());
-            assertEmpty(execute("SELECT * FROM %s WHERE a=? AND b=? AND c=?", 0, map(set(), list(1, 2, 3)), set()));
-
-            execute("DELETE FROM %s WHERE a=? AND b=? AND c=?", 0, map(set(4, 5, 6), list(1, 2, 3)), set(1, 2, 3));
-            assertEmpty(execute("SELECT * FROM %s WHERE a=? AND b=? AND c=?", 0, map(set(4, 5, 6), list(1, 2, 3)), set(1, 2, 3)));
-
-            assertRows(execute("SELECT * FROM %s"),
-                    row(0, map(set(1, 2, 3), list(1, 2, 3)), set(1, 2, 3), 0),
-                    row(0, map(set(7, 8, 9), list(1, 2, 3)), set(1, 2, 3), 0)
-            );
-        }
-    }
-
-    @Test
-    public void testNormalColumnUsage() throws Throwable
-    {
-        for (String option : Arrays.asList("", " WITH COMPACT STORAGE"))
-        {
-            createTable("CREATE TABLE %s (a int PRIMARY KEY, b frozen<map<set<int>, list<int>>>, c frozen<set<int>>)" + option);
-
-            execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, map(), set());
-            execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 1, map(set(), list(99999, 999999, 99999)), set());
-            execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 2, map(set(1, 2, 3), list(1, 2, 3)), set(1, 2, 3));
-            execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 3, map(set(4, 5, 6), list(1, 2, 3)), set(1, 2, 3));
-            execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 4, map(set(7, 8, 9), list(1, 2, 3)), set(1, 2, 3));
-
-            // overwrite with update
-            execute ("UPDATE %s SET b=? WHERE a=?", map(set(), list(1, 2, 3)), 1);
-
-            assertRows(execute("SELECT * FROM %s"),
-                row(0, map(), set()),
-                row(1, map(set(), list(1, 2, 3)), set()),
-                row(2, map(set(1, 2, 3), list(1, 2, 3)), set(1, 2, 3)),
-                row(3, map(set(4, 5, 6), list(1, 2, 3)), set(1, 2, 3)),
-                row(4, map(set(7, 8, 9), list(1, 2, 3)), set(1, 2, 3))
-            );
-
-            assertRows(execute("SELECT b FROM %s"),
-                row(map()),
-                row(map(set(), list(1, 2, 3))),
-                row(map(set(1, 2, 3), list(1, 2, 3))),
-                row(map(set(4, 5, 6), list(1, 2, 3))),
-                row(map(set(7, 8, 9), list(1, 2, 3)))
-            );
-
-            assertRows(execute("SELECT c FROM %s"),
-                row(set()),
-                row(set()),
-                row(set(1, 2, 3)),
-                row(set(1, 2, 3)),
-                row(set(1, 2, 3))
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a=?", 3),
-                row(3, map(set(4, 5, 6), list(1, 2, 3)), set(1, 2, 3))
-            );
-
-            execute("UPDATE %s SET b=? WHERE a=?", null, 1);
-            assertRows(execute("SELECT * FROM %s WHERE a=?", 1),
-                row(1, null, set())
-            );
-
-            execute("UPDATE %s SET b=? WHERE a=?", map(), 1);
-            assertRows(execute("SELECT * FROM %s WHERE a=?", 1),
-                row(1, map(), set())
-            );
-
-            execute("UPDATE %s SET c=? WHERE a=?", null, 2);
-            assertRows(execute("SELECT * FROM %s WHERE a=?", 2),
-                row(2, map(set(1, 2, 3), list(1, 2, 3)), null)
-            );
-
-            execute("UPDATE %s SET c=? WHERE a=?", set(), 2);
-            assertRows(execute("SELECT * FROM %s WHERE a=?", 2),
-                    row(2, map(set(1, 2, 3), list(1, 2, 3)), set())
-            );
-
-            execute("DELETE b FROM %s WHERE a=?", 3);
-            assertRows(execute("SELECT * FROM %s WHERE a=?", 3),
-                row(3, null, set(1, 2, 3))
-            );
-
-            execute("DELETE c FROM %s WHERE a=?", 4);
-            assertRows(execute("SELECT * FROM %s WHERE a=?", 4),
-                row(4, map(set(7, 8, 9), list(1, 2, 3)), null)
-            );
-        }
-    }
-
-    @Test
-    public void testStaticColumnUsage() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a int, b int, c frozen<map<set<int>, list<int>>> static, d int, PRIMARY KEY (a, b))");
-        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, map(), 0);
-        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, map(), 0);
-        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 1, 0, map(set(), list(1, 2, 3)), 0);
-        execute("INSERT INTO %s (a, b, d) VALUES (?, ?, ?)", 1, 1, 0);
-        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 2, 0, map(set(1, 2, 3), list(1, 2, 3)), 0);
-
-        assertRows(execute("SELECT * FROM %s"),
-            row(0, 0, map(), 0),
-            row(0, 1, map(), 0),
-            row(1, 0, map(set(), list(1, 2, 3)), 0),
-            row(1, 1, map(set(), list(1, 2, 3)), 0),
-            row(2, 0, map(set(1, 2, 3), list(1, 2, 3)), 0)
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE a=? AND b=?", 0, 1),
-            row(0, 1, map(), 0)
-        );
-
-        execute("DELETE c FROM %s WHERE a=?", 0);
-        assertRows(execute("SELECT * FROM %s"),
-                row(0, 0, null, 0),
-                row(0, 1, null, 0),
-                row(1, 0, map(set(), list(1, 2, 3)), 0),
-                row(1, 1, map(set(), list(1, 2, 3)), 0),
-                row(2, 0, map(set(1, 2, 3), list(1, 2, 3)), 0)
-        );
-
-        execute("DELETE FROM %s WHERE a=?", 0);
-        assertRows(execute("SELECT * FROM %s"),
-                row(1, 0, map(set(), list(1, 2, 3)), 0),
-                row(1, 1, map(set(), list(1, 2, 3)), 0),
-                row(2, 0, map(set(1, 2, 3), list(1, 2, 3)), 0)
-        );
-
-        execute("UPDATE %s SET c=? WHERE a=?", map(set(1, 2, 3), list(1, 2, 3)), 1);
-        assertRows(execute("SELECT * FROM %s"),
-                row(1, 0, map(set(1, 2, 3), list(1, 2, 3)), 0),
-                row(1, 1, map(set(1, 2, 3), list(1, 2, 3)), 0),
-                row(2, 0, map(set(1, 2, 3), list(1, 2, 3)), 0)
-        );
-    }
-
-    private void assertInvalidCreateWithMessage(String createTableStatement, String errorMessage) throws Throwable
-    {
-         try
-        {
-            createTableMayThrow(createTableStatement);
-            Assert.fail("Expected CREATE TABLE statement to error: " + createTableStatement);
-        }
-        catch (InvalidRequestException | ConfigurationException | SyntaxException ex)
-        {
-            Assert.assertTrue("Expected error message to contain '" + errorMessage + "', but got '" + ex.getMessage() + "'",
-                    ex.getMessage().contains(errorMessage));
-        }
-    }
-
-    private void assertInvalidAlterWithMessage(String createTableStatement, String errorMessage) throws Throwable
-    {
-        try
-        {
-            alterTableMayThrow(createTableStatement);
-            Assert.fail("Expected CREATE TABLE statement to error: " + createTableStatement);
-        }
-        catch (InvalidRequestException | ConfigurationException ex)
-        {
-            Assert.assertTrue("Expected error message to contain '" + errorMessage + "', but got '" + ex.getMessage() + "'",
-                    ex.getMessage().contains(errorMessage));
-        }
-    }
-
-    @Test
-    public void testInvalidOperations() throws Throwable
-    {
-        // lists
-        createTable("CREATE TABLE %s (k int PRIMARY KEY, l frozen<list<int>>)");
-        assertInvalid("UPDATE %s SET l[?]=? WHERE k=?", 0, 0, 0);
-        assertInvalid("UPDATE %s SET l = ? + l WHERE k=?", list(0), 0);
-        assertInvalid("UPDATE %s SET l = l + ? WHERE k=?", list(4), 0);
-        assertInvalid("UPDATE %s SET l = l - ? WHERE k=?", list(3), 0);
-        assertInvalid("DELETE l[?] FROM %s WHERE k=?", 0, 0);
-
-        // sets
-        createTable("CREATE TABLE %s (k int PRIMARY KEY, s frozen<set<int>>)");
-        assertInvalid("UPDATE %s SET s = s + ? WHERE k=?", set(0), 0);
-        assertInvalid("UPDATE %s SET s = s - ? WHERE k=?", set(3), 0);
-
-        // maps
-        createTable("CREATE TABLE %s (k int PRIMARY KEY, m frozen<map<int, int>>)");
-        assertInvalid("UPDATE %s SET m[?]=? WHERE k=?", 0, 0, 0);
-        assertInvalid("UPDATE %s SET m = m + ? WHERE k=?", map(4, 4), 0);
-        assertInvalid("DELETE m[?] FROM %s WHERE k=?", 0, 0);
-
-        assertInvalidCreateWithMessage("CREATE TABLE %s (k int PRIMARY KEY, t set<set<int>>)",
-                "Non-frozen collections are not allowed inside collections");
-
-        assertInvalidCreateWithMessage("CREATE TABLE %s (k int PRIMARY KEY, t frozen<set<counter>>)",
-                                       "Counters are not allowed inside collections");
-
-        assertInvalidCreateWithMessage("CREATE TABLE %s (k int PRIMARY KEY, t frozen<text>)",
-                "frozen<> is only allowed on collections, tuples, and user-defined types");
-    }
-
-    @Test
-    public void testAltering() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a int, b frozen<list<int>>, c frozen<list<int>>, PRIMARY KEY (a, b))");
-
-        alterTable("ALTER TABLE %s ALTER c TYPE frozen<list<blob>>");
-
-        assertInvalidAlterWithMessage("ALTER TABLE %s ALTER b TYPE frozen<list<blob>>",
-                                      "types are not order-compatible");
-
-        assertInvalidAlterWithMessage("ALTER TABLE %s ALTER b TYPE list<int>",
-                                      "types are not order-compatible");
-
-        assertInvalidAlterWithMessage("ALTER TABLE %s ALTER c TYPE list<blob>",
-                                      "types are incompatible");
-
-        alterTable("ALTER TABLE %s DROP c");
-        alterTable("ALTER TABLE %s ADD c frozen<set<int>>");
-        assertInvalidAlterWithMessage("ALTER TABLE %s ALTER c TYPE frozen<set<blob>>",
-                                      "types are incompatible");
-
-        alterTable("ALTER TABLE %s DROP c");
-        alterTable("ALTER TABLE %s ADD c frozen<map<int, int>>");
-        assertInvalidAlterWithMessage("ALTER TABLE %s ALTER c TYPE frozen<map<blob, int>>",
-                                      "types are incompatible");
-        alterTable("ALTER TABLE %s ALTER c TYPE frozen<map<int, blob>>");
-    }
-
-    private void assertInvalidIndexCreationWithMessage(String statement, String errorMessage) throws Throwable
-    {
-        try
-        {
-            createIndexMayThrow(statement);
-            Assert.fail("Expected index creation to fail: " + statement);
-        }
-        catch (InvalidRequestException ex)
-        {
-            Assert.assertTrue("Expected error message to contain '" + errorMessage + "', but got '" + ex.getMessage() + "'",
-                              ex.getMessage().contains(errorMessage));
-        }
-    }
-
-    @Test
-    public void testSecondaryIndex() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a frozen<map<int, text>> PRIMARY KEY, b frozen<map<int, text>>)");
-
-        // for now, we don't support indexing values or keys of collections in the primary key
-        assertInvalidIndexCreationWithMessage("CREATE INDEX ON %s (full(a))", "Cannot create secondary index on partition key column");
-        assertInvalidIndexCreationWithMessage("CREATE INDEX ON %s (keys(a))", "Cannot create index on keys of frozen<map> column");
-        assertInvalidIndexCreationWithMessage("CREATE INDEX ON %s (keys(b))", "Cannot create index on keys of frozen<map> column");
-
-        createTable("CREATE TABLE %s (a int, b frozen<list<int>>, c frozen<set<int>>, d frozen<map<int, text>>, PRIMARY KEY (a, b))");
-
-        createIndex("CREATE INDEX ON %s (full(b))");
-        createIndex("CREATE INDEX ON %s (full(c))");
-        createIndex("CREATE INDEX ON %s (full(d))");
-
-        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, list(1, 2, 3), set(1, 2, 3), map(1, "a"));
-        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, list(4, 5, 6), set(1, 2, 3), map(1, "a"));
-        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 1, list(1, 2, 3), set(4, 5, 6), map(2, "b"));
-        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 1, list(4, 5, 6), set(4, 5, 6), map(2, "b"));
-
-        // CONTAINS KEY doesn't work on non-maps
-        assertInvalidMessage("Cannot use CONTAINS KEY on non-map column",
-                             "SELECT * FROM %s WHERE b CONTAINS KEY ?", 1);
-
-        assertInvalidMessage("Cannot use CONTAINS KEY on non-map column",
-                             "SELECT * FROM %s WHERE b CONTAINS KEY ? ALLOW FILTERING", 1);
-
-        assertInvalidMessage("Cannot use CONTAINS KEY on non-map column",
-                             "SELECT * FROM %s WHERE c CONTAINS KEY ?", 1);
-
-        // normal indexes on frozen collections don't support CONTAINS or CONTAINS KEY
-        assertInvalidMessage("Cannot restrict column \"b\" by a CONTAINS relation without a secondary index",
-                             "SELECT * FROM %s WHERE b CONTAINS ?", 1);
-
-        assertInvalidMessage("Cannot restrict column \"b\" by a CONTAINS relation without a secondary index",
-                             "SELECT * FROM %s WHERE b CONTAINS ? ALLOW FILTERING", 1);
-
-        assertInvalidMessage("No secondary indexes on the restricted columns support the provided operator",
-                             "SELECT * FROM %s WHERE d CONTAINS KEY ?", 1);
-
-        assertInvalidMessage("No secondary indexes on the restricted columns support the provided operator",
-                             "SELECT * FROM %s WHERE d CONTAINS KEY ? ALLOW FILTERING", 1);
-
-        assertInvalidMessage("Cannot restrict column \"b\" by a CONTAINS relation without a secondary index",
-                             "SELECT * FROM %s WHERE b CONTAINS ? AND d CONTAINS KEY ? ALLOW FILTERING", 1, 1);
-
-        // index lookup on b
-        assertRows(execute("SELECT * FROM %s WHERE b=?", list(1, 2, 3)),
-            row(0, list(1, 2, 3), set(1, 2, 3), map(1, "a")),
-            row(1, list(1, 2, 3), set(4, 5, 6), map(2, "b"))
-        );
-
-        assertEmpty(execute("SELECT * FROM %s WHERE b=?", list(-1)));
-
-        assertInvalidMessage("ALLOW FILTERING", "SELECT * FROM %s WHERE b=? AND c=?", list(1, 2, 3), set(4, 5, 6));
-        assertRows(execute("SELECT * FROM %s WHERE b=? AND c=? ALLOW FILTERING", list(1, 2, 3), set(4, 5, 6)),
-            row(1, list(1, 2, 3), set(4, 5, 6), map(2, "b"))
-        );
-
-        assertInvalidMessage("ALLOW FILTERING", "SELECT * FROM %s WHERE b=? AND c CONTAINS ?", list(1, 2, 3), 5);
-        assertRows(execute("SELECT * FROM %s WHERE b=? AND c CONTAINS ? ALLOW FILTERING", list(1, 2, 3), 5),
-            row(1, list(1, 2, 3), set(4, 5, 6), map(2, "b"))
-        );
-
-        assertInvalidMessage("ALLOW FILTERING", "SELECT * FROM %s WHERE b=? AND d=?", list(1, 2, 3), map(1, "a"));
-        assertRows(execute("SELECT * FROM %s WHERE b=? AND d=? ALLOW FILTERING", list(1, 2, 3), map(1, "a")),
-            row(0, list(1, 2, 3), set(1, 2, 3), map(1, "a"))
-        );
-
-        assertInvalidMessage("ALLOW FILTERING", "SELECT * FROM %s WHERE b=? AND d CONTAINS ?", list(1, 2, 3), "a");
-        assertRows(execute("SELECT * FROM %s WHERE b=? AND d CONTAINS ? ALLOW FILTERING", list(1, 2, 3), "a"),
-            row(0, list(1, 2, 3), set(1, 2, 3), map(1, "a"))
-        );
-
-        assertInvalidMessage("ALLOW FILTERING", "SELECT * FROM %s WHERE b=? AND d CONTAINS KEY ?", list(1, 2, 3), 1);
-        assertRows(execute("SELECT * FROM %s WHERE b=? AND d CONTAINS KEY ? ALLOW FILTERING", list(1, 2, 3), 1),
-            row(0, list(1, 2, 3), set(1, 2, 3), map(1, "a"))
-        );
-
-        // index lookup on c
-        assertRows(execute("SELECT * FROM %s WHERE c=?", set(1, 2, 3)),
-            row(0, list(1, 2, 3), set(1, 2, 3), map(1, "a")),
-            row(0, list(4, 5, 6), set(1, 2, 3), map(1, "a"))
-        );
-
-        // ordering of c should not matter
-        assertRows(execute("SELECT * FROM %s WHERE c=?", set(2, 1, 3)),
-                row(0, list(1, 2, 3), set(1, 2, 3), map(1, "a")),
-                row(0, list(4, 5, 6), set(1, 2, 3), map(1, "a"))
-        );
-
-        assertEmpty(execute("SELECT * FROM %s WHERE c=?", set(-1)));
-
-        assertInvalidMessage("ALLOW FILTERING", "SELECT * FROM %s WHERE c=? AND b=?", set(1, 2, 3), list(1, 2, 3));
-        assertRows(execute("SELECT * FROM %s WHERE c=? AND b=? ALLOW FILTERING", set(1, 2, 3), list(1, 2, 3)),
-            row(0, list(1, 2, 3), set(1, 2, 3), map(1, "a"))
-        );
-
-        assertInvalidMessage("ALLOW FILTERING", "SELECT * FROM %s WHERE c=? AND b CONTAINS ?", set(1, 2, 3), 1);
-        assertRows(execute("SELECT * FROM %s WHERE c=? AND b CONTAINS ? ALLOW FILTERING", set(1, 2, 3), 1),
-            row(0, list(1, 2, 3), set(1, 2, 3), map(1, "a"))
-        );
-
-        assertInvalidMessage("ALLOW FILTERING", "SELECT * FROM %s WHERE c=? AND d = ?", set(1, 2, 3), map(1, "a"));
-        assertRows(execute("SELECT * FROM %s WHERE c=? AND d = ? ALLOW FILTERING", set(1, 2, 3), map(1, "a")),
-            row(0, list(1, 2, 3), set(1, 2, 3), map(1, "a")),
-            row(0, list(4, 5, 6), set(1, 2, 3), map(1, "a"))
-        );
-
-        assertInvalidMessage("ALLOW FILTERING", "SELECT * FROM %s WHERE c=? AND d CONTAINS ?", set(1, 2, 3), "a");
-        assertRows(execute("SELECT * FROM %s WHERE c=? AND d CONTAINS ? ALLOW FILTERING", set(1, 2, 3), "a"),
-            row(0, list(1, 2, 3), set(1, 2, 3), map(1, "a")),
-            row(0, list(4, 5, 6), set(1, 2, 3), map(1, "a"))
-        );
-
-        assertInvalidMessage("ALLOW FILTERING", "SELECT * FROM %s WHERE c=? AND d CONTAINS KEY ?", set(1, 2, 3), 1);
-        assertRows(execute("SELECT * FROM %s WHERE c=? AND d CONTAINS KEY ? ALLOW FILTERING", set(1, 2, 3), 1),
-            row(0, list(1, 2, 3), set(1, 2, 3), map(1, "a")),
-            row(0, list(4, 5, 6), set(1, 2, 3), map(1, "a"))
-        );
-
-        // index lookup on d
-        assertRows(execute("SELECT * FROM %s WHERE d=?", map(1, "a")),
-            row(0, list(1, 2, 3), set(1, 2, 3), map(1, "a")),
-            row(0, list(4, 5, 6), set(1, 2, 3), map(1, "a"))
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE d=?", map(2, "b")),
-            row(1, list(1, 2, 3), set(4, 5, 6), map(2, "b")),
-            row(1, list(4, 5, 6), set(4, 5, 6), map(2, "b"))
-        );
-
-        assertEmpty(execute("SELECT * FROM %s WHERE d=?", map(3, "c")));
-
-        assertInvalidMessage("ALLOW FILTERING", "SELECT * FROM %s WHERE d=? AND c=?", map(1, "a"), set(1, 2, 3));
-        assertRows(execute("SELECT * FROM %s WHERE d=? AND b=? ALLOW FILTERING", map(1, "a"), list(1, 2, 3)),
-            row(0, list(1, 2, 3), set(1, 2, 3), map(1, "a"))
-        );
-
-        assertInvalidMessage("ALLOW FILTERING", "SELECT * FROM %s WHERE d=? AND b CONTAINS ?", map(1, "a"), 3);
-        assertRows(execute("SELECT * FROM %s WHERE d=? AND b CONTAINS ? ALLOW FILTERING", map(1, "a"), 3),
-            row(0, list(1, 2, 3), set(1, 2, 3), map(1, "a"))
-        );
-
-        assertInvalidMessage("ALLOW FILTERING", "SELECT * FROM %s WHERE d=? AND b=? AND c=?", map(1, "a"), list(1, 2, 3), set(1, 2, 3));
-        assertRows(execute("SELECT * FROM %s WHERE d=? AND b=? AND c=? ALLOW FILTERING", map(1, "a"), list(1, 2, 3), set(1, 2, 3)),
-            row(0, list(1, 2, 3), set(1, 2, 3), map(1, "a"))
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE d=? AND b CONTAINS ? AND c CONTAINS ? ALLOW FILTERING", map(1, "a"), 2, 2),
-            row(0, list(1, 2, 3), set(1, 2, 3), map(1, "a"))
-        );
-
-        execute("DELETE d FROM %s WHERE a=? AND b=?", 0, list(1, 2, 3));
-        assertRows(execute("SELECT * FROM %s WHERE d=?", map(1, "a")),
-            row(0, list(4, 5, 6), set(1, 2, 3), map(1, "a"))
-        );
-    }
-
-    /** Test for CASSANDRA-8302 */
-    @Test
-    public void testClusteringColumnFiltering() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a int, b frozen<map<int, int>>, c int, d int, PRIMARY KEY (a, b, c))");
-        createIndex("CREATE INDEX c_index ON %s (c)");
-        createIndex("CREATE INDEX d_index ON %s (d)");
-
-        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, map(0, 0, 1, 1), 0, 0);
-        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, map(1, 1, 2, 2), 0, 0);
-        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 1, map(0, 0, 1, 1), 0, 0);
-        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 1, map(1, 1, 2, 2), 0, 0);
-
-        assertRows(execute("SELECT * FROM %s WHERE d=? AND b CONTAINS ? ALLOW FILTERING", 0, 0),
-                row(0, map(0, 0, 1, 1), 0, 0),
-                row(1, map(0, 0, 1, 1), 0, 0)
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE d=? AND b CONTAINS KEY ? ALLOW FILTERING", 0, 0),
-                row(0, map(0, 0, 1, 1), 0, 0),
-                row(1, map(0, 0, 1, 1), 0, 0)
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE a=? AND d=? AND b CONTAINS ? ALLOW FILTERING", 0, 0, 0),
-                row(0, map(0, 0, 1, 1), 0, 0)
-        );
-        assertRows(execute("SELECT * FROM %s WHERE a=? AND d=? AND b CONTAINS KEY ? ALLOW FILTERING", 0, 0, 0),
-                row(0, map(0, 0, 1, 1), 0, 0)
-        );
-
-        dropIndex("DROP INDEX %s.d_index");
-
-        assertRows(execute("SELECT * FROM %s WHERE c=? AND b CONTAINS ? ALLOW FILTERING", 0, 0),
-                row(0, map(0, 0, 1, 1), 0, 0),
-                row(1, map(0, 0, 1, 1), 0, 0)
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE c=? AND b CONTAINS KEY ? ALLOW FILTERING", 0, 0),
-                row(0, map(0, 0, 1, 1), 0, 0),
-                row(1, map(0, 0, 1, 1), 0, 0)
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE a=? AND c=? AND b CONTAINS ? ALLOW FILTERING", 0, 0, 0),
-                row(0, map(0, 0, 1, 1), 0, 0)
-        );
-        assertRows(execute("SELECT * FROM %s WHERE a=? AND c=? AND b CONTAINS KEY ? ALLOW FILTERING", 0, 0, 0),
-                row(0, map(0, 0, 1, 1), 0, 0)
-        );
-    }
-
-    @Test
-    public void testFrozenListInMap() throws Throwable
-    {
-        createTable("CREATE TABLE %s (k int primary key, m map<frozen<list<int>>, int>)");
-
-        execute("INSERT INTO %s (k, m) VALUES (1, {[1, 2, 3] : 1})");
-        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
-                row(1, map(list(1, 2, 3), 1)));
-
-        execute("UPDATE %s SET m[[1, 2, 3]]=2 WHERE k=1");
-        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
-                row(1, map(list(1, 2, 3), 2)));
-
-        execute("UPDATE %s SET m = m + ? WHERE k=1", map(list(4, 5, 6), 3));
-        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
-                row(1,
-                    map(list(1, 2, 3), 2,
-                        list(4, 5, 6), 3)));
-
-        execute("DELETE m[[1, 2, 3]] FROM %s WHERE k = 1");
-        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
-                row(1, map(list(4, 5, 6), 3)));
-    }
-
-    @Test
-    public void testFrozenListInSet() throws Throwable
-    {
-        createTable("CREATE TABLE %s (k int primary key, s set<frozen<list<int>>>)");
-
-        execute("INSERT INTO %s (k, s) VALUES (1, {[1, 2, 3]})");
-        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
-                row(1, set(list(1, 2, 3)))
-        );
-
-        execute("UPDATE %s SET s = s + ? WHERE k=1", set(list(4, 5, 6)));
-        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
-                row(1, set(list(1, 2, 3), list(4, 5, 6)))
-        );
-
-        execute("UPDATE %s SET s = s - ? WHERE k=1", set(list(4, 5, 6)));
-        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
-                row(1, set(list(1, 2, 3)))
-        );
-
-        execute("DELETE s[[1, 2, 3]] FROM %s WHERE k = 1");
-        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
-                row(1, null)
-        );
-    }
-
-    @Test
-    public void testFrozenListInList() throws Throwable
-    {
-        createTable("CREATE TABLE %s (k int primary key, l list<frozen<list<int>>>)");
-
-        execute("INSERT INTO %s (k, l) VALUES (1, [[1, 2, 3]])");
-        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
-                row(1, list(list(1, 2, 3)))
-        );
-
-        execute("UPDATE %s SET l[?]=? WHERE k=1", 0, list(4, 5, 6));
-        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
-                row(1, list(list(4, 5, 6)))
-        );
-
-        execute("UPDATE %s SET l = ? + l WHERE k=1", list(list(1, 2, 3)));
-        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
-                row(1, list(list(1, 2, 3), list(4, 5, 6)))
-        );
-
-        execute("UPDATE %s SET l = l + ? WHERE k=1", list(list(7, 8, 9)));
-        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
-                row(1, list(list(1, 2, 3), list(4, 5, 6), list(7, 8, 9)))
-        );
-
-        execute("UPDATE %s SET l = l - ? WHERE k=1", list(list(4, 5, 6)));
-        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
-                row(1, list(list(1, 2, 3), list(7, 8, 9)))
-        );
-
-        execute("DELETE l[0] FROM %s WHERE k = 1");
-        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
-                row(1, list(list(7, 8, 9)))
-        );
-    }
-
-    @Test
-    public void testFrozenMapInMap() throws Throwable
-    {
-        createTable("CREATE TABLE %s (k int primary key, m map<frozen<map<int, int>>, int>)");
-
-        execute("INSERT INTO %s (k, m) VALUES (1, {{1 : 1, 2 : 2} : 1})");
-        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
-                row(1, map(map(1, 1, 2, 2), 1)));
-
-        execute("UPDATE %s SET m[?]=2 WHERE k=1", map(1, 1, 2, 2));
-        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
-                row(1, map(map(1, 1, 2, 2), 2)));
-
-        execute("UPDATE %s SET m = m + ? WHERE k=1", map(map(3, 3, 4, 4), 3));
-        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
-                row(1,
-                    map(map(1, 1, 2, 2), 2,
-                        map(3, 3, 4, 4), 3)));
-
-        execute("DELETE m[?] FROM %s WHERE k = 1", map(1, 1, 2, 2));
-        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
-                row(1, map(map(3, 3, 4, 4), 3)));
-    }
-
-    @Test
-    public void testFrozenMapInSet() throws Throwable
-    {
-        createTable("CREATE TABLE %s (k int primary key, s set<frozen<map<int, int>>>)");
-
-        execute("INSERT INTO %s (k, s) VALUES (1, {{1 : 1, 2 : 2}})");
-
-        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
-                row(1, set(map(1, 1, 2, 2)))
-        );
-
-        execute("UPDATE %s SET s = s + ? WHERE k=1", set(map(3, 3, 4, 4)));
-        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
-                row(1, set(map(1, 1, 2, 2), map(3, 3, 4, 4)))
-        );
-
-        execute("UPDATE %s SET s = s - ? WHERE k=1", set(map(3, 3, 4, 4)));
-        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
-                row(1, set(map(1, 1, 2, 2)))
-        );
-
-        execute("DELETE s[?] FROM %s WHERE k = 1", map(1, 1, 2, 2));
-        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
-                row(1, null)
-        );
-    }
-
-    @Test
-    public void testFrozenMapInList() throws Throwable
-    {
-        createTable("CREATE TABLE %s (k int primary key, l list<frozen<map<int, int>>>)");
-
-        execute("INSERT INTO %s (k, l) VALUES (1, [{1 : 1, 2 : 2}])");
-        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
-                row(1, list(map(1, 1, 2, 2)))
-        );
-
-        execute("UPDATE %s SET l[?]=? WHERE k=1", 0, map(3, 3, 4, 4));
-        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
-                row(1, list(map(3, 3, 4, 4)))
-        );
-
-        execute("UPDATE %s SET l = ? + l WHERE k=1", list(map(1, 1, 2, 2)));
-        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
-                row(1, list(map(1, 1, 2, 2), map(3, 3, 4, 4)))
-        );
-
-        execute("UPDATE %s SET l = l + ? WHERE k=1", list(map(5, 5, 6, 6)));
-        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
-                row(1, list(map(1, 1, 2, 2), map(3, 3, 4, 4), map(5, 5, 6, 6)))
-        );
-
-        execute("UPDATE %s SET l = l - ? WHERE k=1", list(map(3, 3, 4, 4)));
-        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
-                row(1, list(map(1, 1, 2, 2), map(5, 5, 6, 6)))
-        );
-
-        execute("DELETE l[0] FROM %s WHERE k = 1");
-        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
-                row(1, list(map(5, 5, 6, 6)))
-        );
-    }
-
-    @Test
-    public void testFrozenSetInMap() throws Throwable
-    {
-        createTable("CREATE TABLE %s (k int primary key, m map<frozen<set<int>>, int>)");
-
-        execute("INSERT INTO %s (k, m) VALUES (1, {{1, 2, 3} : 1})");
-        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
-                row(1, map(set(1, 2, 3), 1)));
-
-        execute("UPDATE %s SET m[?]=2 WHERE k=1", set(1, 2, 3));
-        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
-                row(1, map(set(1, 2, 3), 2)));
-
-        execute("UPDATE %s SET m = m + ? WHERE k=1", map(set(4, 5, 6), 3));
-        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
-                row(1,
-                    map(set(1, 2, 3), 2,
-                        set(4, 5, 6), 3)));
-
-        execute("DELETE m[?] FROM %s WHERE k = 1", set(1, 2, 3));
-        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
-                row(1, map(set(4, 5, 6), 3)));
-    }
-
-    @Test
-    public void testFrozenSetInSet() throws Throwable
-    {
-        createTable("CREATE TABLE %s (k int primary key, s set<frozen<set<int>>>)");
-
-        execute("INSERT INTO %s (k, s) VALUES (1, {{1, 2, 3}})");
-
-        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
-                row(1, set(set(1, 2, 3)))
-        );
-
-        execute("UPDATE %s SET s = s + ? WHERE k=1", set(set(4, 5, 6)));
-        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
-                row(1, set(set(1, 2, 3), set(4, 5, 6)))
-        );
-
-        execute("UPDATE %s SET s = s - ? WHERE k=1", set(set(4, 5, 6)));
-        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
-                row(1, set(set(1, 2, 3)))
-        );
-
-        execute("DELETE s[?] FROM %s WHERE k = 1", set(1, 2, 3));
-        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
-                row(1, null)
-        );
-    }
-
-    @Test
-    public void testFrozenSetInList() throws Throwable
-    {
-        createTable("CREATE TABLE %s (k int primary key, l list<frozen<set<int>>>)");
-
-        execute("INSERT INTO %s (k, l) VALUES (1, [{1, 2, 3}])");
-        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
-                row(1, list(set(1, 2, 3)))
-        );
-
-        execute("UPDATE %s SET l[?]=? WHERE k=1", 0, set(4, 5, 6));
-        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
-                row(1, list(set(4, 5, 6)))
-        );
-
-        execute("UPDATE %s SET l = ? + l WHERE k=1", list(set(1, 2, 3)));
-        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
-                row(1, list(set(1, 2, 3), set(4, 5, 6)))
-        );
-
-        execute("UPDATE %s SET l = l + ? WHERE k=1", list(set(7, 8, 9)));
-        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
-                row(1, list(set(1, 2, 3), set(4, 5, 6), set(7, 8, 9)))
-        );
-
-        execute("UPDATE %s SET l = l - ? WHERE k=1", list(set(4, 5, 6)));
-        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
-                row(1, list(set(1, 2, 3), set(7, 8, 9)))
-        );
-
-        execute("DELETE l[0] FROM %s WHERE k = 1");
-        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
-                row(1, list(set(7, 8, 9)))
-        );
-    }
-
-    @Test
-    public void testUserDefinedTypes() throws Throwable
-    {
-        String myType = createType("CREATE TYPE %s (a set<int>, b tuple<list<int>>)");
-        createTable("CREATE TABLE %s (k int PRIMARY KEY, v frozen<" + myType + ">)");
-        execute("INSERT INTO %s (k, v) VALUES (?, {a: ?, b: ?})", 0, set(1, 2, 3), tuple(list(1, 2, 3)));
-        assertRows(execute("SELECT v.a, v.b FROM %s WHERE k=?", 0),
-            row(set(1, 2, 3), tuple(list(1, 2, 3)))
-        );
-    }
-
-    private static String clean(String classname)
-    {
-        return StringUtils.remove(classname, "org.apache.cassandra.db.marshal.");
-    }
-
-    @Test
-    public void testToString()
-    {
-        // set<frozen<list<int>>>
-        SetType t = SetType.getInstance(ListType.getInstance(Int32Type.instance, false), true);
-        assertEquals("SetType(FrozenType(ListType(Int32Type)))", clean(t.toString()));
-        assertEquals("SetType(ListType(Int32Type))", clean(t.toString(true)));
-
-        // frozen<set<list<int>>>
-        t = SetType.getInstance(ListType.getInstance(Int32Type.instance, false), false);
-        assertEquals("FrozenType(SetType(ListType(Int32Type)))", clean(t.toString()));
-        assertEquals("SetType(ListType(Int32Type))", clean(t.toString(true)));
-
-        // map<frozen<list<int>>, int>
-        MapType m = MapType.getInstance(ListType.getInstance(Int32Type.instance, false), Int32Type.instance, true);
-        assertEquals("MapType(FrozenType(ListType(Int32Type)),Int32Type)", clean(m.toString()));
-        assertEquals("MapType(ListType(Int32Type),Int32Type)", clean(m.toString(true)));
-
-        // frozen<map<list<int>, int>>
-        m = MapType.getInstance(ListType.getInstance(Int32Type.instance, false), Int32Type.instance, false);
-        assertEquals("FrozenType(MapType(ListType(Int32Type),Int32Type))", clean(m.toString()));
-        assertEquals("MapType(ListType(Int32Type),Int32Type)", clean(m.toString(true)));
-
-        // tuple<set<int>>
-        List<AbstractType<?>> types = new ArrayList<>();
-        types.add(SetType.getInstance(Int32Type.instance, true));
-        TupleType tuple = new TupleType(types);
-        assertEquals("TupleType(SetType(Int32Type))", clean(tuple.toString()));
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f797bfa4/test/unit/org/apache/cassandra/cql3/IndexedValuesValidationTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/IndexedValuesValidationTest.java b/test/unit/org/apache/cassandra/cql3/IndexedValuesValidationTest.java
deleted file mode 100644
index 487fbc2..0000000
--- a/test/unit/org/apache/cassandra/cql3/IndexedValuesValidationTest.java
+++ /dev/null
@@ -1,149 +0,0 @@
-/*
- *
- *  * 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;
-
-import java.nio.ByteBuffer;
-import java.util.HashMap;
-import java.util.Map;
-
-import org.junit.Test;
-
-import org.apache.cassandra.utils.FBUtilities;
-
-import static org.junit.Assert.fail;
-
-public class IndexedValuesValidationTest extends CQLTester
-{
-    private static final int TOO_BIG = 1024 * 65;
-    // CASSANDRA-8280/8081
-    // reject updates with indexed values where value > 64k
-    @Test
-    public void testIndexOnCompositeValueOver64k() throws Throwable
-    {
-        createTable("CREATE TABLE %s(a int, b int, c blob, PRIMARY KEY (a))");
-        createIndex("CREATE INDEX ON %s(c)");
-        failInsert("INSERT INTO %s (a, b, c) VALUES (0, 0, ?)", ByteBuffer.allocate(TOO_BIG));
-    }
-
-    @Test
-    public void testIndexOnClusteringColumnInsertPartitionKeyAndClusteringsOver64k() throws Throwable
-    {
-        createTable("CREATE TABLE %s(a blob, b blob, c blob, d int, PRIMARY KEY (a, b, c))");
-        createIndex("CREATE INDEX ON %s(b)");
-
-        // CompositeIndexOnClusteringKey creates index entries composed of the
-        // PK plus all of the non-indexed clustering columns from the primary row
-        // so we should reject where len(a) + len(c) > 65560 as this will form the
-        // total clustering in the index table
-        ByteBuffer a = ByteBuffer.allocate(100);
-        ByteBuffer b = ByteBuffer.allocate(10);
-        ByteBuffer c = ByteBuffer.allocate(FBUtilities.MAX_UNSIGNED_SHORT - 99);
-
-        failInsert("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, 0)", a, b, c);
-    }
-
-    @Test
-    public void testCompactTableWithValueOver64k() throws Throwable
-    {
-        createTable("CREATE TABLE %s(a int, b blob, PRIMARY KEY (a)) WITH COMPACT STORAGE");
-        createIndex("CREATE INDEX ON %s(b)");
-        failInsert("INSERT INTO %s (a, b) VALUES (0, ?)", ByteBuffer.allocate(TOO_BIG));
-    }
-
-    @Test
-    public void testIndexOnCollectionValueInsertPartitionKeyAndCollectionKeyOver64k() throws Throwable
-    {
-        createTable("CREATE TABLE %s(a blob , b map<blob, int>, PRIMARY KEY (a))");
-        createIndex("CREATE INDEX ON %s(b)");
-
-        // A collection key > 64k by itself will be rejected from
-        // the primary table.
-        // To test index validation we need to ensure that
-        // len(b) < 64k, but len(a) + len(b) > 64k as that will
-        // form the clustering in the index table
-        ByteBuffer a = ByteBuffer.allocate(100);
-        ByteBuffer b = ByteBuffer.allocate(FBUtilities.MAX_UNSIGNED_SHORT - 100);
-
-        failInsert("UPDATE %s SET b[?] = 0 WHERE a = ?", b, a);
-    }
-
-    @Test
-    public void testIndexOnCollectionKeyInsertPartitionKeyAndClusteringOver64k() throws Throwable
-    {
-        createTable("CREATE TABLE %s(a blob, b blob, c map<blob, int>, PRIMARY KEY (a, b))");
-        createIndex("CREATE INDEX ON %s(KEYS(c))");
-
-        // Basically the same as the case with non-collection clustering
-        // CompositeIndexOnCollectionKeyy creates index entries composed of the
-        // PK plus all of the clustering columns from the primary row, except the
-        // collection element - which becomes the partition key in the index table
-        ByteBuffer a = ByteBuffer.allocate(100);
-        ByteBuffer b = ByteBuffer.allocate(FBUtilities.MAX_UNSIGNED_SHORT - 100);
-        ByteBuffer c = ByteBuffer.allocate(10);
-
-        failInsert("UPDATE %s SET c[?] = 0 WHERE a = ? and b = ?", c, a, b);
-    }
-
-    @Test
-    public void testIndexOnPartitionKeyInsertValueOver64k() throws Throwable
-    {
-        createTable("CREATE TABLE %s(a int, b int, c blob, PRIMARY KEY ((a, b)))");
-        createIndex("CREATE INDEX ON %s(a)");
-        succeedInsert("INSERT INTO %s (a, b, c) VALUES (0, 0, ?)", ByteBuffer.allocate(TOO_BIG));
-    }
-
-    @Test
-    public void testIndexOnClusteringColumnInsertValueOver64k() throws Throwable
-    {
-        createTable("CREATE TABLE %s(a int, b int, c blob, PRIMARY KEY (a, b))");
-        createIndex("CREATE INDEX ON %s(b)");
-        succeedInsert("INSERT INTO %s (a, b, c) VALUES (0, 0, ?)", ByteBuffer.allocate(TOO_BIG));
-    }
-
-    @Test
-    public void testIndexOnFullCollectionEntryInsertCollectionValueOver64k() throws Throwable
-    {
-        createTable("CREATE TABLE %s(a int, b frozen<map<int, blob>>, PRIMARY KEY (a))");
-        createIndex("CREATE INDEX ON %s(full(b))");
-        Map<Integer, ByteBuffer> map = new HashMap();
-        map.put(0, ByteBuffer.allocate(1024 * 65));
-        failInsert("INSERT INTO %s (a, b) VALUES (0, ?)", map);
-    }
-
-    public void failInsert(String insertCQL, Object...args) throws Throwable
-    {
-        try
-        {
-            execute(insertCQL, args);
-            fail("Expected statement to fail validation");
-        }
-        catch (Exception e)
-        {
-            // as expected
-        }
-    }
-
-    public void succeedInsert(String insertCQL, Object...args) throws Throwable
-    {
-        execute(insertCQL, args);
-        flush();
-    }
-}


[18/32] cassandra git commit: 2.2 commit for CASSANDRA-9160

Posted by jm...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/01115f72/test/unit/org/apache/cassandra/cql3/validation/entities/UFAuthTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/UFAuthTest.java b/test/unit/org/apache/cassandra/cql3/validation/entities/UFAuthTest.java
new file mode 100644
index 0000000..498f0dd
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/UFAuthTest.java
@@ -0,0 +1,728 @@
+/*
+ * 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.validation.entities;
+
+import java.lang.reflect.Field;
+import java.util.*;
+
+import com.google.common.base.Joiner;
+import com.google.common.collect.ImmutableSet;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.auth.*;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.cql3.Attributes;
+import org.apache.cassandra.cql3.CQLStatement;
+import org.apache.cassandra.cql3.QueryProcessor;
+import org.apache.cassandra.cql3.functions.Function;
+import org.apache.cassandra.cql3.functions.FunctionName;
+import org.apache.cassandra.cql3.functions.Functions;
+import org.apache.cassandra.cql3.statements.BatchStatement;
+import org.apache.cassandra.cql3.statements.ModificationStatement;
+import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.exceptions.*;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.utils.Pair;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+public class UFAuthTest extends CQLTester
+{
+    private static final Logger logger = LoggerFactory.getLogger(UFAuthTest.class);
+
+    String roleName = "test_role";
+    AuthenticatedUser user;
+    RoleResource role;
+    ClientState clientState;
+
+    @BeforeClass
+    public static void setupAuthorizer()
+    {
+        try
+        {
+            IAuthorizer authorizer = new StubAuthorizer();
+            Field authorizerField = DatabaseDescriptor.class.getDeclaredField("authorizer");
+            authorizerField.setAccessible(true);
+            authorizerField.set(null, authorizer);
+            DatabaseDescriptor.setPermissionsValidity(0);
+        }
+        catch (IllegalAccessException | NoSuchFieldException e)
+        {
+            throw new RuntimeException(e);
+        }
+    }
+
+    @Before
+    public void setup() throws Throwable
+    {
+        ((StubAuthorizer) DatabaseDescriptor.getAuthorizer()).clear();
+        setupClientState();
+        setupTable("CREATE TABLE %s (k int, v1 int, v2 int, PRIMARY KEY (k, v1))");
+    }
+
+    @Test
+    public void functionInSelection() throws Throwable
+    {
+        String functionName = createSimpleFunction();
+        String cql = String.format("SELECT k, %s FROM %s WHERE k = 1;",
+                                   functionCall(functionName),
+                                   KEYSPACE + "." + currentTable());
+        assertPermissionsOnFunction(cql, functionName);
+    }
+
+    @Test
+    public void functionInSelectPKRestriction() throws Throwable
+    {
+        String functionName = createSimpleFunction();
+        String cql = String.format("SELECT * FROM %s WHERE k = %s",
+                                   KEYSPACE + "." + currentTable(),
+                                   functionCall(functionName));
+        assertPermissionsOnFunction(cql, functionName);
+    }
+
+    @Test
+    public void functionInSelectClusteringRestriction() throws Throwable
+    {
+        String functionName = createSimpleFunction();
+        String cql = String.format("SELECT * FROM %s WHERE k = 0 AND v1 = %s",
+                                   KEYSPACE + "." + currentTable(),
+                                   functionCall(functionName));
+        assertPermissionsOnFunction(cql, functionName);
+    }
+
+    @Test
+    public void functionInSelectInRestriction() throws Throwable
+    {
+        String functionName = createSimpleFunction();
+        String cql = String.format("SELECT * FROM %s WHERE k IN (%s, %s)",
+                                   KEYSPACE + "." + currentTable(),
+                                   functionCall(functionName),
+                                   functionCall(functionName));
+        assertPermissionsOnFunction(cql, functionName);
+    }
+
+    @Test
+    public void functionInSelectMultiColumnInRestriction() throws Throwable
+    {
+        setupTable("CREATE TABLE %s (k int, v1 int, v2 int, v3 int, PRIMARY KEY (k, v1, v2))");
+        String functionName = createSimpleFunction();
+        String cql = String.format("SELECT * FROM %s WHERE k=0 AND (v1, v2) IN ((%s, %s))",
+                                   KEYSPACE + "." + currentTable(),
+                                   functionCall(functionName),
+                                   functionCall(functionName));
+        assertPermissionsOnFunction(cql, functionName);
+    }
+
+    @Test
+    public void functionInSelectMultiColumnEQRestriction() throws Throwable
+    {
+        setupTable("CREATE TABLE %s (k int, v1 int, v2 int, v3 int, PRIMARY KEY (k, v1, v2))");
+        String functionName = createSimpleFunction();
+        String cql = String.format("SELECT * FROM %s WHERE k=0 AND (v1, v2) = (%s, %s)",
+                                   KEYSPACE + "." + currentTable(),
+                                   functionCall(functionName),
+                                   functionCall(functionName));
+        assertPermissionsOnFunction(cql, functionName);
+    }
+
+    @Test
+    public void functionInSelectMultiColumnSliceRestriction() throws Throwable
+    {
+        setupTable("CREATE TABLE %s (k int, v1 int, v2 int, v3 int, PRIMARY KEY (k, v1, v2))");
+        String functionName = createSimpleFunction();
+        String cql = String.format("SELECT * FROM %s WHERE k=0 AND (v1, v2) < (%s, %s)",
+                                   KEYSPACE + "." + currentTable(),
+                                   functionCall(functionName),
+                                   functionCall(functionName));
+        assertPermissionsOnFunction(cql, functionName);
+    }
+
+    @Test
+    public void functionInSelectTokenEQRestriction() throws Throwable
+    {
+        String functionName = createSimpleFunction();
+        String cql = String.format("SELECT * FROM %s WHERE token(k) = token(%s)",
+                                   KEYSPACE + "." + currentTable(),
+                                   functionCall(functionName));
+        assertPermissionsOnFunction(cql, functionName);
+    }
+
+    @Test
+    public void functionInSelectTokenSliceRestriction() throws Throwable
+    {
+        String functionName = createSimpleFunction();
+        String cql = String.format("SELECT * FROM %s WHERE token(k) < token(%s)",
+                                   KEYSPACE + "." + currentTable(),
+                                   functionCall(functionName));
+        assertPermissionsOnFunction(cql, functionName);
+    }
+
+    @Test
+    public void functionInPKForInsert() throws Throwable
+    {
+        String functionName = createSimpleFunction();
+        String cql = String.format("INSERT INTO %s (k, v1, v2) VALUES (%s, 0, 0)",
+                                   KEYSPACE + "." + currentTable(),
+                                   functionCall(functionName));
+        assertPermissionsOnFunction(cql, functionName);
+    }
+
+    @Test
+    public void functionInClusteringValuesForInsert() throws Throwable
+    {
+        String functionName = createSimpleFunction();
+        String cql = String.format("INSERT INTO %s (k, v1, v2) VALUES (0, %s, 0)",
+                                   KEYSPACE + "." + currentTable(),
+                                   functionCall(functionName));
+        assertPermissionsOnFunction(cql, functionName);
+    }
+
+    @Test
+    public void functionInPKForDelete() throws Throwable
+    {
+        String functionName = createSimpleFunction();
+        String cql = String.format("DELETE FROM %s WHERE k = %s",
+                                   KEYSPACE + "." + currentTable(),
+                                   functionCall(functionName));
+        assertPermissionsOnFunction(cql, functionName);
+    }
+
+    @Test
+    public void functionInClusteringValuesForDelete() throws Throwable
+    {
+        String functionName = createSimpleFunction();
+        String cql = String.format("DELETE FROM %s WHERE k = 0 AND v1 = %s",
+                                   KEYSPACE + "." + currentTable(),
+                                   functionCall(functionName));
+        assertPermissionsOnFunction(cql, functionName);
+    }
+
+    @Test
+    public void testBatchStatement() throws Throwable
+    {
+        List<ModificationStatement> statements = new ArrayList<>();
+        List<String> functions = new ArrayList<>();
+        for (int i = 0; i < 3; i++)
+        {
+            String functionName = createSimpleFunction();
+            ModificationStatement stmt =
+            (ModificationStatement) getStatement(String.format("INSERT INTO %s (k, v1, v2) " +
+                                                               "VALUES (%s, %s, %s)",
+                                                               KEYSPACE + "." + currentTable(),
+                                                               i, i, functionCall(functionName)));
+            functions.add(functionName);
+            statements.add(stmt);
+        }
+        BatchStatement batch = new BatchStatement(-1, BatchStatement.Type.LOGGED, statements, Attributes.none());
+        assertUnauthorized(batch, functions);
+
+        grantExecuteOnFunction(functions.get(0));
+        assertUnauthorized(batch, functions.subList(1, functions.size()));
+
+        grantExecuteOnFunction(functions.get(1));
+        assertUnauthorized(batch, functions.subList(2, functions.size()));
+
+        grantExecuteOnFunction(functions.get(2));
+        batch.checkAccess(clientState);
+    }
+
+    @Test
+    public void testNestedFunctions() throws Throwable
+    {
+        String innerFunctionName = createSimpleFunction();
+        String outerFunctionName = createFunction("int",
+                                                  "CREATE FUNCTION %s(input int) " +
+                                                  " CALLED ON NULL INPUT" +
+                                                  " RETURNS int" +
+                                                  " LANGUAGE java" +
+                                                  " AS 'return Integer.valueOf(0);'");
+        assertPermissionsOnNestedFunctions(innerFunctionName, outerFunctionName);
+    }
+
+    @Test
+    public void functionInStaticColumnRestrictionInSelect() throws Throwable
+    {
+        setupTable("CREATE TABLE %s (k int, s int STATIC, v1 int, v2 int, PRIMARY KEY(k, v1))");
+        String functionName = createSimpleFunction();
+        String cql = String.format("SELECT k FROM %s WHERE k = 0 AND s = %s",
+                                   KEYSPACE + "." + currentTable(),
+                                   functionCall(functionName));
+        assertPermissionsOnFunction(cql, functionName);
+    }
+
+    @Test
+    public void functionInRegularCondition() throws Throwable
+    {
+        String functionName = createSimpleFunction();
+        String cql = String.format("UPDATE %s SET v2 = 0 WHERE k = 0 AND v1 = 0 IF v2 = %s",
+                                   KEYSPACE + "." + currentTable(),
+                                   functionCall(functionName));
+        assertPermissionsOnFunction(cql, functionName);
+    }
+    @Test
+    public void functionInStaticColumnCondition() throws Throwable
+    {
+        setupTable("CREATE TABLE %s (k int, s int STATIC, v1 int, v2 int, PRIMARY KEY(k, v1))");
+        String functionName = createSimpleFunction();
+        String cql = String.format("UPDATE %s SET v2 = 0 WHERE k = 0 AND v1 = 0 IF s = %s",
+                                   KEYSPACE + "." + currentTable(),
+                                   functionCall(functionName));
+        assertPermissionsOnFunction(cql, functionName);
+    }
+
+    @Test
+    public void functionInCollectionLiteralCondition() throws Throwable
+    {
+        setupTable("CREATE TABLE %s (k int, v1 int, m_val map<int, int>, PRIMARY KEY(k))");
+        String functionName = createSimpleFunction();
+        String cql = String.format("UPDATE %s SET v1 = 0 WHERE k = 0 IF m_val = {%s : %s}",
+                                   KEYSPACE + "." + currentTable(),
+                                   functionCall(functionName),
+                                   functionCall(functionName));
+        assertPermissionsOnFunction(cql, functionName);
+    }
+
+    @Test
+    public void functionInCollectionElementCondition() throws Throwable
+    {
+        setupTable("CREATE TABLE %s (k int, v1 int, m_val map<int, int>, PRIMARY KEY(k))");
+        String functionName = createSimpleFunction();
+        String cql = String.format("UPDATE %s SET v1 = 0 WHERE k = 0 IF m_val[%s] = %s",
+                                   KEYSPACE + "." + currentTable(),
+                                   functionCall(functionName),
+                                   functionCall(functionName));
+        assertPermissionsOnFunction(cql, functionName);
+    }
+
+    @Test
+    public void systemFunctionsRequireNoExplicitPrivileges() throws Throwable
+    {
+        // with terminal arguments, so evaluated at prepare time
+        String cql = String.format("UPDATE %s SET v2 = 0 WHERE k = blobasint(intasblob(0))",
+                                   KEYSPACE + "." + currentTable());
+        getStatement(cql).checkAccess(clientState);
+
+        // with non-terminal arguments, so evaluated at execution
+        String functionName = createSimpleFunction();
+        grantExecuteOnFunction(functionName);
+        cql = String.format("UPDATE %s SET v2 = 0 WHERE k = blobasint(intasblob(%s))",
+                            KEYSPACE + "." + currentTable(),
+                            functionCall(functionName));
+        getStatement(cql).checkAccess(clientState);
+    }
+
+    @Test
+    public void requireExecutePermissionOnComponentFunctionsWhenDefiningAggregate() throws Throwable
+    {
+        String sFunc = createSimpleStateFunction();
+        String fFunc = createSimpleFinalFunction();
+        // aside from the component functions, we need CREATE on the keyspace's functions
+        DatabaseDescriptor.getAuthorizer().grant(AuthenticatedUser.SYSTEM_USER,
+                                                 ImmutableSet.of(Permission.CREATE),
+                                                 FunctionResource.keyspace(KEYSPACE),
+                                                 role);
+        String aggDef = String.format(aggregateCql(sFunc, fFunc),
+                                      KEYSPACE + ".aggregate_for_permissions_test");
+
+        assertUnauthorized(aggDef, sFunc, "int, int");
+        grantExecuteOnFunction(sFunc);
+
+        assertUnauthorized(aggDef, fFunc, "int");
+        grantExecuteOnFunction(fFunc);
+
+        getStatement(aggDef).checkAccess(clientState);
+    }
+
+    @Test
+    public void revokeExecutePermissionsOnAggregateComponents() throws Throwable
+    {
+        String sFunc = createSimpleStateFunction();
+        String fFunc = createSimpleFinalFunction();
+        String aggDef = aggregateCql(sFunc, fFunc);
+        grantExecuteOnFunction(sFunc);
+        grantExecuteOnFunction(fFunc);
+
+        String aggregate = createAggregate(KEYSPACE, "int", aggDef);
+        grantExecuteOnFunction(aggregate);
+
+        String cql = String.format("SELECT %s(v1) FROM %s",
+                                   aggregate,
+                                   KEYSPACE + "." + currentTable());
+        getStatement(cql).checkAccess(clientState);
+
+        // check that revoking EXECUTE permission on any one of the
+        // component functions means we lose the ability to execute it
+        revokeExecuteOnFunction(aggregate);
+        assertUnauthorized(cql, aggregate, "int");
+        grantExecuteOnFunction(aggregate);
+        getStatement(cql).checkAccess(clientState);
+
+        revokeExecuteOnFunction(sFunc);
+        assertUnauthorized(cql, sFunc, "int, int");
+        grantExecuteOnFunction(sFunc);
+        getStatement(cql).checkAccess(clientState);
+
+        revokeExecuteOnFunction(fFunc);
+        assertUnauthorized(cql, fFunc, "int");
+        grantExecuteOnFunction(fFunc);
+        getStatement(cql).checkAccess(clientState);
+    }
+
+    @Test
+    public void functionWrappingAggregate() throws Throwable
+    {
+        String outerFunc = createFunction("int",
+                                          "CREATE FUNCTION %s(input int) " +
+                                          "CALLED ON NULL INPUT " +
+                                          "RETURNS int " +
+                                          "LANGUAGE java " +
+                                          "AS 'return input;'");
+
+        String sFunc = createSimpleStateFunction();
+        String fFunc = createSimpleFinalFunction();
+        String aggDef = aggregateCql(sFunc, fFunc);
+        grantExecuteOnFunction(sFunc);
+        grantExecuteOnFunction(fFunc);
+
+        String aggregate = createAggregate(KEYSPACE, "int", aggDef);
+
+        String cql = String.format("SELECT %s(%s(v1)) FROM %s",
+                                   outerFunc,
+                                   aggregate,
+                                   KEYSPACE + "." + currentTable());
+
+        assertUnauthorized(cql, outerFunc, "int");
+        grantExecuteOnFunction(outerFunc);
+
+        assertUnauthorized(cql, aggregate, "int");
+        grantExecuteOnFunction(aggregate);
+
+        getStatement(cql).checkAccess(clientState);
+    }
+
+    @Test
+    public void aggregateWrappingFunction() throws Throwable
+    {
+        String innerFunc = createFunction("int",
+                                          "CREATE FUNCTION %s(input int) " +
+                                          "CALLED ON NULL INPUT " +
+                                          "RETURNS int " +
+                                          "LANGUAGE java " +
+                                          "AS 'return input;'");
+
+        String sFunc = createSimpleStateFunction();
+        String fFunc = createSimpleFinalFunction();
+        String aggDef = aggregateCql(sFunc, fFunc);
+        grantExecuteOnFunction(sFunc);
+        grantExecuteOnFunction(fFunc);
+
+        String aggregate = createAggregate(KEYSPACE, "int", aggDef);
+
+        String cql = String.format("SELECT %s(%s(v1)) FROM %s",
+                                   aggregate,
+                                   innerFunc,
+                                   KEYSPACE + "." + currentTable());
+
+        assertUnauthorized(cql, aggregate, "int");
+        grantExecuteOnFunction(aggregate);
+
+        assertUnauthorized(cql, innerFunc, "int");
+        grantExecuteOnFunction(innerFunc);
+
+        getStatement(cql).checkAccess(clientState);
+    }
+
+    private void assertPermissionsOnNestedFunctions(String innerFunction, String outerFunction) throws Throwable
+    {
+        String cql = String.format("SELECT k, %s FROM %s WHERE k=0",
+                                   functionCall(outerFunction, functionCall(innerFunction)),
+                                   KEYSPACE + "." + currentTable());
+        // fail fast with an UAE on the first function
+        assertUnauthorized(cql, outerFunction, "int");
+        grantExecuteOnFunction(outerFunction);
+
+        // after granting execute on the first function, still fail due to the inner function
+        assertUnauthorized(cql, innerFunction, "");
+        grantExecuteOnFunction(innerFunction);
+
+        // now execution of both is permitted
+        getStatement(cql).checkAccess(clientState);
+    }
+
+    private void assertPermissionsOnFunction(String cql, String functionName) throws Throwable
+    {
+        assertPermissionsOnFunction(cql, functionName, "");
+    }
+
+    private void assertPermissionsOnFunction(String cql, String functionName, String argTypes) throws Throwable
+    {
+        assertUnauthorized(cql, functionName, argTypes);
+        grantExecuteOnFunction(functionName);
+        getStatement(cql).checkAccess(clientState);
+    }
+
+    private void assertUnauthorized(BatchStatement batch, Iterable<String> functionNames) throws Throwable
+    {
+        try
+        {
+            batch.checkAccess(clientState);
+            fail("Expected an UnauthorizedException, but none was thrown");
+        }
+        catch (UnauthorizedException e)
+        {
+            String functions = String.format("(%s)", Joiner.on("|").join(functionNames));
+            assertTrue(e.getLocalizedMessage()
+                        .matches(String.format("User %s has no EXECUTE permission on <function %s\\(\\)> or any of its parents",
+                                               roleName,
+                                               functions)));
+        }
+    }
+
+    private void assertUnauthorized(String cql, String functionName, String argTypes) throws Throwable
+    {
+        try
+        {
+            getStatement(cql).checkAccess(clientState);
+            fail("Expected an UnauthorizedException, but none was thrown");
+        }
+        catch (UnauthorizedException e)
+        {
+            assertEquals(String.format("User %s has no EXECUTE permission on <function %s(%s)> or any of its parents",
+                                       roleName,
+                                       functionName,
+                                       argTypes),
+                         e.getLocalizedMessage());
+        }
+    }
+
+    private void grantExecuteOnFunction(String functionName)
+    {
+            DatabaseDescriptor.getAuthorizer().grant(AuthenticatedUser.SYSTEM_USER,
+                                                     ImmutableSet.of(Permission.EXECUTE),
+                                                     functionResource(functionName),
+                                                     role);
+    }
+
+    private void revokeExecuteOnFunction(String functionName)
+    {
+        DatabaseDescriptor.getAuthorizer().revoke(AuthenticatedUser.SYSTEM_USER,
+                                                  ImmutableSet.of(Permission.EXECUTE),
+                                                  functionResource(functionName),
+                                                  role);
+    }
+
+    void setupClientState()
+    {
+
+        try
+        {
+            role = RoleResource.role(roleName);
+            // use reflection to set the logged in user so that we don't need to
+            // bother setting up an IRoleManager
+            user = new AuthenticatedUser(roleName);
+            clientState = ClientState.forInternalCalls();
+            Field userField = ClientState.class.getDeclaredField("user");
+            userField.setAccessible(true);
+            userField.set(clientState, user);
+        }
+        catch (IllegalAccessException | NoSuchFieldException e)
+        {
+            throw new RuntimeException(e);
+        }
+    }
+
+    private void setupTable(String tableDef) throws Throwable
+    {
+        createTable(tableDef);
+        // test user needs SELECT & MODIFY on the table regardless of permissions on any function
+        DatabaseDescriptor.getAuthorizer().grant(AuthenticatedUser.SYSTEM_USER,
+                                                 ImmutableSet.of(Permission.SELECT, Permission.MODIFY),
+                                                 DataResource.table(KEYSPACE, currentTable()),
+                                                 RoleResource.role(user.getName()));
+    }
+
+    private String aggregateCql(String sFunc, String fFunc)
+    {
+        return "CREATE AGGREGATE %s(int) " +
+               "SFUNC " + shortFunctionName(sFunc) + " " +
+               "STYPE int " +
+               "FINALFUNC " + shortFunctionName(fFunc) + " " +
+               "INITCOND 0";
+    }
+
+    private String createSimpleStateFunction() throws Throwable
+    {
+        return createFunction("int, int",
+                              "CREATE FUNCTION %s(a int, b int) " +
+                              "CALLED ON NULL INPUT " +
+                              "RETURNS int " +
+                              "LANGUAGE java " +
+                              "AS 'return Integer.valueOf( (a != null ? a.intValue() : 0 ) + b.intValue());'");
+    }
+
+    private String createSimpleFinalFunction() throws Throwable
+    {
+        return createFunction("int",
+                              "CREATE FUNCTION %s(a int) " +
+                              "CALLED ON NULL INPUT " +
+                              "RETURNS int " +
+                              "LANGUAGE java " +
+                              "AS 'return a;'");
+    }
+
+    private String createSimpleFunction() throws Throwable
+    {
+        return createFunction("",
+                              "CREATE FUNCTION %s() " +
+                              "  CALLED ON NULL INPUT " +
+                              "  RETURNS int " +
+                              "  LANGUAGE java " +
+                              "  AS 'return Integer.valueOf(0);'");
+    }
+
+    private String createFunction(String argTypes, String functionDef) throws Throwable
+    {
+        return createFunction(KEYSPACE, argTypes, functionDef);
+    }
+
+    private CQLStatement getStatement(String cql)
+    {
+        return QueryProcessor.getStatement(cql, clientState).statement;
+    }
+
+    private FunctionResource functionResource(String functionName)
+    {
+        // Note that this is somewhat brittle as it assumes that function names are
+        // truly unique. As such, it will break in the face of overloading.
+        // It is here to avoid having to duplicate the functionality of CqlParser
+        // for transforming cql types into AbstractTypes
+        FunctionName fn = parseFunctionName(functionName);
+        List<Function> functions = Functions.find(fn);
+        assertEquals(String.format("Expected a single function definition for %s, but found %s",
+                                   functionName,
+                                   functions.size()),
+                     1, functions.size());
+        return FunctionResource.function(fn.keyspace, fn.name, functions.get(0).argTypes());
+    }
+
+    private String functionCall(String functionName, String...args)
+    {
+        return String.format("%s(%s)", functionName, Joiner.on(",").join(args));
+    }
+
+    static class StubAuthorizer implements IAuthorizer
+    {
+        Map<Pair<String, IResource>, Set<Permission>> userPermissions = new HashMap<>();
+
+        private void clear()
+        {
+            userPermissions.clear();
+        }
+
+        public Set<Permission> authorize(AuthenticatedUser user, IResource resource)
+        {
+            Pair<String, IResource> key = Pair.create(user.getName(), resource);
+            Set<Permission> perms = userPermissions.get(key);
+            return perms != null ? perms : Collections.<Permission>emptySet();
+        }
+
+        public void grant(AuthenticatedUser performer,
+                          Set<Permission> permissions,
+                          IResource resource,
+                          RoleResource grantee) throws RequestValidationException, RequestExecutionException
+        {
+            Pair<String, IResource> key = Pair.create(grantee.getRoleName(), resource);
+            Set<Permission> perms = userPermissions.get(key);
+            if (null == perms)
+            {
+                perms = new HashSet<>();
+                userPermissions.put(key, perms);
+            }
+            perms.addAll(permissions);
+        }
+
+        public void revoke(AuthenticatedUser performer,
+                           Set<Permission> permissions,
+                           IResource resource,
+                           RoleResource revokee) throws RequestValidationException, RequestExecutionException
+        {
+            Pair<String, IResource> key = Pair.create(revokee.getRoleName(), resource);
+            Set<Permission> perms = userPermissions.get(key);
+            if (null != perms)
+                perms.removeAll(permissions);
+            if (perms.isEmpty())
+                userPermissions.remove(key);
+        }
+
+        public Set<PermissionDetails> list(AuthenticatedUser performer,
+                                           Set<Permission> permissions,
+                                           IResource resource,
+                                           RoleResource grantee) throws RequestValidationException, RequestExecutionException
+        {
+            Pair<String, IResource> key = Pair.create(grantee.getRoleName(), resource);
+            Set<Permission> perms = userPermissions.get(key);
+            if (perms == null)
+                return Collections.emptySet();
+
+
+            Set<PermissionDetails> details = new HashSet<>();
+            for (Permission permission : perms)
+            {
+                if (permissions.contains(permission))
+                    details.add(new PermissionDetails(grantee.getRoleName(), resource, permission));
+            }
+            return details;
+        }
+
+        public void revokeAllFrom(RoleResource revokee)
+        {
+            for (Pair<String, IResource> key : userPermissions.keySet())
+                if (key.left.equals(revokee.getRoleName()))
+                    userPermissions.remove(key);
+        }
+
+        public void revokeAllOn(IResource droppedResource)
+        {
+            for (Pair<String, IResource> key : userPermissions.keySet())
+                if (key.right.equals(droppedResource))
+                    userPermissions.remove(key);
+
+        }
+
+        public Set<? extends IResource> protectedResources()
+        {
+            return Collections.emptySet();
+        }
+
+        public void validateConfiguration() throws ConfigurationException
+        {
+
+        }
+
+        public void setup()
+        {
+
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/01115f72/test/unit/org/apache/cassandra/cql3/validation/entities/UFIdentificationTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/UFIdentificationTest.java b/test/unit/org/apache/cassandra/cql3/validation/entities/UFIdentificationTest.java
new file mode 100644
index 0000000..28b8afc
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/UFIdentificationTest.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.validation.entities;
+
+import java.util.*;
+
+import com.google.common.base.Joiner;
+import com.google.common.collect.Iterables;
+import org.junit.Before;
+import org.junit.Ignore;
+import org.junit.Test;
+
+import org.apache.cassandra.cql3.Attributes;
+import org.apache.cassandra.cql3.CQLStatement;
+import org.apache.cassandra.cql3.QueryProcessor;
+import org.apache.cassandra.cql3.functions.Function;
+import org.apache.cassandra.cql3.statements.BatchStatement;
+import org.apache.cassandra.cql3.statements.ModificationStatement;
+import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.service.ClientState;
+
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Checks the collection of Function objects returned by CQLStatement.getFunction
+ * matches expectations. This is intended to verify the various subcomponents of
+ * the statement (Operations, Terms, Restrictions, RestrictionSet, Selection,
+ * Selector, SelectorFactories etc) properly report any constituent functions.
+ * Some purely terminal functions are resolved at preparation, so those are not
+ * included in the reported list. They still need to be surveyed, to verify the
+ * calling client has the necessary permissions. UFAuthTest includes tests which
+ * verify this more thoroughly than we can here.
+ */
+public class UFIdentificationTest extends CQLTester
+{
+    private com.google.common.base.Function<Function, String> toFunctionNames = new com.google.common.base.Function<Function, String>()
+    {
+        public String apply(Function f)
+        {
+            return f.name().keyspace + "." + f.name().name;
+        }
+    };
+
+    String tFunc;
+    String iFunc;
+    String lFunc;
+    String sFunc;
+    String mFunc;
+    String uFunc;
+    String udtFunc;
+
+    String userType;
+
+    @Before
+    public void setup() throws Throwable
+    {
+        userType = KEYSPACE + "." + createType("CREATE TYPE %s (t text, i int)");
+
+        createTable("CREATE TABLE %s (" +
+                    "   key int, " +
+                    "   t_sc text STATIC," +
+                    "   i_cc int, " +
+                    "   t_cc text, " +
+                    "   i_val int," +
+                    "   l_val list<int>," +
+                    "   s_val set<int>," +
+                    "   m_val map<int, int>," +
+                    "   u_val timeuuid," +
+                    "   udt_val frozen<" + userType + ">," +
+                    "   PRIMARY KEY (key, i_cc, t_cc)" +
+                    ")");
+
+        tFunc = createEchoFunction("text");
+        iFunc = createEchoFunction("int");
+        lFunc = createEchoFunction("list<int>");
+        sFunc = createEchoFunction("set<int>");
+        mFunc = createEchoFunction("map<int, int>");
+        uFunc = createEchoFunction("timeuuid");
+        udtFunc = createEchoFunction(userType);
+    }
+
+    @Test
+    public void testSimpleModificationStatement() throws Throwable
+    {
+        assertFunctions(cql("INSERT INTO %s (key, t_sc) VALUES (0, %s)", functionCall(tFunc, "'foo'")), tFunc);
+        assertFunctions(cql("INSERT INTO %s (key, i_cc) VALUES (0, %s)", functionCall(iFunc, "1")), iFunc);
+        assertFunctions(cql("INSERT INTO %s (key, t_cc) VALUES (0, %s)", functionCall(tFunc, "'foo'")), tFunc);
+        assertFunctions(cql("INSERT INTO %s (key, i_val) VALUES (0, %s)", functionCall(iFunc, "1")), iFunc);
+        assertFunctions(cql("INSERT INTO %s (key, l_val) VALUES (0, %s)", functionCall(lFunc, "[1]")), lFunc);
+        assertFunctions(cql("INSERT INTO %s (key, s_val) VALUES (0, %s)", functionCall(sFunc, "{1}")), sFunc);
+        assertFunctions(cql("INSERT INTO %s (key, m_val) VALUES (0, %s)", functionCall(mFunc, "{1:1}")), mFunc);
+        assertFunctions(cql("INSERT INTO %s (key, udt_val) VALUES (0,%s)", functionCall(udtFunc, "{i : 1, t : 'foo'}")), udtFunc);
+        assertFunctions(cql("INSERT INTO %s (key, u_val) VALUES (0, %s)", functionCall(uFunc, "now()")), uFunc, "system.now");
+    }
+
+    @Test
+    public void testNonTerminalCollectionLiterals() throws Throwable
+    {
+        String iFunc2 = createEchoFunction("int");
+        String mapValue = String.format("{%s:%s}", functionCall(iFunc, "1"), functionCall(iFunc2, "1"));
+        assertFunctions(cql("INSERT INTO %s (key, m_val) VALUES (0, %s)", mapValue), iFunc, iFunc2);
+
+        String listValue = String.format("[%s]", functionCall(iFunc, "1"));
+        assertFunctions(cql("INSERT INTO %s (key, l_val) VALUES (0, %s)", listValue), iFunc);
+
+        String setValue = String.format("{%s}", functionCall(iFunc, "1"));
+        assertFunctions(cql("INSERT INTO %s (key, s_val) VALUES (0, %s)", setValue), iFunc);
+    }
+
+    @Test
+    public void testNonTerminalUDTLiterals() throws Throwable
+    {
+        String udtValue = String.format("{ i: %s, t : %s } ", functionCall(iFunc, "1"), functionCall(tFunc, "'foo'"));
+        assertFunctions(cql("INSERT INTO %s (key, udt_val) VALUES (0, %s)", udtValue), iFunc, tFunc);
+    }
+
+    @Test
+    public void testModificationStatementWithConditions() throws Throwable
+    {
+        assertFunctions(cql("UPDATE %s SET i_val=0 WHERE key=0 IF t_sc=%s", functionCall(tFunc, "'foo'")), tFunc);
+        assertFunctions(cql("UPDATE %s SET i_val=0 WHERE key=0 IF i_val=%s", functionCall(iFunc, "1")), iFunc);
+        assertFunctions(cql("UPDATE %s SET i_val=0 WHERE key=0 IF l_val=%s", functionCall(lFunc, "[1]")), lFunc);
+        assertFunctions(cql("UPDATE %s SET i_val=0 WHERE key=0 IF s_val=%s", functionCall(sFunc, "{1}")), sFunc);
+        assertFunctions(cql("UPDATE %s SET i_val=0 WHERE key=0 IF m_val=%s", functionCall(mFunc, "{1:1}")), mFunc);
+
+
+        String iFunc2 = createEchoFunction("int");
+        assertFunctions(cql("UPDATE %s SET i_val=0 WHERE key=0 IF i_val IN (%s, %S)",
+                            functionCall(iFunc, "1"),
+                            functionCall(iFunc2, "2")),
+                        iFunc, iFunc2);
+
+        assertFunctions(cql("UPDATE %s SET i_val=0 WHERE key=0 IF u_val=%s",
+                            functionCall(uFunc, "now()")),
+                        uFunc, "system.now");
+
+        // conditions on collection elements
+        assertFunctions(cql("UPDATE %s SET i_val=0 WHERE key=0 IF l_val[%s] = %s",
+                            functionCall(iFunc, "1"),
+                            functionCall(iFunc2, "1")),
+                        iFunc, iFunc2);
+        assertFunctions(cql("UPDATE %s SET i_val=0 WHERE key=0 IF m_val[%s] = %s",
+                            functionCall(iFunc, "1"),
+                            functionCall(iFunc2, "1")),
+                        iFunc, iFunc2);
+    }
+
+    @Test @Ignore
+    // Technically, attributes like timestamp and ttl are Terms so could potentially
+    // resolve to function calls (& so you can call getFunctions on them)
+    // However, this is currently disallowed by CQL syntax
+    public void testModificationStatementWithAttributesFromFunction() throws Throwable
+    {
+        String longFunc = createEchoFunction("bigint");
+        assertFunctions(cql("INSERT INTO %s (key, i_cc, t_cc, i_val) VALUES (0, 0, 'foo', 0) USING TIMESTAMP %s",
+                            functionCall(longFunc, "9999")),
+                        longFunc);
+
+        assertFunctions(cql("INSERT INTO %s (key, i_cc, t_cc, i_val) VALUES (0, 0, 'foo', 0) USING TTL %s",
+                            functionCall(iFunc, "8888")),
+                        iFunc);
+
+        assertFunctions(cql("INSERT INTO %s (key, i_cc, t_cc, i_val) VALUES (0, 0, 'foo', 0) USING TIMESTAMP %s AND TTL %s",
+                            functionCall(longFunc, "9999"), functionCall(iFunc, "8888")),
+                        longFunc, iFunc);
+    }
+
+    @Test
+    public void testModificationStatementWithNestedFunctions() throws Throwable
+    {
+        String iFunc2 = createEchoFunction("int");
+        String iFunc3 = createEchoFunction("int");
+        String iFunc4 = createEchoFunction("int");
+        String iFunc5 = createEchoFunction("int");
+        String iFunc6 = createEchoFunction("int");
+        String nestedFunctionCall = nestedFunctionCall(iFunc6, iFunc5,
+                                                       nestedFunctionCall(iFunc4, iFunc3,
+                                                                          nestedFunctionCall(iFunc2, iFunc, "1")));
+
+        assertFunctions(cql("DELETE FROM %s WHERE key=%s", nestedFunctionCall),
+                        iFunc, iFunc2, iFunc3, iFunc4, iFunc5, iFunc6);
+    }
+
+    @Test
+    public void testSelectStatementSimpleRestrictions() throws Throwable
+    {
+        assertFunctions(cql("SELECT i_val FROM %s WHERE key=%s", functionCall(iFunc, "1")), iFunc);
+        assertFunctions(cql("SELECT i_val FROM %s WHERE key=0 AND t_sc=%s", functionCall(tFunc, "'foo'")), tFunc);
+        assertFunctions(cql("SELECT i_val FROM %s WHERE key=0 AND i_cc=%s AND t_cc='foo'", functionCall(iFunc, "1")), iFunc);
+        assertFunctions(cql("SELECT i_val FROM %s WHERE key=0 AND i_cc=0 AND t_cc=%s", functionCall(tFunc, "'foo'")), tFunc);
+
+        String iFunc2 = createEchoFunction("int");
+        String tFunc2 = createEchoFunction("text");
+        assertFunctions(cql("SELECT i_val FROM %s WHERE key=%s AND t_sc=%s AND i_cc=%s AND t_cc=%s",
+                            functionCall(iFunc, "1"),
+                            functionCall(tFunc, "'foo'"),
+                            functionCall(iFunc2, "1"),
+                            functionCall(tFunc2, "'foo'")),
+                        iFunc, tFunc, iFunc2, tFunc2);
+    }
+
+    @Test
+    public void testSelectStatementRestrictionsWithNestedFunctions() throws Throwable
+    {
+        String iFunc2 = createEchoFunction("int");
+        String iFunc3 = createEchoFunction("int");
+        String iFunc4 = createEchoFunction("int");
+        String iFunc5 = createEchoFunction("int");
+        String iFunc6 = createEchoFunction("int");
+        String nestedFunctionCall = nestedFunctionCall(iFunc6, iFunc5,
+                                                       nestedFunctionCall(iFunc3, iFunc4,
+                                                                          nestedFunctionCall(iFunc, iFunc2, "1")));
+
+        assertFunctions(cql("SELECT i_val FROM %s WHERE key=%s", nestedFunctionCall),
+                        iFunc, iFunc2, iFunc3, iFunc4, iFunc5, iFunc6);
+    }
+
+    @Test
+    public void testNonTerminalTupleInSelectRestrictions() throws Throwable
+    {
+        assertFunctions(cql("SELECT i_val FROM %s WHERE key=0 AND (i_cc, t_cc) IN ((%s, %s))",
+                            functionCall(iFunc, "1"),
+                            functionCall(tFunc, "'foo'")),
+                        iFunc, tFunc);
+
+        assertFunctions(cql("SELECT i_val FROM %s WHERE key=0 AND (i_cc, t_cc) = (%s, %s)",
+                            functionCall(iFunc, "1"),
+                            functionCall(tFunc, "'foo'")),
+                        iFunc, tFunc);
+
+        assertFunctions(cql("SELECT i_val FROM %s WHERE key=0 AND (i_cc, t_cc) > (%s, %s)",
+                            functionCall(iFunc, "1"),
+                            functionCall(tFunc, "'foo'")),
+                        iFunc, tFunc);
+
+        assertFunctions(cql("SELECT i_val FROM %s WHERE key=0 AND (i_cc, t_cc) < (%s, %s)",
+                            functionCall(iFunc, "1"),
+                            functionCall(tFunc, "'foo'")),
+                        iFunc, tFunc);
+
+         assertFunctions(cql("SELECT i_val FROM %s WHERE key=0 AND (i_cc, t_cc) > (%s, %s) AND (i_cc, t_cc) < (%s, %s)",
+                            functionCall(iFunc, "1"),
+                            functionCall(tFunc, "'foo'"),
+                            functionCall(iFunc, "1"),
+                            functionCall(tFunc, "'foo'")),
+                         iFunc, tFunc);
+    }
+
+    @Test
+    public void testNestedFunctionInTokenRestriction() throws Throwable
+    {
+        String iFunc2 = createEchoFunction("int");
+        assertFunctions(cql("SELECT i_val FROM %s WHERE token(key) = token(%s)", functionCall(iFunc, "1")),
+                        "system.token", iFunc);
+        assertFunctions(cql("SELECT i_val FROM %s WHERE token(key) > token(%s)", functionCall(iFunc, "1")),
+                        "system.token", iFunc);
+        assertFunctions(cql("SELECT i_val FROM %s WHERE token(key) < token(%s)", functionCall(iFunc, "1")),
+                        "system.token", iFunc);
+        assertFunctions(cql("SELECT i_val FROM %s WHERE token(key) > token(%s) AND token(key) < token(%s)",
+                            functionCall(iFunc, "1"),
+                            functionCall(iFunc2, "1")),
+                        "system.token", iFunc, iFunc2);
+    }
+
+    @Test
+    public void testSelectStatementSimpleSelections() throws Throwable
+    {
+        String iFunc2 = createEchoFunction("int");
+        execute("INSERT INTO %s (key, i_cc, t_cc, i_val) VALUES (0, 0, 'foo', 0)");
+        assertFunctions(cql2("SELECT i_val, %s FROM %s WHERE key=0", functionCall(iFunc, "i_val")), iFunc);
+        assertFunctions(cql2("SELECT i_val, %s FROM %s WHERE key=0", nestedFunctionCall(iFunc, iFunc2, "i_val")), iFunc, iFunc2);
+    }
+
+    @Test
+    public void testSelectStatementNestedSelections() throws Throwable
+    {
+        String iFunc2 = createEchoFunction("int");
+        execute("INSERT INTO %s (key, i_cc, t_cc, i_val) VALUES (0, 0, 'foo', 0)");
+        assertFunctions(cql2("SELECT i_val, %s FROM %s WHERE key=0", functionCall(iFunc, "i_val")), iFunc);
+        assertFunctions(cql2("SELECT i_val, %s FROM %s WHERE key=0", nestedFunctionCall(iFunc, iFunc2, "i_val")), iFunc, iFunc2);
+    }
+
+    @Test
+    public void testBatchStatement() throws Throwable
+    {
+        String iFunc2 = createEchoFunction("int");
+        List<ModificationStatement> statements = new ArrayList<>();
+        statements.add(modificationStatement(cql("INSERT INTO %s (key, i_cc, t_cc) VALUES (%s, 0, 'foo')",
+                                                 functionCall(iFunc, "0"))));
+        statements.add(modificationStatement(cql("INSERT INTO %s (key, i_cc, t_cc) VALUES (1, %s, 'foo')",
+                                                 functionCall(iFunc2, "1"))));
+        statements.add(modificationStatement(cql("INSERT INTO %s (key, i_cc, t_cc) VALUES (2, 2, %s)",
+                                                 functionCall(tFunc, "'foo'"))));
+
+        BatchStatement batch = new BatchStatement(-1, BatchStatement.Type.LOGGED, statements, Attributes.none());
+        assertFunctions(batch, iFunc, iFunc2, tFunc);
+    }
+
+    @Test
+    public void testBatchStatementWithConditions() throws Throwable
+    {
+        List<ModificationStatement> statements = new ArrayList<>();
+        statements.add(modificationStatement(cql("UPDATE %s SET i_val = %s WHERE key=0 AND i_cc=0 and t_cc='foo' IF l_val = %s",
+                                                 functionCall(iFunc, "0"), functionCall(lFunc, "[1]"))));
+        statements.add(modificationStatement(cql("UPDATE %s SET i_val = %s WHERE key=0 AND i_cc=1 and t_cc='foo' IF s_val = %s",
+                                                 functionCall(iFunc, "0"), functionCall(sFunc, "{1}"))));
+
+        BatchStatement batch = new BatchStatement(-1, BatchStatement.Type.LOGGED, statements, Attributes.none());
+        assertFunctions(batch, iFunc, lFunc, sFunc);
+    }
+
+    private ModificationStatement modificationStatement(String cql)
+    {
+        return (ModificationStatement) QueryProcessor.getStatement(cql, ClientState.forInternalCalls()).statement;
+    }
+
+    private void assertFunctions(String cql, String... function)
+    {
+        CQLStatement stmt = QueryProcessor.getStatement(cql, ClientState.forInternalCalls()).statement;
+        assertFunctions(stmt, function);
+    }
+
+    private void assertFunctions(CQLStatement stmt, String... function)
+    {
+        Set<String> expected = com.google.common.collect.Sets.newHashSet(function);
+        Set<String> actual = com.google.common.collect.Sets.newHashSet(Iterables.transform(stmt.getFunctions(),
+                                                                                           toFunctionNames));
+        assertTrue(com.google.common.collect.Sets.symmetricDifference(expected, actual).isEmpty());
+    }
+
+    private String cql(String template, String... params)
+    {
+        String tableName = KEYSPACE + "." + currentTable();
+        return String.format(template, com.google.common.collect.Lists.asList(tableName, params).toArray());
+    }
+
+    // Alternative query builder - appends the table name to the supplied params,
+    // for stmts of the form "SELECT x, %s FROM %s WHERE y=0"
+    private String cql2(String template, String... params)
+    {
+        Object[] args = Arrays.copyOf(params, params.length + 1);
+        args[params.length] = KEYSPACE + "." + currentTable();
+        return String.format(template, args);
+    }
+
+    private String functionCall(String fName, String... args)
+    {
+        return String.format("%s(%s)", fName, Joiner.on(",").join(args));
+    }
+
+    private String nestedFunctionCall(String outer, String inner, String innerArgs)
+    {
+        return functionCall(outer, functionCall(inner, innerArgs));
+    }
+
+    private String createEchoFunction(String type) throws Throwable
+    {
+        return createFunction(KEYSPACE, type,
+           "CREATE FUNCTION %s(input " + type + ")" +
+           " CALLED ON NULL INPUT" +
+           " RETURNS " + type +
+           " LANGUAGE java" +
+           " AS ' return input;'");
+    }
+}


[21/32] cassandra git commit: 2.2 commit for CASSANDRA-9160

Posted by jm...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/01115f72/test/unit/org/apache/cassandra/cql3/validation/entities/FrozenCollectionsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/FrozenCollectionsTest.java b/test/unit/org/apache/cassandra/cql3/validation/entities/FrozenCollectionsTest.java
new file mode 100644
index 0000000..857139d
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/FrozenCollectionsTest.java
@@ -0,0 +1,1111 @@
+/*
+ * 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.validation.entities;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.db.marshal.*;
+import org.apache.cassandra.dht.ByteOrderedPartitioner;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.exceptions.SyntaxException;
+import org.apache.commons.lang3.StringUtils;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+
+public class FrozenCollectionsTest extends CQLTester
+{
+    @BeforeClass
+    public static void setUpClass()
+    {
+        DatabaseDescriptor.setPartitioner(new ByteOrderedPartitioner());
+    }
+
+    @Test
+    public void testPartitionKeyUsage() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k frozen<set<int>> PRIMARY KEY, v int)");
+
+        execute("INSERT INTO %s (k, v) VALUES (?, ?)", set(), 1);
+        execute("INSERT INTO %s (k, v) VALUES (?, ?)", set(1, 2, 3), 1);
+        execute("INSERT INTO %s (k, v) VALUES (?, ?)", set(4, 5, 6), 0);
+        execute("INSERT INTO %s (k, v) VALUES (?, ?)", set(7, 8, 9), 0);
+
+        // overwrite with an update
+        execute("UPDATE %s SET v=? WHERE k=?", 0, set());
+        execute("UPDATE %s SET v=? WHERE k=?", 0, set(1, 2, 3));
+
+        assertRows(execute("SELECT * FROM %s"),
+            row(set(), 0),
+            row(set(1, 2, 3), 0),
+            row(set(4, 5, 6), 0),
+            row(set(7, 8, 9), 0)
+        );
+
+        assertRows(execute("SELECT k FROM %s"),
+            row(set()),
+            row(set(1, 2, 3)),
+            row(set(4, 5, 6)),
+            row(set(7, 8, 9))
+        );
+
+        assertRows(execute("SELECT * FROM %s LIMIT 2"),
+                row(set(), 0),
+                row(set(1, 2, 3), 0)
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE k=?", set(4, 5, 6)),
+            row(set(4, 5, 6), 0)
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE k=?", set()),
+                row(set(), 0)
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE k IN ?", list(set(4, 5, 6), set())),
+                   row(set(), 0),
+                   row(set(4, 5, 6), 0)
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE token(k) >= token(?)", set(4, 5, 6)),
+                row(set(4, 5, 6), 0),
+                row(set(7, 8, 9), 0)
+        );
+
+        assertInvalid("INSERT INTO %s (k, v) VALUES (null, 0)");
+
+        execute("DELETE FROM %s WHERE k=?", set());
+        execute("DELETE FROM %s WHERE k=?", set(4, 5, 6));
+        assertRows(execute("SELECT * FROM %s"),
+            row(set(1, 2, 3), 0),
+            row(set(7, 8, 9), 0)
+        );
+    }
+
+    @Test
+    public void testNestedPartitionKeyUsage() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k frozen<map<set<int>, list<int>>> PRIMARY KEY, v int)");
+
+        execute("INSERT INTO %s (k, v) VALUES (?, ?)", map(), 1);
+        execute("INSERT INTO %s (k, v) VALUES (?, ?)", map(set(), list(1, 2, 3)), 0);
+        execute("INSERT INTO %s (k, v) VALUES (?, ?)", map(set(1, 2, 3), list(1, 2, 3)), 1);
+        execute("INSERT INTO %s (k, v) VALUES (?, ?)", map(set(4, 5, 6), list(1, 2, 3)), 0);
+        execute("INSERT INTO %s (k, v) VALUES (?, ?)", map(set(7, 8, 9), list(1, 2, 3)), 0);
+
+        // overwrite with an update
+        execute("UPDATE %s SET v=? WHERE k=?", 0, map());
+        execute("UPDATE %s SET v=? WHERE k=?", 0, map(set(1, 2, 3), list(1, 2, 3)));
+
+        assertRows(execute("SELECT * FROM %s"),
+            row(map(), 0),
+            row(map(set(), list(1, 2, 3)), 0),
+            row(map(set(1, 2, 3), list(1, 2, 3)), 0),
+            row(map(set(4, 5, 6), list(1, 2, 3)), 0),
+            row(map(set(7, 8, 9), list(1, 2, 3)), 0)
+        );
+
+        assertRows(execute("SELECT k FROM %s"),
+            row(map()),
+            row(map(set(), list(1, 2, 3))),
+            row(map(set(1, 2, 3), list(1, 2, 3))),
+            row(map(set(4, 5, 6), list(1, 2, 3))),
+            row(map(set(7, 8, 9), list(1, 2, 3)))
+        );
+
+        assertRows(execute("SELECT * FROM %s LIMIT 3"),
+            row(map(), 0),
+            row(map(set(), list(1, 2, 3)), 0),
+            row(map(set(1, 2, 3), list(1, 2, 3)), 0)
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE k=?", map(set(4, 5, 6), list(1, 2, 3))),
+            row(map(set(4, 5, 6), list(1, 2, 3)), 0)
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE k=?", map()),
+                row(map(), 0)
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE k=?", map(set(), list(1, 2, 3))),
+                row(map(set(), list(1, 2, 3)), 0)
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE k IN ?", list(map(set(4, 5, 6), list(1, 2, 3)), map(), map(set(), list(1, 2, 3)))),
+                   row(map(), 0),
+                   row(map(set(), list(1, 2, 3)), 0),
+                   row(map(set(4, 5, 6), list(1, 2, 3)), 0)
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE token(k) >= token(?)", map(set(4, 5, 6), list(1, 2, 3))),
+            row(map(set(4, 5, 6), list(1, 2, 3)), 0),
+            row(map(set(7, 8, 9), list(1, 2, 3)), 0)
+        );
+
+        execute("DELETE FROM %s WHERE k=?", map());
+        execute("DELETE FROM %s WHERE k=?", map(set(), list(1, 2, 3)));
+        execute("DELETE FROM %s WHERE k=?", map(set(4, 5, 6), list(1, 2, 3)));
+        assertRows(execute("SELECT * FROM %s"),
+            row(map(set(1, 2, 3), list(1, 2, 3)), 0),
+            row(map(set(7, 8, 9), list(1, 2, 3)), 0)
+        );
+
+    }
+
+    @Test
+    public void testClusteringKeyUsage() throws Throwable
+    {
+        for (String option : Arrays.asList("", " WITH COMPACT STORAGE"))
+        {
+            createTable("CREATE TABLE %s (a int, b frozen<set<int>>, c int, PRIMARY KEY (a, b))" + option);
+
+            execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, set(), 1);
+            execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, set(1, 2, 3), 1);
+            execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, set(4, 5, 6), 0);
+            execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, set(7, 8, 9), 0);
+
+            // overwrite with an update
+            execute("UPDATE %s SET c=? WHERE a=? AND b=?", 0, 0, set());
+            execute("UPDATE %s SET c=? WHERE a=? AND b=?", 0, 0, set(1, 2, 3));
+
+            assertRows(execute("SELECT * FROM %s"),
+                row(0, set(), 0),
+                row(0, set(1, 2, 3), 0),
+                row(0, set(4, 5, 6), 0),
+                row(0, set(7, 8, 9), 0)
+            );
+
+            assertRows(execute("SELECT b FROM %s"),
+                row(set()),
+                row(set(1, 2, 3)),
+                row(set(4, 5, 6)),
+                row(set(7, 8, 9))
+            );
+
+            assertRows(execute("SELECT * FROM %s LIMIT 2"),
+                row(0, set(), 0),
+                row(0, set(1, 2, 3), 0)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a=? AND b=?", 0, set(4, 5, 6)),
+                row(0, set(4, 5, 6), 0)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a=? AND b=?", 0, set()),
+                row(0, set(), 0)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a=? AND b IN ?", 0, list(set(4, 5, 6), set())),
+                row(0, set(), 0),
+                row(0, set(4, 5, 6), 0)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a=? AND b > ?", 0, set(4, 5, 6)),
+                row(0, set(7, 8, 9), 0)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a=? AND b >= ?", 0, set(4, 5, 6)),
+                row(0, set(4, 5, 6), 0),
+                row(0, set(7, 8, 9), 0)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a=? AND b < ?", 0, set(4, 5, 6)),
+                row(0, set(), 0),
+                row(0, set(1, 2, 3), 0)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a=? AND b <= ?", 0, set(4, 5, 6)),
+                row(0, set(), 0),
+                row(0, set(1, 2, 3), 0),
+                row(0, set(4, 5, 6), 0)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a=? AND b > ? AND b <= ?", 0, set(1, 2, 3), set(4, 5, 6)),
+                row(0, set(4, 5, 6), 0)
+            );
+
+            execute("DELETE FROM %s WHERE a=? AND b=?", 0, set());
+            execute("DELETE FROM %s WHERE a=? AND b=?", 0, set(4, 5, 6));
+            assertRows(execute("SELECT * FROM %s"),
+                row(0, set(1, 2, 3), 0),
+                row(0, set(7, 8, 9), 0)
+            );
+        }
+    }
+
+    @Test
+    public void testNestedClusteringKeyUsage() throws Throwable
+    {
+        for (String option : Arrays.asList("", " WITH COMPACT STORAGE"))
+        {
+            createTable("CREATE TABLE %s (a int, b frozen<map<set<int>, list<int>>>, c frozen<set<int>>, d int, PRIMARY KEY (a, b, c))" + option);
+
+            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, map(), set(), 0);
+            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, map(set(), list(1, 2, 3)), set(), 0);
+            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, map(set(1, 2, 3), list(1, 2, 3)), set(1, 2, 3), 0);
+            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, map(set(4, 5, 6), list(1, 2, 3)), set(1, 2, 3), 0);
+            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, map(set(7, 8, 9), list(1, 2, 3)), set(1, 2, 3), 0);
+
+            assertRows(execute("SELECT * FROM %s"),
+                row(0, map(), set(), 0),
+                row(0, map(set(), list(1, 2, 3)), set(), 0),
+                row(0, map(set(1, 2, 3), list(1, 2, 3)), set(1, 2, 3), 0),
+                row(0, map(set(4, 5, 6), list(1, 2, 3)), set(1, 2, 3), 0),
+                row(0, map(set(7, 8, 9), list(1, 2, 3)), set(1, 2, 3), 0)
+            );
+
+            assertRows(execute("SELECT b FROM %s"),
+                row(map()),
+                row(map(set(), list(1, 2, 3))),
+                row(map(set(1, 2, 3), list(1, 2, 3))),
+                row(map(set(4, 5, 6), list(1, 2, 3))),
+                row(map(set(7, 8, 9), list(1, 2, 3)))
+            );
+
+            assertRows(execute("SELECT c FROM %s"),
+                row(set()),
+                row(set()),
+                row(set(1, 2, 3)),
+                row(set(1, 2, 3)),
+                row(set(1, 2, 3))
+            );
+
+            assertRows(execute("SELECT * FROM %s LIMIT 3"),
+                row(0, map(), set(), 0),
+                row(0, map(set(), list(1, 2, 3)), set(), 0),
+                row(0, map(set(1, 2, 3), list(1, 2, 3)), set(1, 2, 3), 0)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a=0 ORDER BY b DESC LIMIT 4"),
+                row(0, map(set(7, 8, 9), list(1, 2, 3)), set(1, 2, 3), 0),
+                row(0, map(set(4, 5, 6), list(1, 2, 3)), set(1, 2, 3), 0),
+                row(0, map(set(1, 2, 3), list(1, 2, 3)), set(1, 2, 3), 0),
+                row(0, map(set(), list(1, 2, 3)), set(), 0)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a=? AND b=?", 0, map()),
+                row(0, map(), set(), 0)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a=? AND b=?", 0, map(set(), list(1, 2, 3))),
+                row(0, map(set(), list(1, 2, 3)), set(), 0)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a=? AND b=?", 0, map(set(1, 2, 3), list(1, 2, 3))),
+                row(0, map(set(1, 2, 3), list(1, 2, 3)), set(1, 2, 3), 0)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a=? AND b=? AND c=?", 0, map(set(), list(1, 2, 3)), set()),
+                    row(0, map(set(), list(1, 2, 3)), set(), 0)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a=? AND (b, c) IN ?", 0, list(tuple(map(set(4, 5, 6), list(1, 2, 3)), set(1, 2, 3)),
+                                                                                     tuple(map(), set()))),
+                row(0, map(), set(), 0),
+                row(0, map(set(4, 5, 6), list(1, 2, 3)), set(1, 2, 3), 0)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a=? AND b > ?", 0, map(set(4, 5, 6), list(1, 2, 3))),
+                row(0, map(set(7, 8, 9), list(1, 2, 3)), set(1, 2, 3), 0)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a=? AND b >= ?", 0, map(set(4, 5, 6), list(1, 2, 3))),
+                row(0, map(set(4, 5, 6), list(1, 2, 3)), set(1, 2, 3), 0),
+                row(0, map(set(7, 8, 9), list(1, 2, 3)), set(1, 2, 3), 0)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a=? AND b < ?", 0, map(set(4, 5, 6), list(1, 2, 3))),
+                row(0, map(), set(), 0),
+                row(0, map(set(), list(1, 2, 3)), set(), 0),
+                row(0, map(set(1, 2, 3), list(1, 2, 3)), set(1, 2, 3), 0)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a=? AND b <= ?", 0, map(set(4, 5, 6), list(1, 2, 3))),
+                row(0, map(), set(), 0),
+                row(0, map(set(), list(1, 2, 3)), set(), 0),
+                row(0, map(set(1, 2, 3), list(1, 2, 3)), set(1, 2, 3), 0),
+                row(0, map(set(4, 5, 6), list(1, 2, 3)), set(1, 2, 3), 0)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a=? AND b > ? AND b <= ?", 0, map(set(1, 2, 3), list(1, 2, 3)), map(set(4, 5, 6), list(1, 2, 3))),
+                row(0, map(set(4, 5, 6), list(1, 2, 3)), set(1, 2, 3), 0)
+            );
+
+            execute("DELETE FROM %s WHERE a=? AND b=? AND c=?", 0, map(), set());
+            assertEmpty(execute("SELECT * FROM %s WHERE a=? AND b=? AND c=?", 0, map(), set()));
+
+            execute("DELETE FROM %s WHERE a=? AND b=? AND c=?", 0, map(set(), list(1, 2, 3)), set());
+            assertEmpty(execute("SELECT * FROM %s WHERE a=? AND b=? AND c=?", 0, map(set(), list(1, 2, 3)), set()));
+
+            execute("DELETE FROM %s WHERE a=? AND b=? AND c=?", 0, map(set(4, 5, 6), list(1, 2, 3)), set(1, 2, 3));
+            assertEmpty(execute("SELECT * FROM %s WHERE a=? AND b=? AND c=?", 0, map(set(4, 5, 6), list(1, 2, 3)), set(1, 2, 3)));
+
+            assertRows(execute("SELECT * FROM %s"),
+                    row(0, map(set(1, 2, 3), list(1, 2, 3)), set(1, 2, 3), 0),
+                    row(0, map(set(7, 8, 9), list(1, 2, 3)), set(1, 2, 3), 0)
+            );
+        }
+    }
+
+    @Test
+    public void testNormalColumnUsage() throws Throwable
+    {
+        for (String option : Arrays.asList("", " WITH COMPACT STORAGE"))
+        {
+            createTable("CREATE TABLE %s (a int PRIMARY KEY, b frozen<map<set<int>, list<int>>>, c frozen<set<int>>)" + option);
+
+            execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, map(), set());
+            execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 1, map(set(), list(99999, 999999, 99999)), set());
+            execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 2, map(set(1, 2, 3), list(1, 2, 3)), set(1, 2, 3));
+            execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 3, map(set(4, 5, 6), list(1, 2, 3)), set(1, 2, 3));
+            execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 4, map(set(7, 8, 9), list(1, 2, 3)), set(1, 2, 3));
+
+            // overwrite with update
+            execute ("UPDATE %s SET b=? WHERE a=?", map(set(), list(1, 2, 3)), 1);
+
+            assertRows(execute("SELECT * FROM %s"),
+                row(0, map(), set()),
+                row(1, map(set(), list(1, 2, 3)), set()),
+                row(2, map(set(1, 2, 3), list(1, 2, 3)), set(1, 2, 3)),
+                row(3, map(set(4, 5, 6), list(1, 2, 3)), set(1, 2, 3)),
+                row(4, map(set(7, 8, 9), list(1, 2, 3)), set(1, 2, 3))
+            );
+
+            assertRows(execute("SELECT b FROM %s"),
+                row(map()),
+                row(map(set(), list(1, 2, 3))),
+                row(map(set(1, 2, 3), list(1, 2, 3))),
+                row(map(set(4, 5, 6), list(1, 2, 3))),
+                row(map(set(7, 8, 9), list(1, 2, 3)))
+            );
+
+            assertRows(execute("SELECT c FROM %s"),
+                row(set()),
+                row(set()),
+                row(set(1, 2, 3)),
+                row(set(1, 2, 3)),
+                row(set(1, 2, 3))
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a=?", 3),
+                row(3, map(set(4, 5, 6), list(1, 2, 3)), set(1, 2, 3))
+            );
+
+            execute("UPDATE %s SET b=? WHERE a=?", null, 1);
+            assertRows(execute("SELECT * FROM %s WHERE a=?", 1),
+                row(1, null, set())
+            );
+
+            execute("UPDATE %s SET b=? WHERE a=?", map(), 1);
+            assertRows(execute("SELECT * FROM %s WHERE a=?", 1),
+                row(1, map(), set())
+            );
+
+            execute("UPDATE %s SET c=? WHERE a=?", null, 2);
+            assertRows(execute("SELECT * FROM %s WHERE a=?", 2),
+                row(2, map(set(1, 2, 3), list(1, 2, 3)), null)
+            );
+
+            execute("UPDATE %s SET c=? WHERE a=?", set(), 2);
+            assertRows(execute("SELECT * FROM %s WHERE a=?", 2),
+                    row(2, map(set(1, 2, 3), list(1, 2, 3)), set())
+            );
+
+            execute("DELETE b FROM %s WHERE a=?", 3);
+            assertRows(execute("SELECT * FROM %s WHERE a=?", 3),
+                row(3, null, set(1, 2, 3))
+            );
+
+            execute("DELETE c FROM %s WHERE a=?", 4);
+            assertRows(execute("SELECT * FROM %s WHERE a=?", 4),
+                row(4, map(set(7, 8, 9), list(1, 2, 3)), null)
+            );
+        }
+    }
+
+    @Test
+    public void testStaticColumnUsage() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int, b int, c frozen<map<set<int>, list<int>>> static, d int, PRIMARY KEY (a, b))");
+        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, map(), 0);
+        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, map(), 0);
+        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 1, 0, map(set(), list(1, 2, 3)), 0);
+        execute("INSERT INTO %s (a, b, d) VALUES (?, ?, ?)", 1, 1, 0);
+        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 2, 0, map(set(1, 2, 3), list(1, 2, 3)), 0);
+
+        assertRows(execute("SELECT * FROM %s"),
+            row(0, 0, map(), 0),
+            row(0, 1, map(), 0),
+            row(1, 0, map(set(), list(1, 2, 3)), 0),
+            row(1, 1, map(set(), list(1, 2, 3)), 0),
+            row(2, 0, map(set(1, 2, 3), list(1, 2, 3)), 0)
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE a=? AND b=?", 0, 1),
+            row(0, 1, map(), 0)
+        );
+
+        execute("DELETE c FROM %s WHERE a=?", 0);
+        assertRows(execute("SELECT * FROM %s"),
+                row(0, 0, null, 0),
+                row(0, 1, null, 0),
+                row(1, 0, map(set(), list(1, 2, 3)), 0),
+                row(1, 1, map(set(), list(1, 2, 3)), 0),
+                row(2, 0, map(set(1, 2, 3), list(1, 2, 3)), 0)
+        );
+
+        execute("DELETE FROM %s WHERE a=?", 0);
+        assertRows(execute("SELECT * FROM %s"),
+                row(1, 0, map(set(), list(1, 2, 3)), 0),
+                row(1, 1, map(set(), list(1, 2, 3)), 0),
+                row(2, 0, map(set(1, 2, 3), list(1, 2, 3)), 0)
+        );
+
+        execute("UPDATE %s SET c=? WHERE a=?", map(set(1, 2, 3), list(1, 2, 3)), 1);
+        assertRows(execute("SELECT * FROM %s"),
+                row(1, 0, map(set(1, 2, 3), list(1, 2, 3)), 0),
+                row(1, 1, map(set(1, 2, 3), list(1, 2, 3)), 0),
+                row(2, 0, map(set(1, 2, 3), list(1, 2, 3)), 0)
+        );
+    }
+
+    private void assertInvalidCreateWithMessage(String createTableStatement, String errorMessage) throws Throwable
+    {
+         try
+        {
+            createTableMayThrow(createTableStatement);
+            Assert.fail("Expected CREATE TABLE statement to error: " + createTableStatement);
+        }
+        catch (InvalidRequestException | ConfigurationException | SyntaxException ex)
+        {
+            Assert.assertTrue("Expected error message to contain '" + errorMessage + "', but got '" + ex.getMessage() + "'",
+                    ex.getMessage().contains(errorMessage));
+        }
+    }
+
+    private void assertInvalidAlterWithMessage(String createTableStatement, String errorMessage) throws Throwable
+    {
+        try
+        {
+            alterTableMayThrow(createTableStatement);
+            Assert.fail("Expected CREATE TABLE statement to error: " + createTableStatement);
+        }
+        catch (InvalidRequestException | ConfigurationException ex)
+        {
+            Assert.assertTrue("Expected error message to contain '" + errorMessage + "', but got '" + ex.getMessage() + "'",
+                    ex.getMessage().contains(errorMessage));
+        }
+    }
+
+    @Test
+    public void testInvalidOperations() throws Throwable
+    {
+        // lists
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, l frozen<list<int>>)");
+        assertInvalid("UPDATE %s SET l[?]=? WHERE k=?", 0, 0, 0);
+        assertInvalid("UPDATE %s SET l = ? + l WHERE k=?", list(0), 0);
+        assertInvalid("UPDATE %s SET l = l + ? WHERE k=?", list(4), 0);
+        assertInvalid("UPDATE %s SET l = l - ? WHERE k=?", list(3), 0);
+        assertInvalid("DELETE l[?] FROM %s WHERE k=?", 0, 0);
+
+        // sets
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, s frozen<set<int>>)");
+        assertInvalid("UPDATE %s SET s = s + ? WHERE k=?", set(0), 0);
+        assertInvalid("UPDATE %s SET s = s - ? WHERE k=?", set(3), 0);
+
+        // maps
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, m frozen<map<int, int>>)");
+        assertInvalid("UPDATE %s SET m[?]=? WHERE k=?", 0, 0, 0);
+        assertInvalid("UPDATE %s SET m = m + ? WHERE k=?", map(4, 4), 0);
+        assertInvalid("DELETE m[?] FROM %s WHERE k=?", 0, 0);
+
+        assertInvalidCreateWithMessage("CREATE TABLE %s (k int PRIMARY KEY, t set<set<int>>)",
+                "Non-frozen collections are not allowed inside collections");
+
+        assertInvalidCreateWithMessage("CREATE TABLE %s (k int PRIMARY KEY, t frozen<set<counter>>)",
+                                       "Counters are not allowed inside collections");
+
+        assertInvalidCreateWithMessage("CREATE TABLE %s (k int PRIMARY KEY, t frozen<text>)",
+                "frozen<> is only allowed on collections, tuples, and user-defined types");
+    }
+
+    @Test
+    public void testAltering() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int, b frozen<list<int>>, c frozen<list<int>>, PRIMARY KEY (a, b))");
+
+        alterTable("ALTER TABLE %s ALTER c TYPE frozen<list<blob>>");
+
+        assertInvalidAlterWithMessage("ALTER TABLE %s ALTER b TYPE frozen<list<blob>>",
+                                      "types are not order-compatible");
+
+        assertInvalidAlterWithMessage("ALTER TABLE %s ALTER b TYPE list<int>",
+                                      "types are not order-compatible");
+
+        assertInvalidAlterWithMessage("ALTER TABLE %s ALTER c TYPE list<blob>",
+                                      "types are incompatible");
+
+        alterTable("ALTER TABLE %s DROP c");
+        alterTable("ALTER TABLE %s ADD c frozen<set<int>>");
+        assertInvalidAlterWithMessage("ALTER TABLE %s ALTER c TYPE frozen<set<blob>>",
+                                      "types are incompatible");
+
+        alterTable("ALTER TABLE %s DROP c");
+        alterTable("ALTER TABLE %s ADD c frozen<map<int, int>>");
+        assertInvalidAlterWithMessage("ALTER TABLE %s ALTER c TYPE frozen<map<blob, int>>",
+                                      "types are incompatible");
+        alterTable("ALTER TABLE %s ALTER c TYPE frozen<map<int, blob>>");
+    }
+
+    private void assertInvalidIndexCreationWithMessage(String statement, String errorMessage) throws Throwable
+    {
+        try
+        {
+            createIndexMayThrow(statement);
+            Assert.fail("Expected index creation to fail: " + statement);
+        }
+        catch (InvalidRequestException ex)
+        {
+            Assert.assertTrue("Expected error message to contain '" + errorMessage + "', but got '" + ex.getMessage() + "'",
+                              ex.getMessage().contains(errorMessage));
+        }
+    }
+
+    @Test
+    public void testSecondaryIndex() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a frozen<map<int, text>> PRIMARY KEY, b frozen<map<int, text>>)");
+
+        // for now, we don't support indexing values or keys of collections in the primary key
+        assertInvalidIndexCreationWithMessage("CREATE INDEX ON %s (full(a))", "Cannot create secondary index on partition key column");
+        assertInvalidIndexCreationWithMessage("CREATE INDEX ON %s (keys(a))", "Cannot create index on keys of frozen<map> column");
+        assertInvalidIndexCreationWithMessage("CREATE INDEX ON %s (keys(b))", "Cannot create index on keys of frozen<map> column");
+
+        createTable("CREATE TABLE %s (a int, b frozen<list<int>>, c frozen<set<int>>, d frozen<map<int, text>>, PRIMARY KEY (a, b))");
+
+        createIndex("CREATE INDEX ON %s (full(b))");
+        createIndex("CREATE INDEX ON %s (full(c))");
+        createIndex("CREATE INDEX ON %s (full(d))");
+
+        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, list(1, 2, 3), set(1, 2, 3), map(1, "a"));
+        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, list(4, 5, 6), set(1, 2, 3), map(1, "a"));
+        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 1, list(1, 2, 3), set(4, 5, 6), map(2, "b"));
+        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 1, list(4, 5, 6), set(4, 5, 6), map(2, "b"));
+
+        // CONTAINS KEY doesn't work on non-maps
+        assertInvalidMessage("Cannot use CONTAINS KEY on non-map column",
+                             "SELECT * FROM %s WHERE b CONTAINS KEY ?", 1);
+
+        assertInvalidMessage("Cannot use CONTAINS KEY on non-map column",
+                             "SELECT * FROM %s WHERE b CONTAINS KEY ? ALLOW FILTERING", 1);
+
+        assertInvalidMessage("Cannot use CONTAINS KEY on non-map column",
+                             "SELECT * FROM %s WHERE c CONTAINS KEY ?", 1);
+
+        // normal indexes on frozen collections don't support CONTAINS or CONTAINS KEY
+        assertInvalidMessage("Cannot restrict clustering columns by a CONTAINS relation without a secondary index",
+                             "SELECT * FROM %s WHERE b CONTAINS ?", 1);
+
+        assertInvalidMessage("Cannot restrict clustering columns by a CONTAINS relation without a secondary index",
+                             "SELECT * FROM %s WHERE b CONTAINS ? ALLOW FILTERING", 1);
+
+        assertInvalidMessage("No secondary indexes on the restricted columns support the provided operator",
+                             "SELECT * FROM %s WHERE d CONTAINS KEY ?", 1);
+
+        assertInvalidMessage("No secondary indexes on the restricted columns support the provided operator",
+                             "SELECT * FROM %s WHERE d CONTAINS KEY ? ALLOW FILTERING", 1);
+
+        assertInvalidMessage("Cannot restrict clustering columns by a CONTAINS relation without a secondary index",
+                             "SELECT * FROM %s WHERE b CONTAINS ? AND d CONTAINS KEY ? ALLOW FILTERING", 1, 1);
+
+        // index lookup on b
+        assertRows(execute("SELECT * FROM %s WHERE b=?", list(1, 2, 3)),
+            row(0, list(1, 2, 3), set(1, 2, 3), map(1, "a")),
+            row(1, list(1, 2, 3), set(4, 5, 6), map(2, "b"))
+        );
+
+        assertEmpty(execute("SELECT * FROM %s WHERE b=?", list(-1)));
+
+        assertInvalidMessage("ALLOW FILTERING", "SELECT * FROM %s WHERE b=? AND c=?", list(1, 2, 3), set(4, 5, 6));
+        assertRows(execute("SELECT * FROM %s WHERE b=? AND c=? ALLOW FILTERING", list(1, 2, 3), set(4, 5, 6)),
+            row(1, list(1, 2, 3), set(4, 5, 6), map(2, "b"))
+        );
+
+        assertInvalidMessage("ALLOW FILTERING", "SELECT * FROM %s WHERE b=? AND c CONTAINS ?", list(1, 2, 3), 5);
+        assertRows(execute("SELECT * FROM %s WHERE b=? AND c CONTAINS ? ALLOW FILTERING", list(1, 2, 3), 5),
+            row(1, list(1, 2, 3), set(4, 5, 6), map(2, "b"))
+        );
+
+        assertInvalidMessage("ALLOW FILTERING", "SELECT * FROM %s WHERE b=? AND d=?", list(1, 2, 3), map(1, "a"));
+        assertRows(execute("SELECT * FROM %s WHERE b=? AND d=? ALLOW FILTERING", list(1, 2, 3), map(1, "a")),
+            row(0, list(1, 2, 3), set(1, 2, 3), map(1, "a"))
+        );
+
+        assertInvalidMessage("ALLOW FILTERING", "SELECT * FROM %s WHERE b=? AND d CONTAINS ?", list(1, 2, 3), "a");
+        assertRows(execute("SELECT * FROM %s WHERE b=? AND d CONTAINS ? ALLOW FILTERING", list(1, 2, 3), "a"),
+            row(0, list(1, 2, 3), set(1, 2, 3), map(1, "a"))
+        );
+
+        assertInvalidMessage("ALLOW FILTERING", "SELECT * FROM %s WHERE b=? AND d CONTAINS KEY ?", list(1, 2, 3), 1);
+        assertRows(execute("SELECT * FROM %s WHERE b=? AND d CONTAINS KEY ? ALLOW FILTERING", list(1, 2, 3), 1),
+            row(0, list(1, 2, 3), set(1, 2, 3), map(1, "a"))
+        );
+
+        // index lookup on c
+        assertRows(execute("SELECT * FROM %s WHERE c=?", set(1, 2, 3)),
+            row(0, list(1, 2, 3), set(1, 2, 3), map(1, "a")),
+            row(0, list(4, 5, 6), set(1, 2, 3), map(1, "a"))
+        );
+
+        // ordering of c should not matter
+        assertRows(execute("SELECT * FROM %s WHERE c=?", set(2, 1, 3)),
+                row(0, list(1, 2, 3), set(1, 2, 3), map(1, "a")),
+                row(0, list(4, 5, 6), set(1, 2, 3), map(1, "a"))
+        );
+
+        assertEmpty(execute("SELECT * FROM %s WHERE c=?", set(-1)));
+
+        assertInvalidMessage("ALLOW FILTERING", "SELECT * FROM %s WHERE c=? AND b=?", set(1, 2, 3), list(1, 2, 3));
+        assertRows(execute("SELECT * FROM %s WHERE c=? AND b=? ALLOW FILTERING", set(1, 2, 3), list(1, 2, 3)),
+            row(0, list(1, 2, 3), set(1, 2, 3), map(1, "a"))
+        );
+
+        assertInvalidMessage("ALLOW FILTERING", "SELECT * FROM %s WHERE c=? AND b CONTAINS ?", set(1, 2, 3), 1);
+        assertRows(execute("SELECT * FROM %s WHERE c=? AND b CONTAINS ? ALLOW FILTERING", set(1, 2, 3), 1),
+            row(0, list(1, 2, 3), set(1, 2, 3), map(1, "a"))
+        );
+
+        assertInvalidMessage("ALLOW FILTERING", "SELECT * FROM %s WHERE c=? AND d = ?", set(1, 2, 3), map(1, "a"));
+        assertRows(execute("SELECT * FROM %s WHERE c=? AND d = ? ALLOW FILTERING", set(1, 2, 3), map(1, "a")),
+            row(0, list(1, 2, 3), set(1, 2, 3), map(1, "a")),
+            row(0, list(4, 5, 6), set(1, 2, 3), map(1, "a"))
+        );
+
+        assertInvalidMessage("ALLOW FILTERING", "SELECT * FROM %s WHERE c=? AND d CONTAINS ?", set(1, 2, 3), "a");
+        assertRows(execute("SELECT * FROM %s WHERE c=? AND d CONTAINS ? ALLOW FILTERING", set(1, 2, 3), "a"),
+            row(0, list(1, 2, 3), set(1, 2, 3), map(1, "a")),
+            row(0, list(4, 5, 6), set(1, 2, 3), map(1, "a"))
+        );
+
+        assertInvalidMessage("ALLOW FILTERING", "SELECT * FROM %s WHERE c=? AND d CONTAINS KEY ?", set(1, 2, 3), 1);
+        assertRows(execute("SELECT * FROM %s WHERE c=? AND d CONTAINS KEY ? ALLOW FILTERING", set(1, 2, 3), 1),
+            row(0, list(1, 2, 3), set(1, 2, 3), map(1, "a")),
+            row(0, list(4, 5, 6), set(1, 2, 3), map(1, "a"))
+        );
+
+        // index lookup on d
+        assertRows(execute("SELECT * FROM %s WHERE d=?", map(1, "a")),
+            row(0, list(1, 2, 3), set(1, 2, 3), map(1, "a")),
+            row(0, list(4, 5, 6), set(1, 2, 3), map(1, "a"))
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE d=?", map(2, "b")),
+            row(1, list(1, 2, 3), set(4, 5, 6), map(2, "b")),
+            row(1, list(4, 5, 6), set(4, 5, 6), map(2, "b"))
+        );
+
+        assertEmpty(execute("SELECT * FROM %s WHERE d=?", map(3, "c")));
+
+        assertInvalidMessage("ALLOW FILTERING", "SELECT * FROM %s WHERE d=? AND c=?", map(1, "a"), set(1, 2, 3));
+        assertRows(execute("SELECT * FROM %s WHERE d=? AND b=? ALLOW FILTERING", map(1, "a"), list(1, 2, 3)),
+            row(0, list(1, 2, 3), set(1, 2, 3), map(1, "a"))
+        );
+
+        assertInvalidMessage("ALLOW FILTERING", "SELECT * FROM %s WHERE d=? AND b CONTAINS ?", map(1, "a"), 3);
+        assertRows(execute("SELECT * FROM %s WHERE d=? AND b CONTAINS ? ALLOW FILTERING", map(1, "a"), 3),
+            row(0, list(1, 2, 3), set(1, 2, 3), map(1, "a"))
+        );
+
+        assertInvalidMessage("ALLOW FILTERING", "SELECT * FROM %s WHERE d=? AND b=? AND c=?", map(1, "a"), list(1, 2, 3), set(1, 2, 3));
+        assertRows(execute("SELECT * FROM %s WHERE d=? AND b=? AND c=? ALLOW FILTERING", map(1, "a"), list(1, 2, 3), set(1, 2, 3)),
+            row(0, list(1, 2, 3), set(1, 2, 3), map(1, "a"))
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE d=? AND b CONTAINS ? AND c CONTAINS ? ALLOW FILTERING", map(1, "a"), 2, 2),
+            row(0, list(1, 2, 3), set(1, 2, 3), map(1, "a"))
+        );
+
+        execute("DELETE d FROM %s WHERE a=? AND b=?", 0, list(1, 2, 3));
+        assertRows(execute("SELECT * FROM %s WHERE d=?", map(1, "a")),
+            row(0, list(4, 5, 6), set(1, 2, 3), map(1, "a"))
+        );
+    }
+
+    /** Test for CASSANDRA-8302 */
+    @Test
+    public void testClusteringColumnFiltering() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int, b frozen<map<int, int>>, c int, d int, PRIMARY KEY (a, b, c))");
+        createIndex("CREATE INDEX c_index ON %s (c)");
+        createIndex("CREATE INDEX d_index ON %s (d)");
+
+        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, map(0, 0, 1, 1), 0, 0);
+        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, map(1, 1, 2, 2), 0, 0);
+        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 1, map(0, 0, 1, 1), 0, 0);
+        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 1, map(1, 1, 2, 2), 0, 0);
+
+        assertRows(execute("SELECT * FROM %s WHERE d=? AND b CONTAINS ? ALLOW FILTERING", 0, 0),
+                row(0, map(0, 0, 1, 1), 0, 0),
+                row(1, map(0, 0, 1, 1), 0, 0)
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE d=? AND b CONTAINS KEY ? ALLOW FILTERING", 0, 0),
+                row(0, map(0, 0, 1, 1), 0, 0),
+                row(1, map(0, 0, 1, 1), 0, 0)
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE a=? AND d=? AND b CONTAINS ? ALLOW FILTERING", 0, 0, 0),
+                row(0, map(0, 0, 1, 1), 0, 0)
+        );
+        assertRows(execute("SELECT * FROM %s WHERE a=? AND d=? AND b CONTAINS KEY ? ALLOW FILTERING", 0, 0, 0),
+                row(0, map(0, 0, 1, 1), 0, 0)
+        );
+
+        dropIndex("DROP INDEX %s.d_index");
+
+        assertRows(execute("SELECT * FROM %s WHERE c=? AND b CONTAINS ? ALLOW FILTERING", 0, 0),
+                row(0, map(0, 0, 1, 1), 0, 0),
+                row(1, map(0, 0, 1, 1), 0, 0)
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE c=? AND b CONTAINS KEY ? ALLOW FILTERING", 0, 0),
+                row(0, map(0, 0, 1, 1), 0, 0),
+                row(1, map(0, 0, 1, 1), 0, 0)
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE a=? AND c=? AND b CONTAINS ? ALLOW FILTERING", 0, 0, 0),
+                row(0, map(0, 0, 1, 1), 0, 0)
+        );
+        assertRows(execute("SELECT * FROM %s WHERE a=? AND c=? AND b CONTAINS KEY ? ALLOW FILTERING", 0, 0, 0),
+                row(0, map(0, 0, 1, 1), 0, 0)
+        );
+    }
+
+    @Test
+    public void testFrozenListInMap() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int primary key, m map<frozen<list<int>>, int>)");
+
+        execute("INSERT INTO %s (k, m) VALUES (1, {[1, 2, 3] : 1})");
+        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
+                row(1, map(list(1, 2, 3), 1)));
+
+        execute("UPDATE %s SET m[[1, 2, 3]]=2 WHERE k=1");
+        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
+                row(1, map(list(1, 2, 3), 2)));
+
+        execute("UPDATE %s SET m = m + ? WHERE k=1", map(list(4, 5, 6), 3));
+        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
+                row(1,
+                    map(list(1, 2, 3), 2,
+                        list(4, 5, 6), 3)));
+
+        execute("DELETE m[[1, 2, 3]] FROM %s WHERE k = 1");
+        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
+                row(1, map(list(4, 5, 6), 3)));
+    }
+
+    @Test
+    public void testFrozenListInSet() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int primary key, s set<frozen<list<int>>>)");
+
+        execute("INSERT INTO %s (k, s) VALUES (1, {[1, 2, 3]})");
+        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
+                row(1, set(list(1, 2, 3)))
+        );
+
+        execute("UPDATE %s SET s = s + ? WHERE k=1", set(list(4, 5, 6)));
+        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
+                row(1, set(list(1, 2, 3), list(4, 5, 6)))
+        );
+
+        execute("UPDATE %s SET s = s - ? WHERE k=1", set(list(4, 5, 6)));
+        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
+                row(1, set(list(1, 2, 3)))
+        );
+
+        execute("DELETE s[[1, 2, 3]] FROM %s WHERE k = 1");
+        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
+                row(1, null)
+        );
+    }
+
+    @Test
+    public void testFrozenListInList() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int primary key, l list<frozen<list<int>>>)");
+
+        execute("INSERT INTO %s (k, l) VALUES (1, [[1, 2, 3]])");
+        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
+                row(1, list(list(1, 2, 3)))
+        );
+
+        execute("UPDATE %s SET l[?]=? WHERE k=1", 0, list(4, 5, 6));
+        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
+                row(1, list(list(4, 5, 6)))
+        );
+
+        execute("UPDATE %s SET l = ? + l WHERE k=1", list(list(1, 2, 3)));
+        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
+                row(1, list(list(1, 2, 3), list(4, 5, 6)))
+        );
+
+        execute("UPDATE %s SET l = l + ? WHERE k=1", list(list(7, 8, 9)));
+        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
+                row(1, list(list(1, 2, 3), list(4, 5, 6), list(7, 8, 9)))
+        );
+
+        execute("UPDATE %s SET l = l - ? WHERE k=1", list(list(4, 5, 6)));
+        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
+                row(1, list(list(1, 2, 3), list(7, 8, 9)))
+        );
+
+        execute("DELETE l[0] FROM %s WHERE k = 1");
+        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
+                row(1, list(list(7, 8, 9)))
+        );
+    }
+
+    @Test
+    public void testFrozenMapInMap() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int primary key, m map<frozen<map<int, int>>, int>)");
+
+        execute("INSERT INTO %s (k, m) VALUES (1, {{1 : 1, 2 : 2} : 1})");
+        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
+                row(1, map(map(1, 1, 2, 2), 1)));
+
+        execute("UPDATE %s SET m[?]=2 WHERE k=1", map(1, 1, 2, 2));
+        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
+                row(1, map(map(1, 1, 2, 2), 2)));
+
+        execute("UPDATE %s SET m = m + ? WHERE k=1", map(map(3, 3, 4, 4), 3));
+        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
+                row(1,
+                    map(map(1, 1, 2, 2), 2,
+                        map(3, 3, 4, 4), 3)));
+
+        execute("DELETE m[?] FROM %s WHERE k = 1", map(1, 1, 2, 2));
+        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
+                row(1, map(map(3, 3, 4, 4), 3)));
+    }
+
+    @Test
+    public void testFrozenMapInSet() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int primary key, s set<frozen<map<int, int>>>)");
+
+        execute("INSERT INTO %s (k, s) VALUES (1, {{1 : 1, 2 : 2}})");
+
+        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
+                row(1, set(map(1, 1, 2, 2)))
+        );
+
+        execute("UPDATE %s SET s = s + ? WHERE k=1", set(map(3, 3, 4, 4)));
+        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
+                row(1, set(map(1, 1, 2, 2), map(3, 3, 4, 4)))
+        );
+
+        execute("UPDATE %s SET s = s - ? WHERE k=1", set(map(3, 3, 4, 4)));
+        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
+                row(1, set(map(1, 1, 2, 2)))
+        );
+
+        execute("DELETE s[?] FROM %s WHERE k = 1", map(1, 1, 2, 2));
+        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
+                row(1, null)
+        );
+    }
+
+    @Test
+    public void testFrozenMapInList() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int primary key, l list<frozen<map<int, int>>>)");
+
+        execute("INSERT INTO %s (k, l) VALUES (1, [{1 : 1, 2 : 2}])");
+        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
+                row(1, list(map(1, 1, 2, 2)))
+        );
+
+        execute("UPDATE %s SET l[?]=? WHERE k=1", 0, map(3, 3, 4, 4));
+        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
+                row(1, list(map(3, 3, 4, 4)))
+        );
+
+        execute("UPDATE %s SET l = ? + l WHERE k=1", list(map(1, 1, 2, 2)));
+        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
+                row(1, list(map(1, 1, 2, 2), map(3, 3, 4, 4)))
+        );
+
+        execute("UPDATE %s SET l = l + ? WHERE k=1", list(map(5, 5, 6, 6)));
+        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
+                row(1, list(map(1, 1, 2, 2), map(3, 3, 4, 4), map(5, 5, 6, 6)))
+        );
+
+        execute("UPDATE %s SET l = l - ? WHERE k=1", list(map(3, 3, 4, 4)));
+        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
+                row(1, list(map(1, 1, 2, 2), map(5, 5, 6, 6)))
+        );
+
+        execute("DELETE l[0] FROM %s WHERE k = 1");
+        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
+                row(1, list(map(5, 5, 6, 6)))
+        );
+    }
+
+    @Test
+    public void testFrozenSetInMap() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int primary key, m map<frozen<set<int>>, int>)");
+
+        execute("INSERT INTO %s (k, m) VALUES (1, {{1, 2, 3} : 1})");
+        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
+                row(1, map(set(1, 2, 3), 1)));
+
+        execute("UPDATE %s SET m[?]=2 WHERE k=1", set(1, 2, 3));
+        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
+                row(1, map(set(1, 2, 3), 2)));
+
+        execute("UPDATE %s SET m = m + ? WHERE k=1", map(set(4, 5, 6), 3));
+        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
+                row(1,
+                    map(set(1, 2, 3), 2,
+                        set(4, 5, 6), 3)));
+
+        execute("DELETE m[?] FROM %s WHERE k = 1", set(1, 2, 3));
+        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
+                row(1, map(set(4, 5, 6), 3)));
+    }
+
+    @Test
+    public void testFrozenSetInSet() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int primary key, s set<frozen<set<int>>>)");
+
+        execute("INSERT INTO %s (k, s) VALUES (1, {{1, 2, 3}})");
+
+        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
+                row(1, set(set(1, 2, 3)))
+        );
+
+        execute("UPDATE %s SET s = s + ? WHERE k=1", set(set(4, 5, 6)));
+        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
+                row(1, set(set(1, 2, 3), set(4, 5, 6)))
+        );
+
+        execute("UPDATE %s SET s = s - ? WHERE k=1", set(set(4, 5, 6)));
+        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
+                row(1, set(set(1, 2, 3)))
+        );
+
+        execute("DELETE s[?] FROM %s WHERE k = 1", set(1, 2, 3));
+        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
+                row(1, null)
+        );
+    }
+
+    @Test
+    public void testFrozenSetInList() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int primary key, l list<frozen<set<int>>>)");
+
+        execute("INSERT INTO %s (k, l) VALUES (1, [{1, 2, 3}])");
+        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
+                row(1, list(set(1, 2, 3)))
+        );
+
+        execute("UPDATE %s SET l[?]=? WHERE k=1", 0, set(4, 5, 6));
+        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
+                row(1, list(set(4, 5, 6)))
+        );
+
+        execute("UPDATE %s SET l = ? + l WHERE k=1", list(set(1, 2, 3)));
+        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
+                row(1, list(set(1, 2, 3), set(4, 5, 6)))
+        );
+
+        execute("UPDATE %s SET l = l + ? WHERE k=1", list(set(7, 8, 9)));
+        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
+                row(1, list(set(1, 2, 3), set(4, 5, 6), set(7, 8, 9)))
+        );
+
+        execute("UPDATE %s SET l = l - ? WHERE k=1", list(set(4, 5, 6)));
+        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
+                row(1, list(set(1, 2, 3), set(7, 8, 9)))
+        );
+
+        execute("DELETE l[0] FROM %s WHERE k = 1");
+        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
+                row(1, list(set(7, 8, 9)))
+        );
+    }
+
+    @Test
+    public void testUserDefinedTypes() throws Throwable
+    {
+        String myType = createType("CREATE TYPE %s (a set<int>, b tuple<list<int>>)");
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, v frozen<" + myType + ">)");
+        execute("INSERT INTO %s (k, v) VALUES (?, {a: ?, b: ?})", 0, set(1, 2, 3), tuple(list(1, 2, 3)));
+        assertRows(execute("SELECT v.a, v.b FROM %s WHERE k=?", 0),
+            row(set(1, 2, 3), tuple(list(1, 2, 3)))
+        );
+    }
+
+    private static String clean(String classname)
+    {
+        return StringUtils.remove(classname, "org.apache.cassandra.db.marshal.");
+    }
+
+    @Test
+    public void testToString()
+    {
+        // set<frozen<list<int>>>
+        SetType t = SetType.getInstance(ListType.getInstance(Int32Type.instance, false), true);
+        assertEquals("SetType(FrozenType(ListType(Int32Type)))", clean(t.toString()));
+        assertEquals("SetType(ListType(Int32Type))", clean(t.toString(true)));
+
+        // frozen<set<list<int>>>
+        t = SetType.getInstance(ListType.getInstance(Int32Type.instance, false), false);
+        assertEquals("FrozenType(SetType(ListType(Int32Type)))", clean(t.toString()));
+        assertEquals("SetType(ListType(Int32Type))", clean(t.toString(true)));
+
+        // map<frozen<list<int>>, int>
+        MapType m = MapType.getInstance(ListType.getInstance(Int32Type.instance, false), Int32Type.instance, true);
+        assertEquals("MapType(FrozenType(ListType(Int32Type)),Int32Type)", clean(m.toString()));
+        assertEquals("MapType(ListType(Int32Type),Int32Type)", clean(m.toString(true)));
+
+        // frozen<map<list<int>, int>>
+        m = MapType.getInstance(ListType.getInstance(Int32Type.instance, false), Int32Type.instance, false);
+        assertEquals("FrozenType(MapType(ListType(Int32Type),Int32Type))", clean(m.toString()));
+        assertEquals("MapType(ListType(Int32Type),Int32Type)", clean(m.toString(true)));
+
+        // tuple<set<int>>
+        List<AbstractType<?>> types = new ArrayList<>();
+        types.add(SetType.getInstance(Int32Type.instance, true));
+        TupleType tuple = new TupleType(types);
+        assertEquals("TupleType(SetType(Int32Type))", clean(tuple.toString()));
+    }
+}


[07/32] cassandra git commit: Migrate CQL tests from dtest to unit tests

Posted by jm...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/f797bfa4/test/unit/org/apache/cassandra/cql3/MultiColumnRelationTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/MultiColumnRelationTest.java b/test/unit/org/apache/cassandra/cql3/MultiColumnRelationTest.java
deleted file mode 100644
index a4ee5d7..0000000
--- a/test/unit/org/apache/cassandra/cql3/MultiColumnRelationTest.java
+++ /dev/null
@@ -1,808 +0,0 @@
-/*
- * 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;
-
-import org.junit.Test;
-
-public class MultiColumnRelationTest extends CQLTester
-{
-    @Test
-    public void testSingleClusteringInvalidQueries() throws Throwable
-    {
-        for (String compactOption : new String[]{"", " WITH COMPACT STORAGE"})
-        {
-            createTable("CREATE TABLE %s (a int, b int, c int, PRIMARY KEY (a, b))" + compactOption);
-
-            assertInvalidSyntax("SELECT * FROM %s WHERE () = (?, ?)", 1, 2);
-            assertInvalidMessage("Column \"b\" cannot be restricted by an equality relation and an inequality relation",
-                                 "SELECT * FROM %s WHERE a = 0 AND (b) = (?) AND (b) > (?)", 0, 0);
-            assertInvalidMessage("More than one restriction was found for the start bound on b",
-                                 "SELECT * FROM %s WHERE a = 0 AND (b) > (?) AND (b) > (?)", 0, 1);
-            assertInvalidMessage("Multi-column relations can only be applied to clustering columns: a",
-                                 "SELECT * FROM %s WHERE (a, b) = (?, ?)", 0, 0);
-        }
-    }
-
-    @Test
-    public void testMultiClusteringInvalidQueries() throws Throwable
-    {
-        for (String compactOption : new String[]{"", " WITH COMPACT STORAGE"})
-        {
-            createTable("CREATE TABLE %s (a int, b int, c int, d int, PRIMARY KEY (a, b, c, d))" + compactOption);
-
-            assertInvalidSyntax("SELECT * FROM %s WHERE a = 0 AND (b, c) > ()");
-            assertInvalidMessage("Expected 2 elements in value tuple, but got 3: (?, ?, ?)",
-                                 "SELECT * FROM %s WHERE a = 0 AND (b, c) > (?, ?, ?)", 1, 2, 3);
-            assertInvalidMessage("Invalid null value in condition for column c",
-                                 "SELECT * FROM %s WHERE a = 0 AND (b, c) > (?, ?)", 1, null);
-
-            // Wrong order of columns
-            assertInvalidMessage("Clustering columns must appear in the PRIMARY KEY order in multi-column relations: (d, c, b) = (?, ?, ?)",
-                                 "SELECT * FROM %s WHERE a = 0 AND (d, c, b) = (?, ?, ?)", 0, 0, 0);
-            assertInvalidMessage("Clustering columns must appear in the PRIMARY KEY order in multi-column relations: (d, c, b) > (?, ?, ?)",
-                                 "SELECT * FROM %s WHERE a = 0 AND (d, c, b) > (?, ?, ?)", 0, 0, 0);
-
-            // Wrong number of values
-            assertInvalidMessage("Expected 3 elements in value tuple, but got 2: (?, ?)",
-                                 "SELECT * FROM %s WHERE a=0 AND (b, c, d) IN ((?, ?))", 0, 1);
-            assertInvalidMessage("Expected 3 elements in value tuple, but got 5: (?, ?, ?, ?, ?)",
-                                 "SELECT * FROM %s WHERE a=0 AND (b, c, d) IN ((?, ?, ?, ?, ?))", 0, 1, 2, 3, 4);
-
-            // Missing first clustering column
-            assertInvalidMessage("PRIMARY KEY column \"c\" cannot be restricted (preceding column \"b\" is not restricted)",
-                                 "SELECT * FROM %s WHERE a = 0 AND (c, d) = (?, ?)", 0, 0);
-            assertInvalidMessage("PRIMARY KEY column \"c\" cannot be restricted (preceding column \"b\" is not restricted)",
-                                 "SELECT * FROM %s WHERE a = 0 AND (c, d) > (?, ?)", 0, 0);
-
-            // Nulls
-            assertInvalidMessage("Invalid null value in condition for column d",
-                                 "SELECT * FROM %s WHERE a = 0 AND (b, c, d) IN ((?, ?, ?))", 1, 2, null);
-
-            // Wrong type for 'd'
-            assertInvalidMessage("Expected 4 or 0 byte int (6)",
-                                 "SELECT * FROM %s WHERE a = 0 AND (b, c, d) = (?, ?, ?)", 1, 2, "foobar");
-
-            assertInvalidMessage("Invalid tuple type literal for b of type int",
-                                 "SELECT * FROM %s WHERE a = 0 AND b = (?, ?, ?)", 1, 2, 3);
-
-            // Mix single and tuple inequalities
-            assertInvalidMessage("Column \"b\" cannot be restricted by both a tuple notation inequality and a single column inequality (b < ?)",
-                                 "SELECT * FROM %s WHERE a = 0 AND (b, c, d) > (?, ?, ?) AND b < ?", 0, 1, 0, 1);
-            assertInvalidMessage("Column \"c\" cannot be restricted by both a tuple notation inequality and a single column inequality (c < ?)",
-                                 "SELECT * FROM %s WHERE a = 0 AND (b, c, d) > (?, ?, ?) AND c < ?", 0, 1, 0, 1);
-            assertInvalidMessage("Column \"b\" cannot have both tuple-notation inequalities and single-column inequalities: (b, c, d) < (?, ?, ?)",
-                                 "SELECT * FROM %s WHERE a = 0 AND b > ? AND (b, c, d) < (?, ?, ?)", 1, 1, 1, 0);
-            assertInvalidMessage("Column \"c\" cannot have both tuple-notation inequalities and single-column inequalities: (b, c, d) < (?, ?, ?)",
-                                 "SELECT * FROM %s WHERE a = 0 AND c > ? AND (b, c, d) < (?, ?, ?)", 1, 1, 1, 0);
-
-            assertInvalidMessage("Multi-column relations can only be applied to clustering columns: a",
-                                 "SELECT * FROM %s WHERE (a, b, c, d) IN ((?, ?, ?, ?))", 0, 1, 2, 3);
-            assertInvalidMessage("PRIMARY KEY column \"c\" cannot be restricted (preceding column \"b\" is not restricted)",
-                                 "SELECT * FROM %s WHERE (c, d) IN ((?, ?))", 0, 1);
-            assertInvalidMessage("PRIMARY KEY column \"c\" cannot be restricted (preceding column \"b\" is restricted by a non-EQ relation)",
-                                 "SELECT * FROM %s WHERE a = ? AND b > ?  AND (c, d) IN ((?, ?))", 0, 0, 0, 0);
-
-            assertInvalidMessage("PRIMARY KEY column \"c\" cannot be restricted (preceding column \"b\" is restricted by a non-EQ relation)",
-                                 "SELECT * FROM %s WHERE a = ? AND b > ?  AND (c, d) > (?, ?)", 0, 0, 0, 0);
-            assertInvalidMessage("PRIMARY KEY column \"c\" cannot be restricted (preceding column \"b\" is restricted by a non-EQ relation)",
-                                 "SELECT * FROM %s WHERE a = ? AND (c, d) > (?, ?) AND b > ?  ", 0, 0, 0, 0);
-            assertInvalidMessage("Column \"c\" cannot be restricted by two tuple-notation inequalities not starting with the same column: (c) < (?)",
-                                 "SELECT * FROM %s WHERE a = ? AND (b, c) > (?, ?) AND (b) < (?) AND (c) < (?)", 0, 0, 0, 0, 0);
-            assertInvalidMessage("Column \"c\" cannot be restricted by two tuple-notation inequalities not starting with the same column: (b, c) > (?, ?)",
-                                 "SELECT * FROM %s WHERE a = ? AND (c) < (?) AND (b, c) > (?, ?) AND (b) < (?)", 0, 0, 0, 0, 0);
-            assertInvalidMessage("Column \"c\" cannot be restricted by two tuple-notation inequalities not starting with the same column: (b, c) > (?, ?)",
-                                 "SELECT * FROM %s WHERE a = ? AND (b) < (?) AND (c) < (?) AND (b, c) > (?, ?)", 0, 0, 0, 0, 0);
-
-            assertInvalidMessage("Column \"c\" cannot be restricted by two tuple-notation inequalities not starting with the same column: (c) < (?)",
-                                 "SELECT * FROM %s WHERE a = ? AND (b, c) > (?, ?) AND (c) < (?)", 0, 0, 0, 0);
-
-            assertInvalidMessage("PRIMARY KEY column \"d\" cannot be restricted (preceding column \"c\" is restricted by an IN tuple notation)",
-                                 "SELECT * FROM %s WHERE a = ? AND (b, c) in ((?, ?), (?, ?)) AND d > ?", 0, 0, 0, 0, 0, 0);
-        }
-    }
-
-    @Test
-    public void testMultiAndSingleColumnRelationMix() throws Throwable
-    {
-        for (String compactOption : new String[]{"", " WITH COMPACT STORAGE"})
-        {
-            createTable("CREATE TABLE %s (a int, b int, c int, d int, PRIMARY KEY (a, b, c, d))" + compactOption);
-
-            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 0, 0);
-            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 1, 0);
-            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 1, 1);
-
-            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 0, 0);
-            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 1, 0);
-            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 1, 1);
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and b = ? and (c, d) = (?, ?)", 0, 1, 0, 0),
-                       row(0, 1, 0, 0));
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and b = ? and (c) IN ((?))", 0, 1, 0),
-                       row(0, 1, 0, 0));
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and b = ? and (c) IN ((?), (?))", 0, 1, 0, 1),
-                       row(0, 1, 0, 0),
-                       row(0, 1, 1, 0),
-                       row(0, 1, 1, 1));
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and b = ? and (c, d) IN ((?, ?))", 0, 1, 0, 0),
-                       row(0, 1, 0, 0));
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and b = ? and (c, d) IN ((?, ?), (?, ?))", 0, 1, 0, 0, 1, 1),
-                       row(0, 1, 0, 0),
-                       row(0, 1, 1, 1));
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and b = ? and (c, d) > (?, ?)", 0, 1, 0, 0),
-                       row(0, 1, 1, 0),
-                       row(0, 1, 1, 1));
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and b = ? and (c, d) > (?, ?) and (c) <= (?) ", 0, 1, 0, 0, 1),
-                       row(0, 1, 1, 0),
-                       row(0, 1, 1, 1));
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and b = ? and (c, d) >= (?, ?) and (c, d) < (?, ?)", 0, 1, 0, 0, 1, 1),
-                       row(0, 1, 0, 0),
-                       row(0, 1, 1, 0));
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b, c) = (?, ?) and d = ?", 0, 0, 1, 0),
-                       row(0, 0, 1, 0));
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and b = ? and (c) = (?) and d = ?", 0, 0, 1, 0),
-                       row(0, 0, 1, 0));
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b, c) = (?, ?) and d IN (?, ?)", 0, 0, 1, 0, 2),
-                       row(0, 0, 1, 0));
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and b = ? and (c) = (?) and d IN (?, ?)", 0, 0, 1, 0, 2),
-                       row(0, 0, 1, 0));
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b, c) = (?, ?) and d >= ?", 0, 0, 1, 0),
-                       row(0, 0, 1, 0),
-                       row(0, 0, 1, 1));
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and d < 1 and (b, c) = (?, ?) and d >= ?", 0, 0, 1, 0),
-                       row(0, 0, 1, 0));
-        }
-    }
-
-    @Test
-    public void testMultipleMultiColumnRelation() throws Throwable
-    {
-        for (String compactOption : new String[]{"", " WITH COMPACT STORAGE"})
-        {
-            createTable("CREATE TABLE %s (a int, b int, c int, d int, PRIMARY KEY (a, b, c, d))" + compactOption);
-
-            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 0, 0);
-            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 1, 0);
-            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 1, 1);
-
-            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 0, 0);
-            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 1, 0);
-            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 1, 1);
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b) = (?) and (c, d) = (?, ?)", 0, 1, 0, 0),
-                       row(0, 1, 0, 0));
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b) = (?) and (c) = (?) and (d) = (?)", 0, 1, 0, 0),
-                       row(0, 1, 0, 0));
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b) = (?) and (c) IN ((?))", 0, 1, 0),
-                       row(0, 1, 0, 0));
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b) = (?) and (c) IN ((?), (?))", 0, 1, 0, 1),
-                       row(0, 1, 0, 0),
-                       row(0, 1, 1, 0),
-                       row(0, 1, 1, 1));
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b) = (?) and (c, d) IN ((?, ?))", 0, 1, 0, 0),
-                       row(0, 1, 0, 0));
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b) = (?) and (c, d) IN ((?, ?), (?, ?))", 0, 1, 0, 0, 1, 1),
-                       row(0, 1, 0, 0),
-                       row(0, 1, 1, 1));
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b) = (?) and (c, d) > (?, ?)", 0, 1, 0, 0),
-                       row(0, 1, 1, 0),
-                       row(0, 1, 1, 1));
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b) = (?) and (c, d) > (?, ?) and (c) <= (?) ", 0, 1, 0, 0, 1),
-                       row(0, 1, 1, 0),
-                       row(0, 1, 1, 1));
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b) = (?) and (c, d) >= (?, ?) and (c, d) < (?, ?)", 0, 1, 0, 0, 1, 1),
-                       row(0, 1, 0, 0),
-                       row(0, 1, 1, 0));
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b, c) = (?, ?) and (d) = (?)", 0, 0, 1, 0),
-                       row(0, 0, 1, 0));
-        }
-    }
-
-    @Test
-    public void testSinglePartitionInvalidQueries() throws Throwable
-    {
-        for (String compactOption : new String[]{"", " WITH COMPACT STORAGE"})
-        {
-            createTable("CREATE TABLE %s (a int PRIMARY KEY, b int)" + compactOption);
-
-            assertInvalid("SELECT * FROM %s WHERE (a) > (?)", 0);
-            assertInvalid("SELECT * FROM %s WHERE (a) = (?)", 0);
-            assertInvalid("SELECT * FROM %s WHERE (b) = (?)", 0);
-        }
-    }
-
-    @Test
-    public void testSingleClustering() throws Throwable
-    {
-        for (String compactOption : new String[]{"", " WITH COMPACT STORAGE"})
-        {
-            createTable("CREATE TABLE %s (a int, b int, c int, PRIMARY KEY (a, b))" + compactOption);
-
-            execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, 0, 0);
-            execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, 1, 0);
-            execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, 2, 0);
-
-            // Equalities
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) = (?)", 0, 1),
-                    row(0, 1, 0)
-            );
-
-            // Same but check the whole tuple can be prepared
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) = ?", 0, tuple(1)),
-                    row(0, 1, 0)
-            );
-
-            assertEmpty(execute("SELECT * FROM %s WHERE a = ? AND (b) = (?)", 0, 3));
-
-            // Inequalities
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) > (?)", 0, 0),
-                    row(0, 1, 0),
-                    row(0, 2, 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) >= (?)", 0, 1),
-                    row(0, 1, 0),
-                    row(0, 2, 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) < (?)", 0, 2),
-                    row(0, 0, 0),
-                    row(0, 1, 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) <= (?)", 0, 1),
-                    row(0, 0, 0),
-                    row(0, 1, 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) > (?) AND (b) < (?)", 0, 0, 2),
-                    row(0, 1, 0)
-            );
-        }
-    }
-
-    @Test
-    public void testNonEqualsRelation() throws Throwable
-    {
-        for (String compactOption : new String[]{"", " WITH COMPACT STORAGE"})
-        {
-            createTable("CREATE TABLE %s (a int PRIMARY KEY, b int)" + compactOption);
-            assertInvalid("SELECT * FROM %s WHERE a = 0 AND (b) != (0)");
-        }
-    }
-
-    @Test
-    public void testMultipleClustering() throws Throwable
-    {
-        for (String compactOption : new String[]{"", " WITH COMPACT STORAGE"})
-        {
-            createTable("CREATE TABLE %s (a int, b int, c int, d int, PRIMARY KEY (a, b, c, d))" + compactOption);
-
-            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 0, 0);
-            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 1, 0);
-            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 1, 1);
-
-            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 0, 0);
-            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 1, 0);
-            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 1, 1);
-
-            // Empty query
-            assertEmpty(execute("SELECT * FROM %s WHERE a = 0 AND (b, c, d) IN ()"));
-
-            // Equalities
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) = (?)", 0, 1),
-                    row(0, 1, 0, 0),
-                    row(0, 1, 1, 0),
-                    row(0, 1, 1, 1)
-            );
-
-            // Same with whole tuple prepared
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) = ?", 0, tuple(1)),
-                    row(0, 1, 0, 0),
-                    row(0, 1, 1, 0),
-                    row(0, 1, 1, 1)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c) = (?, ?)", 0, 1, 1),
-                    row(0, 1, 1, 0),
-                    row(0, 1, 1, 1)
-            );
-
-            // Same with whole tuple prepared
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c) = ?", 0, tuple(1, 1)),
-                    row(0, 1, 1, 0),
-                    row(0, 1, 1, 1)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) = (?, ?, ?)", 0, 1, 1, 1),
-                    row(0, 1, 1, 1)
-            );
-
-            // Same with whole tuple prepared
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) = ?", 0, tuple(1, 1, 1)),
-                    row(0, 1, 1, 1)
-            );
-
-            // Inequalities
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) > (?)", 0, 0),
-                    row(0, 1, 0, 0),
-                    row(0, 1, 1, 0),
-                    row(0, 1, 1, 1)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) >= (?)", 0, 0),
-                    row(0, 0, 0, 0),
-                    row(0, 0, 1, 0),
-                    row(0, 0, 1, 1),
-                    row(0, 1, 0, 0),
-                    row(0, 1, 1, 0),
-                    row(0, 1, 1, 1)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c) > (?, ?)", 0, 1, 0),
-                    row(0, 1, 1, 0),
-                    row(0, 1, 1, 1)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c) >= (?, ?)", 0, 1, 0),
-                    row(0, 1, 0, 0),
-                    row(0, 1, 1, 0),
-                    row(0, 1, 1, 1)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) > (?, ?, ?)", 0, 1, 1, 0),
-                    row(0, 1, 1, 1)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) >= (?, ?, ?)", 0, 1, 1, 0),
-                    row(0, 1, 1, 0),
-                    row(0, 1, 1, 1)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) < (?)", 0, 1),
-                    row(0, 0, 0, 0),
-                    row(0, 0, 1, 0),
-                    row(0, 0, 1, 1)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) <= (?)", 0, 1),
-                    row(0, 0, 0, 0),
-                    row(0, 0, 1, 0),
-                    row(0, 0, 1, 1),
-                    row(0, 1, 0, 0),
-                    row(0, 1, 1, 0),
-                    row(0, 1, 1, 1)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c) < (?, ?)", 0, 0, 1),
-                    row(0, 0, 0, 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c) <= (?, ?)", 0, 0, 1),
-                    row(0, 0, 0, 0),
-                    row(0, 0, 1, 0),
-                    row(0, 0, 1, 1)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) < (?, ?, ?)", 0, 0, 1, 1),
-                    row(0, 0, 0, 0),
-                    row(0, 0, 1, 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) <= (?, ?, ?)", 0, 0, 1, 1),
-                    row(0, 0, 0, 0),
-                    row(0, 0, 1, 0),
-                    row(0, 0, 1, 1)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) > (?, ?, ?) AND (b) < (?)", 0, 0, 1, 0, 1),
-                    row(0, 0, 1, 1)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) > (?, ?, ?) AND (b, c) < (?, ?)", 0, 0, 1, 1, 1, 1),
-                    row(0, 1, 0, 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) > (?, ?, ?) AND (b, c, d) < (?, ?, ?)", 0, 0, 1, 1, 1, 1, 0),
-                    row(0, 1, 0, 0)
-            );
-
-            // Same with whole tuple prepared
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) > ? AND (b, c, d) < ?", 0, tuple(0, 1, 1), tuple(1, 1, 0)),
-                    row(0, 1, 0, 0)
-            );
-
-            // reversed
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) > (?) ORDER BY b DESC, c DESC, d DESC", 0, 0),
-                    row(0, 1, 1, 1),
-                    row(0, 1, 1, 0),
-                    row(0, 1, 0, 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) >= (?) ORDER BY b DESC, c DESC, d DESC", 0, 0),
-                    row(0, 1, 1, 1),
-                    row(0, 1, 1, 0),
-                    row(0, 1, 0, 0),
-                    row(0, 0, 1, 1),
-                    row(0, 0, 1, 0),
-                    row(0, 0, 0, 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c) > (?, ?) ORDER BY b DESC, c DESC, d DESC", 0, 1, 0),
-                    row(0, 1, 1, 1),
-                    row(0, 1, 1, 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c) >= (?, ?) ORDER BY b DESC, c DESC, d DESC", 0, 1, 0),
-                    row(0, 1, 1, 1),
-                    row(0, 1, 1, 0),
-                    row(0, 1, 0, 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) > (?, ?, ?) ORDER BY b DESC, c DESC, d DESC", 0, 1, 1, 0),
-                    row(0, 1, 1, 1)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) >= (?, ?, ?) ORDER BY b DESC, c DESC, d DESC", 0, 1, 1, 0),
-                    row(0, 1, 1, 1),
-                    row(0, 1, 1, 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) < (?) ORDER BY b DESC, c DESC, d DESC", 0, 1),
-                    row(0, 0, 1, 1),
-                    row(0, 0, 1, 0),
-                    row(0, 0, 0, 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) <= (?) ORDER BY b DESC, c DESC, d DESC", 0, 1),
-                    row(0, 1, 1, 1),
-                    row(0, 1, 1, 0),
-                    row(0, 1, 0, 0),
-                    row(0, 0, 1, 1),
-                    row(0, 0, 1, 0),
-                    row(0, 0, 0, 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c) < (?, ?) ORDER BY b DESC, c DESC, d DESC", 0, 0, 1),
-                    row(0, 0, 0, 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c) <= (?, ?) ORDER BY b DESC, c DESC, d DESC", 0, 0, 1),
-                    row(0, 0, 1, 1),
-                    row(0, 0, 1, 0),
-                    row(0, 0, 0, 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) < (?, ?, ?) ORDER BY b DESC, c DESC, d DESC", 0, 0, 1, 1),
-                    row(0, 0, 1, 0),
-                    row(0, 0, 0, 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) <= (?, ?, ?) ORDER BY b DESC, c DESC, d DESC", 0, 0, 1, 1),
-                    row(0, 0, 1, 1),
-                    row(0, 0, 1, 0),
-                    row(0, 0, 0, 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) > (?, ?, ?) AND (b) < (?) ORDER BY b DESC, c DESC, d DESC", 0, 0, 1, 0, 1),
-                    row(0, 0, 1, 1)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) > (?, ?, ?) AND (b, c) < (?, ?) ORDER BY b DESC, c DESC, d DESC", 0, 0, 1, 1, 1, 1),
-                    row(0, 1, 0, 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) > (?, ?, ?) AND (b, c, d) < (?, ?, ?) ORDER BY b DESC, c DESC, d DESC", 0, 0, 1, 1, 1, 1, 0),
-                    row(0, 1, 0, 0)
-            );
-
-            // IN
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) IN ((?, ?, ?), (?, ?, ?))", 0, 0, 1, 0, 0, 1, 1),
-                    row(0, 0, 1, 0),
-                    row(0, 0, 1, 1)
-            );
-
-            // same query but with whole tuple prepared
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) IN (?, ?)", 0, tuple(0, 1, 0), tuple(0, 1, 1)),
-                    row(0, 0, 1, 0),
-                    row(0, 0, 1, 1)
-            );
-
-            // same query but with whole IN list prepared
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) IN ?", 0, list(tuple(0, 1, 0), tuple(0, 1, 1))),
-                    row(0, 0, 1, 0),
-                    row(0, 0, 1, 1)
-            );
-
-            // same query, but reversed order for the IN values
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) IN (?, ?)", 0, tuple(0, 1, 1), tuple(0, 1, 0)),
-                    row(0, 0, 1, 0),
-                    row(0, 0, 1, 1)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b, c) IN ((?, ?))", 0, 0, 1),
-                    row(0, 0, 1, 0),
-                    row(0, 0, 1, 1)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b) IN ((?))", 0, 0),
-                    row(0, 0, 0, 0),
-                    row(0, 0, 1, 0),
-                    row(0, 0, 1, 1)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c) IN ((?, ?)) ORDER BY b DESC, c DESC, d DESC", 0, 0, 1),
-                    row(0, 0, 1, 1),
-                    row(0, 0, 1, 0)
-            );
-
-            // IN on both partition key and clustering key
-            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 1, 0, 0, 0);
-            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 1, 0, 1, 0);
-            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 1, 0, 1, 1);
-
-            assertRows(execute("SELECT * FROM %s WHERE a IN (?, ?) AND (b, c, d) IN (?, ?)", 0, 1, tuple(0, 1, 0), tuple(0, 1, 1)),
-                    row(0, 0, 1, 0),
-                    row(0, 0, 1, 1),
-                    row(1, 0, 1, 0),
-                    row(1, 0, 1, 1)
-            );
-
-            // same but with whole IN lists prepared
-            assertRows(execute("SELECT * FROM %s WHERE a IN ? AND (b, c, d) IN ?", list(0, 1), list(tuple(0, 1, 0), tuple(0, 1, 1))),
-                    row(0, 0, 1, 0),
-                    row(0, 0, 1, 1),
-                    row(1, 0, 1, 0),
-                    row(1, 0, 1, 1)
-            );
-
-            // same query, but reversed order for the IN values
-            assertRows(execute("SELECT * FROM %s WHERE a IN (?, ?) AND (b, c, d) IN (?, ?)", 1, 0, tuple(0, 1, 1), tuple(0, 1, 0)),
-                    row(1, 0, 1, 0),
-                    row(1, 0, 1, 1),
-                    row(0, 0, 1, 0),
-                    row(0, 0, 1, 1)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a IN (?, ?) and (b, c) IN ((?, ?))", 0, 1, 0, 1),
-                    row(0, 0, 1, 0),
-                    row(0, 0, 1, 1),
-                    row(1, 0, 1, 0),
-                    row(1, 0, 1, 1)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a IN (?, ?) and (b) IN ((?))", 0, 1, 0),
-                    row(0, 0, 0, 0),
-                    row(0, 0, 1, 0),
-                    row(0, 0, 1, 1),
-                    row(1, 0, 0, 0),
-                    row(1, 0, 1, 0),
-                    row(1, 0, 1, 1)
-            );
-        }
-    }
-
-    @Test
-    public void testMultipleClusteringReversedComponents() throws Throwable
-    {
-        for (String compactOption : new String[]{"", " COMPACT STORAGE AND"})
-        {
-            createTable("CREATE TABLE %s (a int, b int, c int, d int, PRIMARY KEY (a, b, c, d)) WITH" + compactOption + " CLUSTERING ORDER BY (b DESC, c ASC, d DESC)");
-
-            // b and d are reversed in the clustering order
-            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 0, 0);
-            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 1, 1);
-            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 1, 0);
-
-            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 0, 0);
-            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 1, 1);
-            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 1, 0);
-
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) > (?)", 0, 0),
-                    row(0, 1, 0, 0),
-                    row(0, 1, 1, 1),
-                    row(0, 1, 1, 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) >= (?)", 0, 0),
-                    row(0, 1, 0, 0),
-                    row(0, 1, 1, 1),
-                    row(0, 1, 1, 0),
-                    row(0, 0, 0, 0),
-                    row(0, 0, 1, 1),
-                    row(0, 0, 1, 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) < (?)", 0, 1),
-                    row(0, 0, 0, 0),
-                    row(0, 0, 1, 1),
-                    row(0, 0, 1, 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) <= (?)", 0, 1),
-                    row(0, 1, 0, 0),
-                    row(0, 1, 1, 1),
-                    row(0, 1, 1, 0),
-                    row(0, 0, 0, 0),
-                    row(0, 0, 1, 1),
-                    row(0, 0, 1, 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a=? AND (b, c, d) IN ((?, ?, ?), (?, ?, ?))", 0, 1, 1, 1, 0, 1, 1),
-                    row(0, 1, 1, 1),
-                    row(0, 0, 1, 1)
-            );
-
-            // same query, but reversed order for the IN values
-            assertRows(execute("SELECT * FROM %s WHERE a=? AND (b, c, d) IN ((?, ?, ?), (?, ?, ?))", 0, 0, 1, 1, 1, 1, 1),
-                    row(0, 1, 1, 1),
-                    row(0, 0, 1, 1)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) IN (?, ?, ?, ?, ?, ?)",
-                            0, tuple(1, 0, 0), tuple(1, 1, 1), tuple(1, 1, 0), tuple(0, 0, 0), tuple(0, 1, 1), tuple(0, 1, 0)),
-                    row(0, 1, 0, 0),
-                    row(0, 1, 1, 1),
-                    row(0, 1, 1, 0),
-                    row(0, 0, 0, 0),
-                    row(0, 0, 1, 1),
-                    row(0, 0, 1, 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c) IN (?)", 0, tuple(0, 1)),
-                    row(0, 0, 1, 1),
-                    row(0, 0, 1, 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c) IN (?)", 0, tuple(0, 0)),
-                    row(0, 0, 0, 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) IN ((?))", 0, 0),
-                    row(0, 0, 0, 0),
-                    row(0, 0, 1, 1),
-                    row(0, 0, 1, 0)
-            );
-
-            // preserve pre-6875 behavior (even though the query result is technically incorrect)
-            assertEmpty(execute("SELECT * FROM %s WHERE a = ? AND (b, c) > (?, ?)", 0, 1, 0));
-        }
-    }
-
-    @Test
-    public void testMultipleClusteringWithIndex() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a int, b int, c int, d int, e int, PRIMARY KEY (a, b, c, d))");
-        createIndex("CREATE INDEX ON %s (b)");
-        createIndex("CREATE INDEX ON %s (e)");
-        execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 0, 0, 0, 0);
-        execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 0, 1, 0, 1);
-        execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 0, 1, 1, 2);
-        execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 1, 0, 0, 0);
-        execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 1, 1, 0, 1);
-        execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 1, 1, 1, 2);
-        execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 2, 0, 0, 0);
-        assertRows(execute("SELECT * FROM %s WHERE (b) = (?)", 1),
-                   row(0, 1, 0, 0, 0),
-                   row(0, 1, 1, 0, 1),
-                   row(0, 1, 1, 1, 2));
-        assertRows(execute("SELECT * FROM %s WHERE (b, c) = (?, ?) ALLOW FILTERING", 1, 1),
-                   row(0, 1, 1, 0, 1),
-                   row(0, 1, 1, 1, 2));
-        assertRows(execute("SELECT * FROM %s WHERE (b, c) = (?, ?) AND e = ? ALLOW FILTERING", 1, 1, 2),
-                   row(0, 1, 1, 1, 2));
-        assertRows(execute("SELECT * FROM %s WHERE (b) IN ((?)) AND e = ?", 1, 2),
-                   row(0, 1, 1, 1, 2));
-
-        assertRows(execute("SELECT * FROM %s WHERE (b) IN ((?), (?)) AND e = ?", 0, 1, 2),
-                   row(0, 0, 1, 1, 2),
-                   row(0, 1, 1, 1, 2));
-
-        assertRows(execute("SELECT * FROM %s WHERE (b, c) IN ((?, ?)) AND e = ?", 0, 1, 2),
-                   row(0, 0, 1, 1, 2));
-
-        assertRows(execute("SELECT * FROM %s WHERE (b, c) IN ((?, ?), (?, ?)) AND e = ?", 0, 1, 1, 1, 2),
-                   row(0, 0, 1, 1, 2),
-                   row(0, 1, 1, 1, 2));
-
-        assertRows(execute("SELECT * FROM %s WHERE (b) >= (?) AND e = ?", 1, 2),
-                   row(0, 1, 1, 1, 2));
-
-        assertRows(execute("SELECT * FROM %s WHERE (b, c) >= (?, ?) AND e = ?", 1, 1, 2),
-                   row(0, 1, 1, 1, 2));
-    }
-
-    @Test
-    public void testMultiplePartitionKeyAndMultiClusteringWithIndex() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a int, b int, c int, d int, e int, f int, PRIMARY KEY ((a, b), c, d, e))");
-        createIndex("CREATE INDEX ON %s (c)");
-        createIndex("CREATE INDEX ON %s (f)");
-
-        execute("INSERT INTO %s (a, b, c, d, e, f) VALUES (?, ?, ?, ?, ?, ?)", 0, 0, 0, 0, 0, 0);
-        execute("INSERT INTO %s (a, b, c, d, e, f) VALUES (?, ?, ?, ?, ?, ?)", 0, 0, 0, 1, 0, 1);
-        execute("INSERT INTO %s (a, b, c, d, e, f) VALUES (?, ?, ?, ?, ?, ?)", 0, 0, 0, 1, 1, 2);
-
-        execute("INSERT INTO %s (a, b, c, d, e, f) VALUES (?, ?, ?, ?, ?, ?)", 0, 0, 1, 0, 0, 3);
-        execute("INSERT INTO %s (a, b, c, d, e, f) VALUES (?, ?, ?, ?, ?, ?)", 0, 0, 1, 1, 0, 4);
-        execute("INSERT INTO %s (a, b, c, d, e, f) VALUES (?, ?, ?, ?, ?, ?)", 0, 0, 1, 1, 1, 5);
-
-        execute("INSERT INTO %s (a, b, c, d, e, f) VALUES (?, ?, ?, ?, ?, ?)", 0, 0, 2, 0, 0, 5);
-
-        assertRows(execute("SELECT * FROM %s WHERE a = ? AND (c) = (?) ALLOW FILTERING", 0, 1),
-                   row(0, 0, 1, 0, 0, 3),
-                   row(0, 0, 1, 1, 0, 4),
-                   row(0, 0, 1, 1, 1, 5));
-
-        assertRows(execute("SELECT * FROM %s WHERE a = ? AND (c, d) = (?, ?) ALLOW FILTERING", 0, 1, 1),
-                   row(0, 0, 1, 1, 0, 4),
-                   row(0, 0, 1, 1, 1, 5));
-
-        assertInvalidMessage("Partition key part b must be restricted since preceding part is",
-                             "SELECT * FROM %s WHERE a = ? AND (c, d) IN ((?, ?)) ALLOW FILTERING", 0, 1, 1);
-
-        assertInvalidMessage("Partition key part b must be restricted since preceding part is",
-                             "SELECT * FROM %s WHERE a = ? AND (c, d) >= (?, ?) ALLOW FILTERING", 0, 1, 1);
-
-        assertRows(execute("SELECT * FROM %s WHERE a = ? AND (c) IN ((?)) AND f = ? ALLOW FILTERING", 0, 1, 5),
-                   row(0, 0, 1, 1, 1, 5));
-
-        assertRows(execute("SELECT * FROM %s WHERE a = ? AND (c) IN ((?), (?)) AND f = ? ALLOW FILTERING", 0, 1, 2, 5),
-                   row(0, 0, 1, 1, 1, 5),
-                   row(0, 0, 2, 0, 0, 5));
-
-        assertRows(execute("SELECT * FROM %s WHERE a = ? AND (c, d) IN ((?, ?)) AND f = ? ALLOW FILTERING", 0, 1, 0, 3),
-                   row(0, 0, 1, 0, 0, 3));
-
-        assertRows(execute("SELECT * FROM %s WHERE a = ? AND (c) >= (?) AND f = ? ALLOW FILTERING", 0, 1, 5),
-                   row(0, 0, 1, 1, 1, 5),
-                   row(0, 0, 2, 0, 0, 5));
-
-        assertRows(execute("SELECT * FROM %s WHERE a = ? AND (c, d) >= (?, ?) AND f = ? ALLOW FILTERING", 0, 1, 1, 5),
-                   row(0, 0, 1, 1, 1, 5),
-                   row(0, 0, 2, 0, 0, 5));
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f797bfa4/test/unit/org/apache/cassandra/cql3/RangeDeletionTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/RangeDeletionTest.java b/test/unit/org/apache/cassandra/cql3/RangeDeletionTest.java
deleted file mode 100644
index b31d0c2..0000000
--- a/test/unit/org/apache/cassandra/cql3/RangeDeletionTest.java
+++ /dev/null
@@ -1,35 +0,0 @@
-/*
- * 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;
-
-import org.junit.Test;
-
-public class RangeDeletionTest extends CQLTester
-{
-    @Test
-    public void testCassandra8558() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a int, b int, c int, d int, PRIMARY KEY (a, b, c))");
-
-        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 1, 1, 1, 1);
-        flush();
-        execute("DELETE FROM %s WHERE a=? AND b=?", 1, 1);
-        flush();
-        assertEmpty(execute("SELECT * FROM %s WHERE a=? AND b=? AND c=?", 1, 1, 1));
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f797bfa4/test/unit/org/apache/cassandra/cql3/SSTableMetadataTrackingTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/SSTableMetadataTrackingTest.java b/test/unit/org/apache/cassandra/cql3/SSTableMetadataTrackingTest.java
deleted file mode 100644
index 7c3965f..0000000
--- a/test/unit/org/apache/cassandra/cql3/SSTableMetadataTrackingTest.java
+++ /dev/null
@@ -1,160 +0,0 @@
-/*
- * 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;
-
-import org.junit.Test;
-
-import org.apache.cassandra.db.ColumnFamilyStore;
-import org.apache.cassandra.db.Keyspace;
-import org.apache.cassandra.io.sstable.metadata.StatsMetadata;
-import static org.junit.Assert.assertEquals;
-
-public class SSTableMetadataTrackingTest extends CQLTester
-{
-    @Test
-    public void baseCheck() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a int, b int, c text, PRIMARY KEY (a, b))");
-        ColumnFamilyStore cfs = Keyspace.open(keyspace()).getColumnFamilyStore(currentTable());
-        execute("INSERT INTO %s (a,b,c) VALUES (1,1,'1') using timestamp 9999");
-        cfs.forceBlockingFlush();
-        StatsMetadata metadata = cfs.getSSTables().iterator().next().getSSTableMetadata();
-        assertEquals(9999, metadata.minTimestamp);
-        assertEquals(Integer.MAX_VALUE, metadata.maxLocalDeletionTime);
-        cfs.forceMajorCompaction();
-        metadata = cfs.getSSTables().iterator().next().getSSTableMetadata();
-        assertEquals(9999, metadata.minTimestamp);
-        assertEquals(Integer.MAX_VALUE, metadata.maxLocalDeletionTime);
-    }
-
-    @Test
-    public void testMinMaxtimestampRange() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a int, b int, c text, PRIMARY KEY (a, b))");
-        ColumnFamilyStore cfs = Keyspace.open(keyspace()).getColumnFamilyStore(currentTable());
-        execute("INSERT INTO %s (a,b,c) VALUES (1,1,'1') using timestamp 10000");
-        execute("DELETE FROM %s USING TIMESTAMP 9999 WHERE a = 1 and b = 1");
-        cfs.forceBlockingFlush();
-        StatsMetadata metadata = cfs.getSSTables().iterator().next().getSSTableMetadata();
-        assertEquals(9999, metadata.minTimestamp);
-        assertEquals(10000, metadata.maxTimestamp);
-        assertEquals(Integer.MAX_VALUE, metadata.maxLocalDeletionTime, 5);
-        cfs.forceMajorCompaction();
-        metadata = cfs.getSSTables().iterator().next().getSSTableMetadata();
-        assertEquals(9999, metadata.minTimestamp);
-        assertEquals(10000, metadata.maxTimestamp);
-        assertEquals(Integer.MAX_VALUE, metadata.maxLocalDeletionTime, 5);
-    }
-
-    @Test
-    public void testMinMaxtimestampRow() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a int, b int, c text, PRIMARY KEY (a, b))");
-        ColumnFamilyStore cfs = Keyspace.open(keyspace()).getColumnFamilyStore(currentTable());
-        execute("INSERT INTO %s (a,b,c) VALUES (1,1,'1') using timestamp 10000");
-        execute("DELETE FROM %s USING TIMESTAMP 9999 WHERE a = 1");
-        cfs.forceBlockingFlush();
-        StatsMetadata metadata = cfs.getSSTables().iterator().next().getSSTableMetadata();
-        assertEquals(9999, metadata.minTimestamp);
-        assertEquals(10000, metadata.maxTimestamp);
-        assertEquals(Integer.MAX_VALUE, metadata.maxLocalDeletionTime, 5);
-        cfs.forceMajorCompaction();
-        metadata = cfs.getSSTables().iterator().next().getSSTableMetadata();
-        assertEquals(9999, metadata.minTimestamp);
-        assertEquals(10000, metadata.maxTimestamp);
-        assertEquals(Integer.MAX_VALUE, metadata.maxLocalDeletionTime, 5);
-    }
-
-
-    @Test
-    public void testTrackMetadata_rangeTombstone() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a int, b int, c text, PRIMARY KEY (a, b)) WITH gc_grace_seconds = 10000");
-        ColumnFamilyStore cfs = Keyspace.open(keyspace()).getColumnFamilyStore(currentTable());
-        execute("DELETE FROM %s USING TIMESTAMP 9999 WHERE a = 1 and b = 1");
-        cfs.forceBlockingFlush();
-        assertEquals(1, cfs.getSSTables().size());
-        StatsMetadata metadata = cfs.getSSTables().iterator().next().getSSTableMetadata();
-        assertEquals(9999, metadata.minTimestamp);
-        assertEquals(9999, metadata.maxTimestamp);
-        assertEquals(System.currentTimeMillis()/1000, metadata.maxLocalDeletionTime, 5);
-        cfs.forceMajorCompaction();
-        StatsMetadata metadata2 = cfs.getSSTables().iterator().next().getSSTableMetadata();
-        assertEquals(metadata.maxLocalDeletionTime, metadata2.maxLocalDeletionTime);
-        assertEquals(metadata.minTimestamp, metadata2.minTimestamp);
-        assertEquals(metadata.maxTimestamp, metadata2.maxTimestamp);
-    }
-
-    @Test
-    public void testTrackMetadata_rowTombstone() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a int, b int, c text, PRIMARY KEY (a, b))");
-        ColumnFamilyStore cfs = Keyspace.open(keyspace()).getColumnFamilyStore(currentTable());
-        execute("DELETE FROM %s USING TIMESTAMP 9999 WHERE a = 1");
-
-        cfs.forceBlockingFlush();
-        assertEquals(1, cfs.getSSTables().size());
-        StatsMetadata metadata = cfs.getSSTables().iterator().next().getSSTableMetadata();
-        assertEquals(9999, metadata.minTimestamp);
-        assertEquals(9999, metadata.maxTimestamp);
-        assertEquals(System.currentTimeMillis()/1000, metadata.maxLocalDeletionTime, 5);
-        cfs.forceMajorCompaction();
-        StatsMetadata metadata2 = cfs.getSSTables().iterator().next().getSSTableMetadata();
-        assertEquals(metadata.maxLocalDeletionTime, metadata2.maxLocalDeletionTime);
-        assertEquals(metadata.minTimestamp, metadata2.minTimestamp);
-        assertEquals(metadata.maxTimestamp, metadata2.maxTimestamp);
-    }
-
-    @Test
-    public void testTrackMetadata_rowMarker() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a int, PRIMARY KEY (a))");
-        ColumnFamilyStore cfs = Keyspace.open(keyspace()).getColumnFamilyStore(currentTable());
-        execute("INSERT INTO %s (a) VALUES (1) USING TIMESTAMP 9999");
-
-        cfs.forceBlockingFlush();
-        assertEquals(1, cfs.getSSTables().size());
-        StatsMetadata metadata = cfs.getSSTables().iterator().next().getSSTableMetadata();
-        assertEquals(9999, metadata.minTimestamp);
-        assertEquals(9999, metadata.maxTimestamp);
-        assertEquals(Integer.MAX_VALUE, metadata.maxLocalDeletionTime);
-        cfs.forceMajorCompaction();
-        StatsMetadata metadata2 = cfs.getSSTables().iterator().next().getSSTableMetadata();
-        assertEquals(metadata.maxLocalDeletionTime, metadata2.maxLocalDeletionTime);
-        assertEquals(metadata.minTimestamp, metadata2.minTimestamp);
-        assertEquals(metadata.maxTimestamp, metadata2.maxTimestamp);
-    }
-    @Test
-    public void testTrackMetadata_rowMarkerDelete() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a int, PRIMARY KEY (a))");
-        ColumnFamilyStore cfs = Keyspace.open(keyspace()).getColumnFamilyStore(currentTable());
-        execute("DELETE FROM %s USING TIMESTAMP 9999 WHERE a=1");
-        cfs.forceBlockingFlush();
-        assertEquals(1, cfs.getSSTables().size());
-        StatsMetadata metadata = cfs.getSSTables().iterator().next().getSSTableMetadata();
-        assertEquals(9999, metadata.minTimestamp);
-        assertEquals(9999, metadata.maxTimestamp);
-        assertEquals(System.currentTimeMillis()/1000, metadata.maxLocalDeletionTime, 5);
-        cfs.forceMajorCompaction();
-        StatsMetadata metadata2 = cfs.getSSTables().iterator().next().getSSTableMetadata();
-        assertEquals(metadata.maxLocalDeletionTime, metadata2.maxLocalDeletionTime);
-        assertEquals(metadata.minTimestamp, metadata2.minTimestamp);
-        assertEquals(metadata.maxTimestamp, metadata2.maxTimestamp);
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f797bfa4/test/unit/org/apache/cassandra/cql3/SelectWithTokenFunctionTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/SelectWithTokenFunctionTest.java b/test/unit/org/apache/cassandra/cql3/SelectWithTokenFunctionTest.java
deleted file mode 100644
index b2a972b..0000000
--- a/test/unit/org/apache/cassandra/cql3/SelectWithTokenFunctionTest.java
+++ /dev/null
@@ -1,77 +0,0 @@
-/*
- * 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;
-
-import org.junit.Test;
-
-public class SelectWithTokenFunctionTest extends CQLTester
-{
-    @Test
-    public void testTokenFunctionWithSingleColumnPartitionKey() throws Throwable
-    {
-        createTable("CREATE TABLE IF NOT EXISTS %s (a int PRIMARY KEY, b text)");
-        execute("INSERT INTO %s (a, b) VALUES (0, 'a')");
-
-        assertRows(execute("SELECT * FROM %s WHERE token(a) >= token(?)", 0), row(0, "a"));
-        assertRows(execute("SELECT * FROM %s WHERE token(a) >= token(?) and token(a) < token(?)", 0, 1), row(0, "a"));
-        assertInvalid("SELECT * FROM %s WHERE token(a) > token(?)", "a");
-        assertInvalid("SELECT * FROM %s WHERE token(a, b) >= token(?, ?)", "b", 0);
-        assertInvalid("SELECT * FROM %s WHERE token(a) >= token(?) and token(a) >= token(?)", 0, 1);
-        assertInvalid("SELECT * FROM %s WHERE token(a) >= token(?) and token(a) = token(?)", 0, 1);
-        assertInvalidSyntax("SELECT * FROM %s WHERE token(a) = token(?) and token(a) IN (token(?))", 0, 1);
-    }
-
-    @Test
-    public void testTokenFunctionWithPartitionKeyAndClusteringKeyArguments() throws Throwable
-    {
-        createTable("CREATE TABLE IF NOT EXISTS %s (a int, b text, PRIMARY KEY (a, b))");
-        assertInvalid("SELECT * FROM %s WHERE token(a, b) > token(0, 'c')");
-    }
-
-    @Test
-    public void testTokenFunctionWithMultiColumnPartitionKey() throws Throwable
-    {
-        createTable("CREATE TABLE IF NOT EXISTS %s (a int, b text, PRIMARY KEY ((a, b)))");
-        execute("INSERT INTO %s (a, b) VALUES (0, 'a')");
-        execute("INSERT INTO %s (a, b) VALUES (0, 'b')");
-        execute("INSERT INTO %s (a, b) VALUES (0, 'c')");
-
-        assertRows(execute("SELECT * FROM %s WHERE token(a, b) > token(?, ?)", 0, "a"),
-                   row(0, "b"),
-                   row(0, "c"));
-        assertRows(execute("SELECT * FROM %s WHERE token(a, b) > token(?, ?) and token(a, b) < token(?, ?)",
-                           0, "a",
-                           0, "d"),
-                   row(0, "b"),
-                   row(0, "c"));
-        assertInvalid("SELECT * FROM %s WHERE token(a) > token(?) and token(b) > token(?)", 0, "a");
-        assertInvalid("SELECT * FROM %s WHERE token(a) > token(?, ?) and token(a) < token(?, ?) and token(b) > token(?, ?) ", 0, "a", 0, "d", 0, "a");
-        assertInvalid("SELECT * FROM %s WHERE token(b, a) > token(0, 'c')");
-    }
-
-    @Test
-    public void testTokenFunctionWithCompoundPartitionAndClusteringCols() throws Throwable
-    {
-        createTable("CREATE TABLE IF NOT EXISTS %s (a int, b int, c int, d int, PRIMARY KEY ((a, b), c, d))");
-        // just test that the queries don't error
-        execute("SELECT * FROM %s WHERE token(a, b) > token(0, 0) AND c > 10 ALLOW FILTERING;");
-        execute("SELECT * FROM %s WHERE c > 10 AND token(a, b) > token(0, 0) ALLOW FILTERING;");
-        execute("SELECT * FROM %s WHERE token(a, b) > token(0, 0) AND (c, d) > (0, 0) ALLOW FILTERING;");
-        execute("SELECT * FROM %s WHERE (c, d) > (0, 0) AND token(a, b) > token(0, 0) ALLOW FILTERING;");
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f797bfa4/test/unit/org/apache/cassandra/cql3/SelectionOrderingTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/SelectionOrderingTest.java b/test/unit/org/apache/cassandra/cql3/SelectionOrderingTest.java
deleted file mode 100644
index 301aaf4..0000000
--- a/test/unit/org/apache/cassandra/cql3/SelectionOrderingTest.java
+++ /dev/null
@@ -1,233 +0,0 @@
-/*
- * 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;
-
-import org.junit.Test;
-
-public class SelectionOrderingTest extends CQLTester
-{
-
-    @Test
-    public void testNormalSelectionOrderSingleClustering() throws Throwable
-    {
-        for (String descOption : new String[]{"", " WITH CLUSTERING ORDER BY (b DESC)"})
-        {
-            createTable("CREATE TABLE %s (a int, b int, c int, PRIMARY KEY (a, b))" + descOption);
-            execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, 0, 0);
-            execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, 1, 1);
-            execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, 2, 2);
-
-            assertRows(execute("SELECT * FROM %s WHERE a=? ORDER BY b ASC", 0),
-                    row(0, 0, 0),
-                    row(0, 1, 1),
-                    row(0, 2, 2)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a=? ORDER BY b DESC", 0),
-                    row(0, 2, 2),
-                    row(0, 1, 1),
-                    row(0, 0, 0)
-            );
-
-            // order by the only column in the selection
-            assertRows(execute("SELECT b FROM %s WHERE a=? ORDER BY b ASC", 0),
-                    row(0), row(1), row(2));
-
-            assertRows(execute("SELECT b FROM %s WHERE a=? ORDER BY b DESC", 0),
-                    row(2), row(1), row(0));
-
-            // order by a column not in the selection
-            assertRows(execute("SELECT c FROM %s WHERE a=? ORDER BY b ASC", 0),
-                    row(0), row(1), row(2));
-
-            assertRows(execute("SELECT c FROM %s WHERE a=? ORDER BY b DESC", 0),
-                    row(2), row(1), row(0));
-        }
-    }
-
-    @Test
-    public void testFunctionSelectionOrderSingleClustering() throws Throwable
-    {
-        for (String descOption : new String[]{"", " WITH CLUSTERING ORDER BY (b DESC)"})
-        {
-            createTable("CREATE TABLE %s (a int, b int, c int, PRIMARY KEY (a, b))" + descOption);
-            execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, 0, 0);
-            execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, 1, 1);
-            execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, 2, 2);
-
-            // order by the only column in the selection
-            assertRows(execute("SELECT blobAsInt(intAsBlob(b)) FROM %s WHERE a=? ORDER BY b ASC", 0),
-                    row(0), row(1), row(2));
-
-            assertRows(execute("SELECT blobAsInt(intAsBlob(b)) FROM %s WHERE a=? ORDER BY b DESC", 0),
-                    row(2), row(1), row(0));
-
-            // order by a column not in the selection
-            assertRows(execute("SELECT blobAsInt(intAsBlob(c)) FROM %s WHERE a=? ORDER BY b ASC", 0),
-                    row(0), row(1), row(2));
-
-            assertRows(execute("SELECT blobAsInt(intAsBlob(c)) FROM %s WHERE a=? ORDER BY b DESC", 0),
-                    row(2), row(1), row(0));
-
-            assertInvalid("SELECT * FROM %s WHERE a=? ORDER BY c ASC", 0);
-            assertInvalid("SELECT * FROM %s WHERE a=? ORDER BY c DESC", 0);
-        }
-    }
-
-    @Test
-    public void testFieldSelectionOrderSingleClustering() throws Throwable
-    {
-        String type = createType("CREATE TYPE %s (a int)");
-
-        for (String descOption : new String[]{"", " WITH CLUSTERING ORDER BY (b DESC)"})
-        {
-            createTable("CREATE TABLE %s (a int, b int, c frozen<" + type + "   >, PRIMARY KEY (a, b))" + descOption);
-            execute("INSERT INTO %s (a, b, c) VALUES (?, ?, {a: ?})", 0, 0, 0);
-            execute("INSERT INTO %s (a, b, c) VALUES (?, ?, {a: ?})", 0, 1, 1);
-            execute("INSERT INTO %s (a, b, c) VALUES (?, ?, {a: ?})", 0, 2, 2);
-
-            // order by a column not in the selection
-            assertRows(execute("SELECT c.a FROM %s WHERE a=? ORDER BY b ASC", 0),
-                    row(0), row(1), row(2));
-
-            assertRows(execute("SELECT c.a FROM %s WHERE a=? ORDER BY b DESC", 0),
-                    row(2), row(1), row(0));
-
-            assertRows(execute("SELECT blobAsInt(intAsBlob(c.a)) FROM %s WHERE a=? ORDER BY b DESC", 0),
-                    row(2), row(1), row(0));
-            dropTable("DROP TABLE %s");
-        }
-    }
-
-    @Test
-    public void testNormalSelectionOrderMultipleClustering() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a int, b int, c int, d int, PRIMARY KEY (a, b, c))");
-        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 0, 0);
-        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 1, 1);
-        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 2, 2);
-        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 0, 3);
-        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 1, 4);
-        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 2, 5);
-
-        assertRows(execute("SELECT * FROM %s WHERE a=? ORDER BY b ASC", 0),
-                row(0, 0, 0, 0),
-                row(0, 0, 1, 1),
-                row(0, 0, 2, 2),
-                row(0, 1, 0, 3),
-                row(0, 1, 1, 4),
-                row(0, 1, 2, 5)
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE a=? ORDER BY b DESC", 0),
-                row(0, 1, 2, 5),
-                row(0, 1, 1, 4),
-                row(0, 1, 0, 3),
-                row(0, 0, 2, 2),
-                row(0, 0, 1, 1),
-                row(0, 0, 0, 0)
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE a=? ORDER BY b DESC, c DESC", 0),
-                row(0, 1, 2, 5),
-                row(0, 1, 1, 4),
-                row(0, 1, 0, 3),
-                row(0, 0, 2, 2),
-                row(0, 0, 1, 1),
-                row(0, 0, 0, 0)
-        );
-
-        assertInvalid("SELECT * FROM %s WHERE a=? ORDER BY c ASC", 0);
-        assertInvalid("SELECT * FROM %s WHERE a=? ORDER BY c DESC", 0);
-        assertInvalid("SELECT * FROM %s WHERE a=? ORDER BY b ASC, c DESC", 0);
-        assertInvalid("SELECT * FROM %s WHERE a=? ORDER BY b DESC, c ASC", 0);
-        assertInvalid("SELECT * FROM %s WHERE a=? ORDER BY d ASC", 0);
-
-        // select and order by b
-        assertRows(execute("SELECT b FROM %s WHERE a=? ORDER BY b ASC", 0),
-                row(0), row(0), row(0), row(1), row(1), row(1));
-        assertRows(execute("SELECT b FROM %s WHERE a=? ORDER BY b DESC", 0),
-                row(1), row(1), row(1), row(0), row(0), row(0));
-
-        // select c, order by b
-        assertRows(execute("SELECT c FROM %s WHERE a=? ORDER BY b ASC", 0),
-                row(0), row(1), row(2), row(0), row(1), row(2));
-        assertRows(execute("SELECT c FROM %s WHERE a=? ORDER BY b DESC", 0),
-                row(2), row(1), row(0), row(2), row(1), row(0));
-
-        // select c, order by b, c
-        assertRows(execute("SELECT c FROM %s WHERE a=? ORDER BY b ASC, c ASC", 0),
-                row(0), row(1), row(2), row(0), row(1), row(2));
-        assertRows(execute("SELECT c FROM %s WHERE a=? ORDER BY b DESC, c DESC", 0),
-                row(2), row(1), row(0), row(2), row(1), row(0));
-
-        // select d, order by b, c
-        assertRows(execute("SELECT d FROM %s WHERE a=? ORDER BY b ASC, c ASC", 0),
-                row(0), row(1), row(2), row(3), row(4), row(5));
-        assertRows(execute("SELECT d FROM %s WHERE a=? ORDER BY b DESC, c DESC", 0),
-                row(5), row(4), row(3), row(2), row(1), row(0));
-    }
-
-    @Test
-    public void testFunctionSelectionOrderMultipleClustering() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a int, b int, c int, d int, PRIMARY KEY (a, b, c))");
-        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 0, 0);
-        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 1, 1);
-        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 2, 2);
-        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 0, 3);
-        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 1, 4);
-        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 2, 5);
-
-        assertInvalid("SELECT blobAsInt(intAsBlob(b)) FROM %s WHERE a=? ORDER BY c ASC", 0);
-        assertInvalid("SELECT blobAsInt(intAsBlob(b)) FROM %s WHERE a=? ORDER BY c DESC", 0);
-        assertInvalid("SELECT blobAsInt(intAsBlob(b)) FROM %s WHERE a=? ORDER BY b ASC, c DESC", 0);
-        assertInvalid("SELECT blobAsInt(intAsBlob(b)) FROM %s WHERE a=? ORDER BY b DESC, c ASC", 0);
-        assertInvalid("SELECT blobAsInt(intAsBlob(b)) FROM %s WHERE a=? ORDER BY d ASC", 0);
-
-        // select and order by b
-        assertRows(execute("SELECT blobAsInt(intAsBlob(b)) FROM %s WHERE a=? ORDER BY b ASC", 0),
-                row(0), row(0), row(0), row(1), row(1), row(1));
-        assertRows(execute("SELECT blobAsInt(intAsBlob(b)) FROM %s WHERE a=? ORDER BY b DESC", 0),
-                row(1), row(1), row(1), row(0), row(0), row(0));
-
-        assertRows(execute("SELECT b, blobAsInt(intAsBlob(b)) FROM %s WHERE a=? ORDER BY b ASC", 0),
-                row(0, 0), row(0, 0), row(0, 0), row(1, 1), row(1, 1), row(1, 1));
-        assertRows(execute("SELECT b, blobAsInt(intAsBlob(b)) FROM %s WHERE a=? ORDER BY b DESC", 0),
-                row(1, 1), row(1, 1), row(1, 1), row(0, 0), row(0, 0), row(0, 0));
-
-        // select c, order by b
-        assertRows(execute("SELECT blobAsInt(intAsBlob(c)) FROM %s WHERE a=? ORDER BY b ASC", 0),
-                row(0), row(1), row(2), row(0), row(1), row(2));
-        assertRows(execute("SELECT blobAsInt(intAsBlob(c)) FROM %s WHERE a=? ORDER BY b DESC", 0),
-                row(2), row(1), row(0), row(2), row(1), row(0));
-
-        // select c, order by b, c
-        assertRows(execute("SELECT blobAsInt(intAsBlob(c)) FROM %s WHERE a=? ORDER BY b ASC, c ASC", 0),
-                row(0), row(1), row(2), row(0), row(1), row(2));
-        assertRows(execute("SELECT blobAsInt(intAsBlob(c)) FROM %s WHERE a=? ORDER BY b DESC, c DESC", 0),
-                row(2), row(1), row(0), row(2), row(1), row(0));
-
-        // select d, order by b, c
-        assertRows(execute("SELECT blobAsInt(intAsBlob(d)) FROM %s WHERE a=? ORDER BY b ASC, c ASC", 0),
-                row(0), row(1), row(2), row(3), row(4), row(5));
-        assertRows(execute("SELECT blobAsInt(intAsBlob(d)) FROM %s WHERE a=? ORDER BY b DESC, c DESC", 0),
-                row(5), row(4), row(3), row(2), row(1), row(0));
-
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f797bfa4/test/unit/org/apache/cassandra/cql3/SingleColumnRelationTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/SingleColumnRelationTest.java b/test/unit/org/apache/cassandra/cql3/SingleColumnRelationTest.java
deleted file mode 100644
index 604ec60..0000000
--- a/test/unit/org/apache/cassandra/cql3/SingleColumnRelationTest.java
+++ /dev/null
@@ -1,107 +0,0 @@
-/*
- * 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;
-
-import org.junit.Test;
-
-import java.util.ArrayList;
-import java.util.List;
-
-public class SingleColumnRelationTest extends CQLTester
-{
-    @Test
-    public void testInvalidCollectionEqualityRelation() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a int PRIMARY KEY, b set<int>, c list<int>, d map<int, int>)");
-        createIndex("CREATE INDEX ON %s (b)");
-        createIndex("CREATE INDEX ON %s (c)");
-        createIndex("CREATE INDEX ON %s (d)");
-
-        assertInvalid("SELECT * FROM %s WHERE a = 0 AND b=?", set(0));
-        assertInvalid("SELECT * FROM %s WHERE a = 0 AND c=?", list(0));
-        assertInvalid("SELECT * FROM %s WHERE a = 0 AND d=?", map(0, 0));
-    }
-
-    @Test
-    public void testInvalidCollectionNonEQRelation() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a int PRIMARY KEY, b set<int>, c int)");
-        createIndex("CREATE INDEX ON %s (c)");
-        execute("INSERT INTO %s (a, b, c) VALUES (0, {0}, 0)");
-
-        // non-EQ operators
-        assertInvalid("SELECT * FROM %s WHERE c = 0 AND b > ?", set(0));
-        assertInvalid("SELECT * FROM %s WHERE c = 0 AND b >= ?", set(0));
-        assertInvalid("SELECT * FROM %s WHERE c = 0 AND b < ?", set(0));
-        assertInvalid("SELECT * FROM %s WHERE c = 0 AND b <= ?", set(0));
-        assertInvalid("SELECT * FROM %s WHERE c = 0 AND b IN (?)", set(0));
-    }
-
-    @Test
-    public void testLargeClusteringINValues() throws Throwable
-    {
-        createTable("CREATE TABLE %s (k int, c int, v int, PRIMARY KEY (k, c))");
-        execute("INSERT INTO %s (k, c, v) VALUES (0, 0, 0)");
-        List<Integer> inValues = new ArrayList<>(10000);
-        for (int i = 0; i < 10000; i++)
-            inValues.add(i);
-        assertRows(execute("SELECT * FROM %s WHERE k=? AND c IN ?", 0, inValues),
-                row(0, 0, 0)
-        );
-    }
-
-    @Test
-    public void testMultiplePartitionKeyWithIndex() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a int, b int, c int, d int, e int, f int, PRIMARY KEY ((a, b), c, d, e))");
-        createIndex("CREATE INDEX ON %s (c)");
-        createIndex("CREATE INDEX ON %s (f)");
-
-        execute("INSERT INTO %s (a, b, c, d, e, f) VALUES (?, ?, ?, ?, ?, ?)", 0, 0, 0, 0, 0, 0);
-        execute("INSERT INTO %s (a, b, c, d, e, f) VALUES (?, ?, ?, ?, ?, ?)", 0, 0, 0, 1, 0, 1);
-        execute("INSERT INTO %s (a, b, c, d, e, f) VALUES (?, ?, ?, ?, ?, ?)", 0, 0, 0, 1, 1, 2);
-
-        execute("INSERT INTO %s (a, b, c, d, e, f) VALUES (?, ?, ?, ?, ?, ?)", 0, 0, 1, 0, 0, 3);
-        execute("INSERT INTO %s (a, b, c, d, e, f) VALUES (?, ?, ?, ?, ?, ?)", 0, 0, 1, 1, 0, 4);
-        execute("INSERT INTO %s (a, b, c, d, e, f) VALUES (?, ?, ?, ?, ?, ?)", 0, 0, 1, 1, 1, 5);
-
-        execute("INSERT INTO %s (a, b, c, d, e, f) VALUES (?, ?, ?, ?, ?, ?)", 0, 0, 2, 0, 0, 5);
-
-        assertRows(execute("SELECT * FROM %s WHERE a = ? AND c = ? ALLOW FILTERING", 0, 1),
-                   row(0, 0, 1, 0, 0, 3),
-                   row(0, 0, 1, 1, 0, 4),
-                   row(0, 0, 1, 1, 1, 5));
-
-        assertRows(execute("SELECT * FROM %s WHERE a = ? AND c = ? AND d = ? ALLOW FILTERING", 0, 1, 1),
-                   row(0, 0, 1, 1, 0, 4),
-                   row(0, 0, 1, 1, 1, 5));
-
-        assertInvalidMessage("Partition key part b must be restricted since preceding part is",
-                             "SELECT * FROM %s WHERE a = ? AND c >= ? ALLOW FILTERING", 0, 1);
-
-        assertRows(execute("SELECT * FROM %s WHERE a = ? AND c >= ? AND f = ? ALLOW FILTERING", 0, 1, 5),
-                   row(0, 0, 1, 1, 1, 5),
-                   row(0, 0, 2, 0, 0, 5));
-
-        assertRows(execute("SELECT * FROM %s WHERE a = ? AND c = ? AND d >= ? AND f = ? ALLOW FILTERING", 0, 1, 1, 5),
-                   row(0, 0, 1, 1, 1, 5));
-
-        assertInvalidMessage("Cannot execute this query as it might involve data filtering and thus may have unpredictable performance. If you want to execute this query despite the performance unpredictability, use ALLOW FILTERING",
-                             "SELECT * FROM %s WHERE a = ? AND d >= ? AND f = ?", 0, 1, 5);
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f797bfa4/test/unit/org/apache/cassandra/cql3/SliceQueryFilterWithTombstonesTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/SliceQueryFilterWithTombstonesTest.java b/test/unit/org/apache/cassandra/cql3/SliceQueryFilterWithTombstonesTest.java
deleted file mode 100644
index 0cb9819..0000000
--- a/test/unit/org/apache/cassandra/cql3/SliceQueryFilterWithTombstonesTest.java
+++ /dev/null
@@ -1,166 +0,0 @@
-/*
- * 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;
-
-import java.util.concurrent.TimeUnit;
-
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.db.filter.TombstoneOverwhelmingException;
-
-import static junit.framework.Assert.assertTrue;
-import static junit.framework.Assert.fail;
-
-/**
- * Test that TombstoneOverwhelmingException gets thrown when it should be and doesn't when it shouldn't be.
- */
-public class SliceQueryFilterWithTombstonesTest extends CQLTester
-{
-    static final int ORIGINAL_THRESHOLD = DatabaseDescriptor.getTombstoneFailureThreshold();
-    static final int THRESHOLD = 100;
-
-    @BeforeClass
-    public static void setUp() throws Throwable
-    {
-        DatabaseDescriptor.setTombstoneFailureThreshold(THRESHOLD);
-    }
-
-    @AfterClass
-    public static void tearDown()
-    {
-        DatabaseDescriptor.setTombstoneFailureThreshold(ORIGINAL_THRESHOLD);
-    }
-
-    @Test
-    public void testBelowThresholdSelect() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a text, b text, c text, PRIMARY KEY (a, b));");
-
-        // insert exactly the amount of tombstones that shouldn't trigger an exception
-        for (int i = 0; i < THRESHOLD; i++)
-            execute("INSERT INTO %s (a, b, c) VALUES ('key', 'column" + i + "', null);");
-
-        try
-        {
-            execute("SELECT * FROM %s WHERE a = 'key';");
-        }
-        catch (Throwable e)
-        {
-            fail("SELECT with tombstones below the threshold should not have failed, but has: " + e);
-        }
-    }
-
-    @Test
-    public void testBeyondThresholdSelect() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a text, b text, c text, PRIMARY KEY (a, b));");
-
-        // insert exactly the amount of tombstones that *SHOULD* trigger an exception
-        for (int i = 0; i < THRESHOLD + 1; i++)
-            execute("INSERT INTO %s (a, b, c) VALUES ('key', 'column" + i + "', null);");
-
-        try
-        {
-            execute("SELECT * FROM %s WHERE a = 'key';");
-            fail("SELECT with tombstones beyond the threshold should have failed, but hasn't");
-        }
-        catch (Throwable e)
-        {
-            assertTrue(e instanceof TombstoneOverwhelmingException);
-        }
-    }
-
-    @Test
-    public void testAllShadowedSelect() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a text, b text, c text, PRIMARY KEY (a, b));");
-
-        // insert exactly the amount of tombstones that *SHOULD* normally trigger an exception
-        for (int i = 0; i < THRESHOLD + 1; i++)
-            execute("INSERT INTO %s (a, b, c) VALUES ('key', 'column" + i + "', null);");
-
-        // delete all with a partition level tombstone
-        execute("DELETE FROM %s WHERE a = 'key'");
-
-        try
-        {
-            execute("SELECT * FROM %s WHERE a = 'key';");
-        }
-        catch (Throwable e)
-        {
-            fail("SELECT with tombstones shadowed by a partition tombstone should not have failed, but has: " + e);
-        }
-    }
-
-    @Test
-    public void testLiveShadowedCellsSelect() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a text, b text, c text, PRIMARY KEY (a, b));");
-
-        for (int i = 0; i < THRESHOLD + 1; i++)
-            execute("INSERT INTO %s (a, b, c) VALUES ('key', 'column" + i + "', 'column');");
-
-        // delete all with a partition level tombstone
-        execute("DELETE FROM %s WHERE a = 'key'");
-
-        try
-        {
-            execute("SELECT * FROM %s WHERE a = 'key';");
-        }
-        catch (Throwable e)
-        {
-            fail("SELECT with regular cells shadowed by a partition tombstone should not have failed, but has: " + e);
-        }
-    }
-
-    @Test
-    public void testExpiredTombstones() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a text, b text, c text, PRIMARY KEY (a, b)) WITH gc_grace_seconds = 1;");
-
-        for (int i = 0; i < THRESHOLD + 1; i++)
-            execute("INSERT INTO %s (a, b, c) VALUES ('key', 'column" + i + "', null);");
-
-        // not yet past gc grace - must throw a TOE
-        try
-        {
-            execute("SELECT * FROM %s WHERE a = 'key';");
-            fail("SELECT with tombstones beyond the threshold should have failed, but hasn't");
-        }
-        catch (Throwable e)
-        {
-            assertTrue(e instanceof TombstoneOverwhelmingException);
-        }
-
-        // sleep past gc grace
-        TimeUnit.SECONDS.sleep(2);
-
-        // past gc grace - must not throw a TOE now
-        try
-        {
-            execute("SELECT * FROM %s WHERE a = 'key';");
-        }
-        catch (Throwable e)
-        {
-            fail("SELECT with expired tombstones beyond the threshold should not have failed, but has: " + e);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f797bfa4/test/unit/org/apache/cassandra/cql3/StaticColumnsQueryTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/StaticColumnsQueryTest.java b/test/unit/org/apache/cassandra/cql3/StaticColumnsQueryTest.java
deleted file mode 100644
index e27f968..0000000
--- a/test/unit/org/apache/cassandra/cql3/StaticColumnsQueryTest.java
+++ /dev/null
@@ -1,280 +0,0 @@
-/*
- * 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;
-
-import org.junit.Test;
-
-/**
- * Test column ranges and ordering with static column in table
- */
-public class StaticColumnsQueryTest extends CQLTester
-{
-    @Test
-    public void testSingleClustering() throws Throwable
-    {
-        createTable("CREATE TABLE %s (p text, c text, v text, s text static, PRIMARY KEY (p, c))");
-
-        execute("INSERT INTO %s(p, c, v, s) values (?, ?, ?, ?)", "p1", "k1", "v1", "sv1");
-        execute("INSERT INTO %s(p, c, v) values (?, ?, ?)", "p1", "k2", "v2");
-        execute("INSERT INTO %s(p, s) values (?, ?)", "p2", "sv2");
-
-        assertRows(execute("SELECT * FROM %s WHERE p=?", "p1"),
-            row("p1", "k1", "sv1", "v1"),
-            row("p1", "k2", "sv1", "v2")
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE p=?", "p2"),
-            row("p2", null, "sv2", null)
-        );
-
-        // Ascending order
-
-        assertRows(execute("SELECT * FROM %s WHERE p=? ORDER BY c ASC", "p1"),
-            row("p1", "k1", "sv1", "v1"),
-            row("p1", "k2", "sv1", "v2")
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE p=? ORDER BY c ASC", "p2"),
-            row("p2", null, "sv2", null)
-        );
-
-        // Descending order
-
-        assertRows(execute("SELECT * FROM %s WHERE p=? ORDER BY c DESC", "p1"),
-            row("p1", "k2", "sv1", "v2"),
-            row("p1", "k1", "sv1", "v1")
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE p=? ORDER BY c DESC", "p2"),
-            row("p2", null, "sv2", null)
-        );
-
-        // No order with one relation
-
-        assertRows(execute("SELECT * FROM %s WHERE p=? AND c>=?", "p1", "k1"),
-            row("p1", "k1", "sv1", "v1"),
-            row("p1", "k2", "sv1", "v2")
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE p=? AND c>=?", "p1", "k2"),
-            row("p1", "k2", "sv1", "v2")
-        );
-
-        assertEmpty(execute("SELECT * FROM %s WHERE p=? AND c>=?", "p1", "k3"));
-
-        assertRows(execute("SELECT * FROM %s WHERE p=? AND c =?", "p1", "k1"),
-            row("p1", "k1", "sv1", "v1")
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE p=? AND c<=?", "p1", "k1"),
-            row("p1", "k1", "sv1", "v1")
-        );
-
-        assertEmpty(execute("SELECT * FROM %s WHERE p=? AND c<=?", "p1", "k0"));
-
-        // Ascending with one relation
-
-        assertRows(execute("SELECT * FROM %s WHERE p=? AND c>=? ORDER BY c ASC", "p1", "k1"),
-            row("p1", "k1", "sv1", "v1"),
-            row("p1", "k2", "sv1", "v2")
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE p=? AND c>=? ORDER BY c ASC", "p1", "k2"),
-            row("p1", "k2", "sv1", "v2")
-        );
-
-        assertEmpty(execute("SELECT * FROM %s WHERE p=? AND c>=? ORDER BY c ASC", "p1", "k3"));
-
-        assertRows(execute("SELECT * FROM %s WHERE p=? AND c =? ORDER BY c ASC", "p1", "k1"),
-            row("p1", "k1", "sv1", "v1")
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE p=? AND c<=? ORDER BY c ASC", "p1", "k1"),
-            row("p1", "k1", "sv1", "v1")
-        );
-
-        assertEmpty(execute("SELECT * FROM %s WHERE p=? AND c<=? ORDER BY c ASC", "p1", "k0"));
-
-        // Descending with one relation
-
-        assertRows(execute("SELECT * FROM %s WHERE p=? AND c>=? ORDER BY c DESC", "p1", "k1"),
-            row("p1", "k2", "sv1", "v2"),
-            row("p1", "k1", "sv1", "v1")
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE p=? AND c>=? ORDER BY c DESC", "p1", "k2"),
-            row("p1", "k2", "sv1", "v2")
-        );
-
-        assertEmpty(execute("SELECT * FROM %s WHERE p=? AND c>=? ORDER BY c DESC", "p1", "k3"));
-
-        assertRows(execute("SELECT * FROM %s WHERE p=? AND c =? ORDER BY c DESC", "p1", "k1"),
-            row("p1", "k1", "sv1", "v1")
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE p=? AND c<=? ORDER BY c DESC", "p1", "k1"),
-            row("p1", "k1", "sv1", "v1")
-        );
-
-        assertEmpty(execute("SELECT * FROM %s WHERE p=? AND c<=? ORDER BY c DESC", "p1", "k0"));
-
-        // IN
-
-        assertRows(execute("SELECT * FROM %s WHERE p=? AND c IN (?, ?)", "p1", "k1", "k2"),
-            row("p1", "k1", "sv1", "v1"),
-            row("p1", "k2", "sv1", "v2")
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE p=? AND c IN (?, ?) ORDER BY c ASC", "p1", "k1", "k2"),
-            row("p1", "k1", "sv1", "v1"),
-            row("p1", "k2", "sv1", "v2")
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE p=? AND c IN (?, ?) ORDER BY c DESC", "p1", "k1", "k2"),
-            row("p1", "k2", "sv1", "v2"),
-            row("p1", "k1", "sv1", "v1")
-        );
-    }
-
-    @Test
-    public void testSingleClusteringReversed() throws Throwable
-    {
-        createTable("CREATE TABLE %s (p text, c text, v text, s text static, PRIMARY KEY (p, c)) WITH CLUSTERING ORDER BY (c DESC)");
-
-        execute("INSERT INTO %s(p, c, v, s) values (?, ?, ?, ?)", "p1", "k1", "v1", "sv1");
-        execute("INSERT INTO %s(p, c, v) values (?, ?, ?)", "p1", "k2", "v2");
-        execute("INSERT INTO %s(p, s) values (?, ?)", "p2", "sv2");
-
-        assertRows(execute("SELECT * FROM %s WHERE p=?", "p1"),
-            row("p1", "k2", "sv1", "v2"),
-            row("p1", "k1", "sv1", "v1")
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE p=?", "p2"),
-            row("p2", null, "sv2", null)
-        );
-
-        // Ascending order
-
-        assertRows(execute("SELECT * FROM %s WHERE p=? ORDER BY c ASC", "p1"),
-            row("p1", "k1", "sv1", "v1"),
-            row("p1", "k2", "sv1", "v2")
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE p=? ORDER BY c ASC", "p2"),
-            row("p2", null, "sv2", null)
-        );
-
-        // Descending order
-
-        assertRows(execute("SELECT * FROM %s WHERE p=? ORDER BY c DESC", "p1"),
-            row("p1", "k2", "sv1", "v2"),
-            row("p1", "k1", "sv1", "v1")
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE p=? ORDER BY c DESC", "p2"),
-            row("p2", null, "sv2", null)
-        );
-
-        // No order with one relation
-
-        assertRows(execute("SELECT * FROM %s WHERE p=? AND c>=?", "p1", "k1"),
-            row("p1", "k2", "sv1", "v2"),
-            row("p1", "k1", "sv1", "v1")
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE p=? AND c>=?", "p1", "k2"),
-            row("p1", "k2", "sv1", "v2")
-        );
-
-        assertEmpty(execute("SELECT * FROM %s WHERE p=? AND c>=?", "p1", "k3"));
-
-        assertRows(execute("SELECT * FROM %s WHERE p=? AND c=?", "p1", "k1"),
-            row("p1", "k1", "sv1", "v1")
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE p=? AND c<=?", "p1", "k1"),
-            row("p1", "k1", "sv1", "v1")
-        );
-
-        assertEmpty(execute("SELECT * FROM %s WHERE p=? AND c<=?", "p1", "k0"));
-
-        // Ascending with one relation
-
-        assertRows(execute("SELECT * FROM %s WHERE p=? AND c>=? ORDER BY c ASC", "p1", "k1"),
-            row("p1", "k1", "sv1", "v1"),
-            row("p1", "k2", "sv1", "v2")
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE p=? AND c>=? ORDER BY c ASC", "p1", "k2"),
-            row("p1", "k2", "sv1", "v2")
-        );
-
-        assertEmpty(execute("SELECT * FROM %s WHERE p=? AND c>=? ORDER BY c ASC", "p1", "k3"));
-
-        assertRows(execute("SELECT * FROM %s WHERE p=? AND c=? ORDER BY c ASC", "p1", "k1"),
-            row("p1", "k1", "sv1", "v1")
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE p=? AND c<=? ORDER BY c ASC", "p1", "k1"),
-            row("p1", "k1", "sv1", "v1")
-        );
-
-        assertEmpty(execute("SELECT * FROM %s WHERE p=? AND c<=? ORDER BY c ASC", "p1", "k0"));
-
-        // Descending with one relation
-
-        assertRows(execute("SELECT * FROM %s WHERE p=? AND c>=? ORDER BY c DESC", "p1", "k1"),
-            row("p1", "k2", "sv1", "v2"),
-            row("p1", "k1", "sv1", "v1")
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE p=? AND c>=? ORDER BY c DESC", "p1", "k2"),
-            row("p1", "k2", "sv1", "v2")
-        );
-
-        assertEmpty(execute("SELECT * FROM %s WHERE p=? AND c>=? ORDER BY c DESC", "p1", "k3"));
-
-        assertRows(execute("SELECT * FROM %s WHERE p=? AND c=? ORDER BY c DESC", "p1", "k1"),
-            row("p1", "k1", "sv1", "v1")
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE p=? AND c<=? ORDER BY c DESC", "p1", "k1"),
-            row("p1", "k1", "sv1", "v1")
-        );
-
-        assertEmpty(execute("SELECT * FROM %s WHERE p=? AND c<=? ORDER BY c DESC", "p1", "k0"));
-
-        // IN
-
-        assertRows(execute("SELECT * FROM %s WHERE p=? AND c IN (?, ?)", "p1", "k1", "k2"),
-            row("p1", "k2", "sv1", "v2"),
-            row("p1", "k1", "sv1", "v1")
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE p=? AND c IN (?, ?) ORDER BY c ASC", "p1", "k1", "k2"),
-            row("p1", "k1", "sv1", "v1"),
-            row("p1", "k2", "sv1", "v2")
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE p=? AND c IN (?, ?) ORDER BY c DESC", "p1", "k1", "k2"),
-            row("p1", "k2", "sv1", "v2"),
-            row("p1", "k1", "sv1", "v1")
-        );
-    }
-}


[28/32] cassandra git commit: 2.2 commit for CASSANDRA-9160

Posted by jm...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/01115f72/test/unit/org/apache/cassandra/cql3/FrozenCollectionsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/FrozenCollectionsTest.java b/test/unit/org/apache/cassandra/cql3/FrozenCollectionsTest.java
deleted file mode 100644
index cdb7489..0000000
--- a/test/unit/org/apache/cassandra/cql3/FrozenCollectionsTest.java
+++ /dev/null
@@ -1,1101 +0,0 @@
-/*
- * 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;
-
-import org.apache.cassandra.db.marshal.*;
-import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.exceptions.InvalidRequestException;
-import org.apache.cassandra.exceptions.SyntaxException;
-import org.apache.commons.lang3.StringUtils;
-import org.junit.Assert;
-import org.junit.Test;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-
-import static org.junit.Assert.assertEquals;
-
-public class FrozenCollectionsTest extends CQLTester
-{
-    @Test
-    public void testPartitionKeyUsage() throws Throwable
-    {
-        createTable("CREATE TABLE %s (k frozen<set<int>> PRIMARY KEY, v int)");
-
-        execute("INSERT INTO %s (k, v) VALUES (?, ?)", set(), 1);
-        execute("INSERT INTO %s (k, v) VALUES (?, ?)", set(1, 2, 3), 1);
-        execute("INSERT INTO %s (k, v) VALUES (?, ?)", set(4, 5, 6), 0);
-        execute("INSERT INTO %s (k, v) VALUES (?, ?)", set(7, 8, 9), 0);
-
-        // overwrite with an update
-        execute("UPDATE %s SET v=? WHERE k=?", 0, set());
-        execute("UPDATE %s SET v=? WHERE k=?", 0, set(1, 2, 3));
-
-        assertRows(execute("SELECT * FROM %s"),
-            row(set(), 0),
-            row(set(1, 2, 3), 0),
-            row(set(4, 5, 6), 0),
-            row(set(7, 8, 9), 0)
-        );
-
-        assertRows(execute("SELECT k FROM %s"),
-            row(set()),
-            row(set(1, 2, 3)),
-            row(set(4, 5, 6)),
-            row(set(7, 8, 9))
-        );
-
-        assertRows(execute("SELECT * FROM %s LIMIT 2"),
-                row(set(), 0),
-                row(set(1, 2, 3), 0)
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE k=?", set(4, 5, 6)),
-            row(set(4, 5, 6), 0)
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE k=?", set()),
-                row(set(), 0)
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE k IN ?", list(set(4, 5, 6), set())),
-                   row(set(), 0),
-                   row(set(4, 5, 6), 0)
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE token(k) >= token(?)", set(4, 5, 6)),
-                row(set(4, 5, 6), 0),
-                row(set(7, 8, 9), 0)
-        );
-
-        assertInvalid("INSERT INTO %s (k, v) VALUES (null, 0)");
-
-        execute("DELETE FROM %s WHERE k=?", set());
-        execute("DELETE FROM %s WHERE k=?", set(4, 5, 6));
-        assertRows(execute("SELECT * FROM %s"),
-            row(set(1, 2, 3), 0),
-            row(set(7, 8, 9), 0)
-        );
-    }
-
-    @Test
-    public void testNestedPartitionKeyUsage() throws Throwable
-    {
-        createTable("CREATE TABLE %s (k frozen<map<set<int>, list<int>>> PRIMARY KEY, v int)");
-
-        execute("INSERT INTO %s (k, v) VALUES (?, ?)", map(), 1);
-        execute("INSERT INTO %s (k, v) VALUES (?, ?)", map(set(), list(1, 2, 3)), 0);
-        execute("INSERT INTO %s (k, v) VALUES (?, ?)", map(set(1, 2, 3), list(1, 2, 3)), 1);
-        execute("INSERT INTO %s (k, v) VALUES (?, ?)", map(set(4, 5, 6), list(1, 2, 3)), 0);
-        execute("INSERT INTO %s (k, v) VALUES (?, ?)", map(set(7, 8, 9), list(1, 2, 3)), 0);
-
-        // overwrite with an update
-        execute("UPDATE %s SET v=? WHERE k=?", 0, map());
-        execute("UPDATE %s SET v=? WHERE k=?", 0, map(set(1, 2, 3), list(1, 2, 3)));
-
-        assertRows(execute("SELECT * FROM %s"),
-            row(map(), 0),
-            row(map(set(), list(1, 2, 3)), 0),
-            row(map(set(1, 2, 3), list(1, 2, 3)), 0),
-            row(map(set(4, 5, 6), list(1, 2, 3)), 0),
-            row(map(set(7, 8, 9), list(1, 2, 3)), 0)
-        );
-
-        assertRows(execute("SELECT k FROM %s"),
-            row(map()),
-            row(map(set(), list(1, 2, 3))),
-            row(map(set(1, 2, 3), list(1, 2, 3))),
-            row(map(set(4, 5, 6), list(1, 2, 3))),
-            row(map(set(7, 8, 9), list(1, 2, 3)))
-        );
-
-        assertRows(execute("SELECT * FROM %s LIMIT 3"),
-            row(map(), 0),
-            row(map(set(), list(1, 2, 3)), 0),
-            row(map(set(1, 2, 3), list(1, 2, 3)), 0)
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE k=?", map(set(4, 5, 6), list(1, 2, 3))),
-            row(map(set(4, 5, 6), list(1, 2, 3)), 0)
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE k=?", map()),
-                row(map(), 0)
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE k=?", map(set(), list(1, 2, 3))),
-                row(map(set(), list(1, 2, 3)), 0)
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE k IN ?", list(map(set(4, 5, 6), list(1, 2, 3)), map(), map(set(), list(1, 2, 3)))),
-                   row(map(), 0),
-                   row(map(set(), list(1, 2, 3)), 0),
-                   row(map(set(4, 5, 6), list(1, 2, 3)), 0)
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE token(k) >= token(?)", map(set(4, 5, 6), list(1, 2, 3))),
-            row(map(set(4, 5, 6), list(1, 2, 3)), 0),
-            row(map(set(7, 8, 9), list(1, 2, 3)), 0)
-        );
-
-        execute("DELETE FROM %s WHERE k=?", map());
-        execute("DELETE FROM %s WHERE k=?", map(set(), list(1, 2, 3)));
-        execute("DELETE FROM %s WHERE k=?", map(set(4, 5, 6), list(1, 2, 3)));
-        assertRows(execute("SELECT * FROM %s"),
-            row(map(set(1, 2, 3), list(1, 2, 3)), 0),
-            row(map(set(7, 8, 9), list(1, 2, 3)), 0)
-        );
-
-    }
-
-    @Test
-    public void testClusteringKeyUsage() throws Throwable
-    {
-        for (String option : Arrays.asList("", " WITH COMPACT STORAGE"))
-        {
-            createTable("CREATE TABLE %s (a int, b frozen<set<int>>, c int, PRIMARY KEY (a, b))" + option);
-
-            execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, set(), 1);
-            execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, set(1, 2, 3), 1);
-            execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, set(4, 5, 6), 0);
-            execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, set(7, 8, 9), 0);
-
-            // overwrite with an update
-            execute("UPDATE %s SET c=? WHERE a=? AND b=?", 0, 0, set());
-            execute("UPDATE %s SET c=? WHERE a=? AND b=?", 0, 0, set(1, 2, 3));
-
-            assertRows(execute("SELECT * FROM %s"),
-                row(0, set(), 0),
-                row(0, set(1, 2, 3), 0),
-                row(0, set(4, 5, 6), 0),
-                row(0, set(7, 8, 9), 0)
-            );
-
-            assertRows(execute("SELECT b FROM %s"),
-                row(set()),
-                row(set(1, 2, 3)),
-                row(set(4, 5, 6)),
-                row(set(7, 8, 9))
-            );
-
-            assertRows(execute("SELECT * FROM %s LIMIT 2"),
-                row(0, set(), 0),
-                row(0, set(1, 2, 3), 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a=? AND b=?", 0, set(4, 5, 6)),
-                row(0, set(4, 5, 6), 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a=? AND b=?", 0, set()),
-                row(0, set(), 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a=? AND b IN ?", 0, list(set(4, 5, 6), set())),
-                row(0, set(), 0),
-                row(0, set(4, 5, 6), 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a=? AND b > ?", 0, set(4, 5, 6)),
-                row(0, set(7, 8, 9), 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a=? AND b >= ?", 0, set(4, 5, 6)),
-                row(0, set(4, 5, 6), 0),
-                row(0, set(7, 8, 9), 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a=? AND b < ?", 0, set(4, 5, 6)),
-                row(0, set(), 0),
-                row(0, set(1, 2, 3), 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a=? AND b <= ?", 0, set(4, 5, 6)),
-                row(0, set(), 0),
-                row(0, set(1, 2, 3), 0),
-                row(0, set(4, 5, 6), 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a=? AND b > ? AND b <= ?", 0, set(1, 2, 3), set(4, 5, 6)),
-                row(0, set(4, 5, 6), 0)
-            );
-
-            execute("DELETE FROM %s WHERE a=? AND b=?", 0, set());
-            execute("DELETE FROM %s WHERE a=? AND b=?", 0, set(4, 5, 6));
-            assertRows(execute("SELECT * FROM %s"),
-                row(0, set(1, 2, 3), 0),
-                row(0, set(7, 8, 9), 0)
-            );
-        }
-    }
-
-    @Test
-    public void testNestedClusteringKeyUsage() throws Throwable
-    {
-        for (String option : Arrays.asList("", " WITH COMPACT STORAGE"))
-        {
-            createTable("CREATE TABLE %s (a int, b frozen<map<set<int>, list<int>>>, c frozen<set<int>>, d int, PRIMARY KEY (a, b, c))" + option);
-
-            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, map(), set(), 0);
-            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, map(set(), list(1, 2, 3)), set(), 0);
-            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, map(set(1, 2, 3), list(1, 2, 3)), set(1, 2, 3), 0);
-            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, map(set(4, 5, 6), list(1, 2, 3)), set(1, 2, 3), 0);
-            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, map(set(7, 8, 9), list(1, 2, 3)), set(1, 2, 3), 0);
-
-            assertRows(execute("SELECT * FROM %s"),
-                row(0, map(), set(), 0),
-                row(0, map(set(), list(1, 2, 3)), set(), 0),
-                row(0, map(set(1, 2, 3), list(1, 2, 3)), set(1, 2, 3), 0),
-                row(0, map(set(4, 5, 6), list(1, 2, 3)), set(1, 2, 3), 0),
-                row(0, map(set(7, 8, 9), list(1, 2, 3)), set(1, 2, 3), 0)
-            );
-
-            assertRows(execute("SELECT b FROM %s"),
-                row(map()),
-                row(map(set(), list(1, 2, 3))),
-                row(map(set(1, 2, 3), list(1, 2, 3))),
-                row(map(set(4, 5, 6), list(1, 2, 3))),
-                row(map(set(7, 8, 9), list(1, 2, 3)))
-            );
-
-            assertRows(execute("SELECT c FROM %s"),
-                row(set()),
-                row(set()),
-                row(set(1, 2, 3)),
-                row(set(1, 2, 3)),
-                row(set(1, 2, 3))
-            );
-
-            assertRows(execute("SELECT * FROM %s LIMIT 3"),
-                row(0, map(), set(), 0),
-                row(0, map(set(), list(1, 2, 3)), set(), 0),
-                row(0, map(set(1, 2, 3), list(1, 2, 3)), set(1, 2, 3), 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a=0 ORDER BY b DESC LIMIT 4"),
-                row(0, map(set(7, 8, 9), list(1, 2, 3)), set(1, 2, 3), 0),
-                row(0, map(set(4, 5, 6), list(1, 2, 3)), set(1, 2, 3), 0),
-                row(0, map(set(1, 2, 3), list(1, 2, 3)), set(1, 2, 3), 0),
-                row(0, map(set(), list(1, 2, 3)), set(), 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a=? AND b=?", 0, map()),
-                row(0, map(), set(), 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a=? AND b=?", 0, map(set(), list(1, 2, 3))),
-                row(0, map(set(), list(1, 2, 3)), set(), 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a=? AND b=?", 0, map(set(1, 2, 3), list(1, 2, 3))),
-                row(0, map(set(1, 2, 3), list(1, 2, 3)), set(1, 2, 3), 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a=? AND b=? AND c=?", 0, map(set(), list(1, 2, 3)), set()),
-                    row(0, map(set(), list(1, 2, 3)), set(), 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a=? AND (b, c) IN ?", 0, list(tuple(map(set(4, 5, 6), list(1, 2, 3)), set(1, 2, 3)),
-                                                                                     tuple(map(), set()))),
-                row(0, map(), set(), 0),
-                row(0, map(set(4, 5, 6), list(1, 2, 3)), set(1, 2, 3), 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a=? AND b > ?", 0, map(set(4, 5, 6), list(1, 2, 3))),
-                row(0, map(set(7, 8, 9), list(1, 2, 3)), set(1, 2, 3), 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a=? AND b >= ?", 0, map(set(4, 5, 6), list(1, 2, 3))),
-                row(0, map(set(4, 5, 6), list(1, 2, 3)), set(1, 2, 3), 0),
-                row(0, map(set(7, 8, 9), list(1, 2, 3)), set(1, 2, 3), 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a=? AND b < ?", 0, map(set(4, 5, 6), list(1, 2, 3))),
-                row(0, map(), set(), 0),
-                row(0, map(set(), list(1, 2, 3)), set(), 0),
-                row(0, map(set(1, 2, 3), list(1, 2, 3)), set(1, 2, 3), 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a=? AND b <= ?", 0, map(set(4, 5, 6), list(1, 2, 3))),
-                row(0, map(), set(), 0),
-                row(0, map(set(), list(1, 2, 3)), set(), 0),
-                row(0, map(set(1, 2, 3), list(1, 2, 3)), set(1, 2, 3), 0),
-                row(0, map(set(4, 5, 6), list(1, 2, 3)), set(1, 2, 3), 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a=? AND b > ? AND b <= ?", 0, map(set(1, 2, 3), list(1, 2, 3)), map(set(4, 5, 6), list(1, 2, 3))),
-                row(0, map(set(4, 5, 6), list(1, 2, 3)), set(1, 2, 3), 0)
-            );
-
-            execute("DELETE FROM %s WHERE a=? AND b=? AND c=?", 0, map(), set());
-            assertEmpty(execute("SELECT * FROM %s WHERE a=? AND b=? AND c=?", 0, map(), set()));
-
-            execute("DELETE FROM %s WHERE a=? AND b=? AND c=?", 0, map(set(), list(1, 2, 3)), set());
-            assertEmpty(execute("SELECT * FROM %s WHERE a=? AND b=? AND c=?", 0, map(set(), list(1, 2, 3)), set()));
-
-            execute("DELETE FROM %s WHERE a=? AND b=? AND c=?", 0, map(set(4, 5, 6), list(1, 2, 3)), set(1, 2, 3));
-            assertEmpty(execute("SELECT * FROM %s WHERE a=? AND b=? AND c=?", 0, map(set(4, 5, 6), list(1, 2, 3)), set(1, 2, 3)));
-
-            assertRows(execute("SELECT * FROM %s"),
-                    row(0, map(set(1, 2, 3), list(1, 2, 3)), set(1, 2, 3), 0),
-                    row(0, map(set(7, 8, 9), list(1, 2, 3)), set(1, 2, 3), 0)
-            );
-        }
-    }
-
-    @Test
-    public void testNormalColumnUsage() throws Throwable
-    {
-        for (String option : Arrays.asList("", " WITH COMPACT STORAGE"))
-        {
-            createTable("CREATE TABLE %s (a int PRIMARY KEY, b frozen<map<set<int>, list<int>>>, c frozen<set<int>>)" + option);
-
-            execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, map(), set());
-            execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 1, map(set(), list(99999, 999999, 99999)), set());
-            execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 2, map(set(1, 2, 3), list(1, 2, 3)), set(1, 2, 3));
-            execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 3, map(set(4, 5, 6), list(1, 2, 3)), set(1, 2, 3));
-            execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 4, map(set(7, 8, 9), list(1, 2, 3)), set(1, 2, 3));
-
-            // overwrite with update
-            execute ("UPDATE %s SET b=? WHERE a=?", map(set(), list(1, 2, 3)), 1);
-
-            assertRows(execute("SELECT * FROM %s"),
-                row(0, map(), set()),
-                row(1, map(set(), list(1, 2, 3)), set()),
-                row(2, map(set(1, 2, 3), list(1, 2, 3)), set(1, 2, 3)),
-                row(3, map(set(4, 5, 6), list(1, 2, 3)), set(1, 2, 3)),
-                row(4, map(set(7, 8, 9), list(1, 2, 3)), set(1, 2, 3))
-            );
-
-            assertRows(execute("SELECT b FROM %s"),
-                row(map()),
-                row(map(set(), list(1, 2, 3))),
-                row(map(set(1, 2, 3), list(1, 2, 3))),
-                row(map(set(4, 5, 6), list(1, 2, 3))),
-                row(map(set(7, 8, 9), list(1, 2, 3)))
-            );
-
-            assertRows(execute("SELECT c FROM %s"),
-                row(set()),
-                row(set()),
-                row(set(1, 2, 3)),
-                row(set(1, 2, 3)),
-                row(set(1, 2, 3))
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a=?", 3),
-                row(3, map(set(4, 5, 6), list(1, 2, 3)), set(1, 2, 3))
-            );
-
-            execute("UPDATE %s SET b=? WHERE a=?", null, 1);
-            assertRows(execute("SELECT * FROM %s WHERE a=?", 1),
-                row(1, null, set())
-            );
-
-            execute("UPDATE %s SET b=? WHERE a=?", map(), 1);
-            assertRows(execute("SELECT * FROM %s WHERE a=?", 1),
-                row(1, map(), set())
-            );
-
-            execute("UPDATE %s SET c=? WHERE a=?", null, 2);
-            assertRows(execute("SELECT * FROM %s WHERE a=?", 2),
-                row(2, map(set(1, 2, 3), list(1, 2, 3)), null)
-            );
-
-            execute("UPDATE %s SET c=? WHERE a=?", set(), 2);
-            assertRows(execute("SELECT * FROM %s WHERE a=?", 2),
-                    row(2, map(set(1, 2, 3), list(1, 2, 3)), set())
-            );
-
-            execute("DELETE b FROM %s WHERE a=?", 3);
-            assertRows(execute("SELECT * FROM %s WHERE a=?", 3),
-                row(3, null, set(1, 2, 3))
-            );
-
-            execute("DELETE c FROM %s WHERE a=?", 4);
-            assertRows(execute("SELECT * FROM %s WHERE a=?", 4),
-                row(4, map(set(7, 8, 9), list(1, 2, 3)), null)
-            );
-        }
-    }
-
-    @Test
-    public void testStaticColumnUsage() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a int, b int, c frozen<map<set<int>, list<int>>> static, d int, PRIMARY KEY (a, b))");
-        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, map(), 0);
-        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, map(), 0);
-        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 1, 0, map(set(), list(1, 2, 3)), 0);
-        execute("INSERT INTO %s (a, b, d) VALUES (?, ?, ?)", 1, 1, 0);
-        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 2, 0, map(set(1, 2, 3), list(1, 2, 3)), 0);
-
-        assertRows(execute("SELECT * FROM %s"),
-            row(0, 0, map(), 0),
-            row(0, 1, map(), 0),
-            row(1, 0, map(set(), list(1, 2, 3)), 0),
-            row(1, 1, map(set(), list(1, 2, 3)), 0),
-            row(2, 0, map(set(1, 2, 3), list(1, 2, 3)), 0)
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE a=? AND b=?", 0, 1),
-            row(0, 1, map(), 0)
-        );
-
-        execute("DELETE c FROM %s WHERE a=?", 0);
-        assertRows(execute("SELECT * FROM %s"),
-                row(0, 0, null, 0),
-                row(0, 1, null, 0),
-                row(1, 0, map(set(), list(1, 2, 3)), 0),
-                row(1, 1, map(set(), list(1, 2, 3)), 0),
-                row(2, 0, map(set(1, 2, 3), list(1, 2, 3)), 0)
-        );
-
-        execute("DELETE FROM %s WHERE a=?", 0);
-        assertRows(execute("SELECT * FROM %s"),
-                row(1, 0, map(set(), list(1, 2, 3)), 0),
-                row(1, 1, map(set(), list(1, 2, 3)), 0),
-                row(2, 0, map(set(1, 2, 3), list(1, 2, 3)), 0)
-        );
-
-        execute("UPDATE %s SET c=? WHERE a=?", map(set(1, 2, 3), list(1, 2, 3)), 1);
-        assertRows(execute("SELECT * FROM %s"),
-                row(1, 0, map(set(1, 2, 3), list(1, 2, 3)), 0),
-                row(1, 1, map(set(1, 2, 3), list(1, 2, 3)), 0),
-                row(2, 0, map(set(1, 2, 3), list(1, 2, 3)), 0)
-        );
-    }
-
-    private void assertInvalidCreateWithMessage(String createTableStatement, String errorMessage) throws Throwable
-    {
-         try
-        {
-            createTableMayThrow(createTableStatement);
-            Assert.fail("Expected CREATE TABLE statement to error: " + createTableStatement);
-        }
-        catch (InvalidRequestException | ConfigurationException | SyntaxException ex)
-        {
-            Assert.assertTrue("Expected error message to contain '" + errorMessage + "', but got '" + ex.getMessage() + "'",
-                    ex.getMessage().contains(errorMessage));
-        }
-    }
-
-    private void assertInvalidAlterWithMessage(String createTableStatement, String errorMessage) throws Throwable
-    {
-        try
-        {
-            alterTableMayThrow(createTableStatement);
-            Assert.fail("Expected CREATE TABLE statement to error: " + createTableStatement);
-        }
-        catch (InvalidRequestException | ConfigurationException ex)
-        {
-            Assert.assertTrue("Expected error message to contain '" + errorMessage + "', but got '" + ex.getMessage() + "'",
-                    ex.getMessage().contains(errorMessage));
-        }
-    }
-
-    @Test
-    public void testInvalidOperations() throws Throwable
-    {
-        // lists
-        createTable("CREATE TABLE %s (k int PRIMARY KEY, l frozen<list<int>>)");
-        assertInvalid("UPDATE %s SET l[?]=? WHERE k=?", 0, 0, 0);
-        assertInvalid("UPDATE %s SET l = ? + l WHERE k=?", list(0), 0);
-        assertInvalid("UPDATE %s SET l = l + ? WHERE k=?", list(4), 0);
-        assertInvalid("UPDATE %s SET l = l - ? WHERE k=?", list(3), 0);
-        assertInvalid("DELETE l[?] FROM %s WHERE k=?", 0, 0);
-
-        // sets
-        createTable("CREATE TABLE %s (k int PRIMARY KEY, s frozen<set<int>>)");
-        assertInvalid("UPDATE %s SET s = s + ? WHERE k=?", set(0), 0);
-        assertInvalid("UPDATE %s SET s = s - ? WHERE k=?", set(3), 0);
-
-        // maps
-        createTable("CREATE TABLE %s (k int PRIMARY KEY, m frozen<map<int, int>>)");
-        assertInvalid("UPDATE %s SET m[?]=? WHERE k=?", 0, 0, 0);
-        assertInvalid("UPDATE %s SET m = m + ? WHERE k=?", map(4, 4), 0);
-        assertInvalid("DELETE m[?] FROM %s WHERE k=?", 0, 0);
-
-        assertInvalidCreateWithMessage("CREATE TABLE %s (k int PRIMARY KEY, t set<set<int>>)",
-                "Non-frozen collections are not allowed inside collections");
-
-        assertInvalidCreateWithMessage("CREATE TABLE %s (k int PRIMARY KEY, t frozen<set<counter>>)",
-                                       "Counters are not allowed inside collections");
-
-        assertInvalidCreateWithMessage("CREATE TABLE %s (k int PRIMARY KEY, t frozen<text>)",
-                "frozen<> is only allowed on collections, tuples, and user-defined types");
-    }
-
-    @Test
-    public void testAltering() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a int, b frozen<list<int>>, c frozen<list<int>>, PRIMARY KEY (a, b))");
-
-        alterTable("ALTER TABLE %s ALTER c TYPE frozen<list<blob>>");
-
-        assertInvalidAlterWithMessage("ALTER TABLE %s ALTER b TYPE frozen<list<blob>>",
-                                      "types are not order-compatible");
-
-        assertInvalidAlterWithMessage("ALTER TABLE %s ALTER b TYPE list<int>",
-                                      "types are not order-compatible");
-
-        assertInvalidAlterWithMessage("ALTER TABLE %s ALTER c TYPE list<blob>",
-                                      "types are incompatible");
-
-        alterTable("ALTER TABLE %s DROP c");
-        alterTable("ALTER TABLE %s ADD c frozen<set<int>>");
-        assertInvalidAlterWithMessage("ALTER TABLE %s ALTER c TYPE frozen<set<blob>>",
-                                      "types are incompatible");
-
-        alterTable("ALTER TABLE %s DROP c");
-        alterTable("ALTER TABLE %s ADD c frozen<map<int, int>>");
-        assertInvalidAlterWithMessage("ALTER TABLE %s ALTER c TYPE frozen<map<blob, int>>",
-                                      "types are incompatible");
-        alterTable("ALTER TABLE %s ALTER c TYPE frozen<map<int, blob>>");
-    }
-
-    private void assertInvalidIndexCreationWithMessage(String statement, String errorMessage) throws Throwable
-    {
-        try
-        {
-            createIndexMayThrow(statement);
-            Assert.fail("Expected index creation to fail: " + statement);
-        }
-        catch (InvalidRequestException ex)
-        {
-            Assert.assertTrue("Expected error message to contain '" + errorMessage + "', but got '" + ex.getMessage() + "'",
-                              ex.getMessage().contains(errorMessage));
-        }
-    }
-
-    @Test
-    public void testSecondaryIndex() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a frozen<map<int, text>> PRIMARY KEY, b frozen<map<int, text>>)");
-
-        // for now, we don't support indexing values or keys of collections in the primary key
-        assertInvalidIndexCreationWithMessage("CREATE INDEX ON %s (full(a))", "Cannot create secondary index on partition key column");
-        assertInvalidIndexCreationWithMessage("CREATE INDEX ON %s (keys(a))", "Cannot create index on keys of frozen<map> column");
-        assertInvalidIndexCreationWithMessage("CREATE INDEX ON %s (keys(b))", "Cannot create index on keys of frozen<map> column");
-
-        createTable("CREATE TABLE %s (a int, b frozen<list<int>>, c frozen<set<int>>, d frozen<map<int, text>>, PRIMARY KEY (a, b))");
-
-        createIndex("CREATE INDEX ON %s (full(b))");
-        createIndex("CREATE INDEX ON %s (full(c))");
-        createIndex("CREATE INDEX ON %s (full(d))");
-
-        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, list(1, 2, 3), set(1, 2, 3), map(1, "a"));
-        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, list(4, 5, 6), set(1, 2, 3), map(1, "a"));
-        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 1, list(1, 2, 3), set(4, 5, 6), map(2, "b"));
-        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 1, list(4, 5, 6), set(4, 5, 6), map(2, "b"));
-
-        // CONTAINS KEY doesn't work on non-maps
-        assertInvalidMessage("Cannot use CONTAINS KEY on non-map column",
-                             "SELECT * FROM %s WHERE b CONTAINS KEY ?", 1);
-
-        assertInvalidMessage("Cannot use CONTAINS KEY on non-map column",
-                             "SELECT * FROM %s WHERE b CONTAINS KEY ? ALLOW FILTERING", 1);
-
-        assertInvalidMessage("Cannot use CONTAINS KEY on non-map column",
-                             "SELECT * FROM %s WHERE c CONTAINS KEY ?", 1);
-
-        // normal indexes on frozen collections don't support CONTAINS or CONTAINS KEY
-        assertInvalidMessage("Cannot restrict clustering columns by a CONTAINS relation without a secondary index",
-                             "SELECT * FROM %s WHERE b CONTAINS ?", 1);
-
-        assertInvalidMessage("Cannot restrict clustering columns by a CONTAINS relation without a secondary index",
-                             "SELECT * FROM %s WHERE b CONTAINS ? ALLOW FILTERING", 1);
-
-        assertInvalidMessage("No secondary indexes on the restricted columns support the provided operator",
-                             "SELECT * FROM %s WHERE d CONTAINS KEY ?", 1);
-
-        assertInvalidMessage("No secondary indexes on the restricted columns support the provided operator",
-                             "SELECT * FROM %s WHERE d CONTAINS KEY ? ALLOW FILTERING", 1);
-
-        assertInvalidMessage("Cannot restrict clustering columns by a CONTAINS relation without a secondary index",
-                             "SELECT * FROM %s WHERE b CONTAINS ? AND d CONTAINS KEY ? ALLOW FILTERING", 1, 1);
-
-        // index lookup on b
-        assertRows(execute("SELECT * FROM %s WHERE b=?", list(1, 2, 3)),
-            row(0, list(1, 2, 3), set(1, 2, 3), map(1, "a")),
-            row(1, list(1, 2, 3), set(4, 5, 6), map(2, "b"))
-        );
-
-        assertEmpty(execute("SELECT * FROM %s WHERE b=?", list(-1)));
-
-        assertInvalidMessage("ALLOW FILTERING", "SELECT * FROM %s WHERE b=? AND c=?", list(1, 2, 3), set(4, 5, 6));
-        assertRows(execute("SELECT * FROM %s WHERE b=? AND c=? ALLOW FILTERING", list(1, 2, 3), set(4, 5, 6)),
-            row(1, list(1, 2, 3), set(4, 5, 6), map(2, "b"))
-        );
-
-        assertInvalidMessage("ALLOW FILTERING", "SELECT * FROM %s WHERE b=? AND c CONTAINS ?", list(1, 2, 3), 5);
-        assertRows(execute("SELECT * FROM %s WHERE b=? AND c CONTAINS ? ALLOW FILTERING", list(1, 2, 3), 5),
-            row(1, list(1, 2, 3), set(4, 5, 6), map(2, "b"))
-        );
-
-        assertInvalidMessage("ALLOW FILTERING", "SELECT * FROM %s WHERE b=? AND d=?", list(1, 2, 3), map(1, "a"));
-        assertRows(execute("SELECT * FROM %s WHERE b=? AND d=? ALLOW FILTERING", list(1, 2, 3), map(1, "a")),
-            row(0, list(1, 2, 3), set(1, 2, 3), map(1, "a"))
-        );
-
-        assertInvalidMessage("ALLOW FILTERING", "SELECT * FROM %s WHERE b=? AND d CONTAINS ?", list(1, 2, 3), "a");
-        assertRows(execute("SELECT * FROM %s WHERE b=? AND d CONTAINS ? ALLOW FILTERING", list(1, 2, 3), "a"),
-            row(0, list(1, 2, 3), set(1, 2, 3), map(1, "a"))
-        );
-
-        assertInvalidMessage("ALLOW FILTERING", "SELECT * FROM %s WHERE b=? AND d CONTAINS KEY ?", list(1, 2, 3), 1);
-        assertRows(execute("SELECT * FROM %s WHERE b=? AND d CONTAINS KEY ? ALLOW FILTERING", list(1, 2, 3), 1),
-            row(0, list(1, 2, 3), set(1, 2, 3), map(1, "a"))
-        );
-
-        // index lookup on c
-        assertRows(execute("SELECT * FROM %s WHERE c=?", set(1, 2, 3)),
-            row(0, list(1, 2, 3), set(1, 2, 3), map(1, "a")),
-            row(0, list(4, 5, 6), set(1, 2, 3), map(1, "a"))
-        );
-
-        // ordering of c should not matter
-        assertRows(execute("SELECT * FROM %s WHERE c=?", set(2, 1, 3)),
-                row(0, list(1, 2, 3), set(1, 2, 3), map(1, "a")),
-                row(0, list(4, 5, 6), set(1, 2, 3), map(1, "a"))
-        );
-
-        assertEmpty(execute("SELECT * FROM %s WHERE c=?", set(-1)));
-
-        assertInvalidMessage("ALLOW FILTERING", "SELECT * FROM %s WHERE c=? AND b=?", set(1, 2, 3), list(1, 2, 3));
-        assertRows(execute("SELECT * FROM %s WHERE c=? AND b=? ALLOW FILTERING", set(1, 2, 3), list(1, 2, 3)),
-            row(0, list(1, 2, 3), set(1, 2, 3), map(1, "a"))
-        );
-
-        assertInvalidMessage("ALLOW FILTERING", "SELECT * FROM %s WHERE c=? AND b CONTAINS ?", set(1, 2, 3), 1);
-        assertRows(execute("SELECT * FROM %s WHERE c=? AND b CONTAINS ? ALLOW FILTERING", set(1, 2, 3), 1),
-            row(0, list(1, 2, 3), set(1, 2, 3), map(1, "a"))
-        );
-
-        assertInvalidMessage("ALLOW FILTERING", "SELECT * FROM %s WHERE c=? AND d = ?", set(1, 2, 3), map(1, "a"));
-        assertRows(execute("SELECT * FROM %s WHERE c=? AND d = ? ALLOW FILTERING", set(1, 2, 3), map(1, "a")),
-            row(0, list(1, 2, 3), set(1, 2, 3), map(1, "a")),
-            row(0, list(4, 5, 6), set(1, 2, 3), map(1, "a"))
-        );
-
-        assertInvalidMessage("ALLOW FILTERING", "SELECT * FROM %s WHERE c=? AND d CONTAINS ?", set(1, 2, 3), "a");
-        assertRows(execute("SELECT * FROM %s WHERE c=? AND d CONTAINS ? ALLOW FILTERING", set(1, 2, 3), "a"),
-            row(0, list(1, 2, 3), set(1, 2, 3), map(1, "a")),
-            row(0, list(4, 5, 6), set(1, 2, 3), map(1, "a"))
-        );
-
-        assertInvalidMessage("ALLOW FILTERING", "SELECT * FROM %s WHERE c=? AND d CONTAINS KEY ?", set(1, 2, 3), 1);
-        assertRows(execute("SELECT * FROM %s WHERE c=? AND d CONTAINS KEY ? ALLOW FILTERING", set(1, 2, 3), 1),
-            row(0, list(1, 2, 3), set(1, 2, 3), map(1, "a")),
-            row(0, list(4, 5, 6), set(1, 2, 3), map(1, "a"))
-        );
-
-        // index lookup on d
-        assertRows(execute("SELECT * FROM %s WHERE d=?", map(1, "a")),
-            row(0, list(1, 2, 3), set(1, 2, 3), map(1, "a")),
-            row(0, list(4, 5, 6), set(1, 2, 3), map(1, "a"))
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE d=?", map(2, "b")),
-            row(1, list(1, 2, 3), set(4, 5, 6), map(2, "b")),
-            row(1, list(4, 5, 6), set(4, 5, 6), map(2, "b"))
-        );
-
-        assertEmpty(execute("SELECT * FROM %s WHERE d=?", map(3, "c")));
-
-        assertInvalidMessage("ALLOW FILTERING", "SELECT * FROM %s WHERE d=? AND c=?", map(1, "a"), set(1, 2, 3));
-        assertRows(execute("SELECT * FROM %s WHERE d=? AND b=? ALLOW FILTERING", map(1, "a"), list(1, 2, 3)),
-            row(0, list(1, 2, 3), set(1, 2, 3), map(1, "a"))
-        );
-
-        assertInvalidMessage("ALLOW FILTERING", "SELECT * FROM %s WHERE d=? AND b CONTAINS ?", map(1, "a"), 3);
-        assertRows(execute("SELECT * FROM %s WHERE d=? AND b CONTAINS ? ALLOW FILTERING", map(1, "a"), 3),
-            row(0, list(1, 2, 3), set(1, 2, 3), map(1, "a"))
-        );
-
-        assertInvalidMessage("ALLOW FILTERING", "SELECT * FROM %s WHERE d=? AND b=? AND c=?", map(1, "a"), list(1, 2, 3), set(1, 2, 3));
-        assertRows(execute("SELECT * FROM %s WHERE d=? AND b=? AND c=? ALLOW FILTERING", map(1, "a"), list(1, 2, 3), set(1, 2, 3)),
-            row(0, list(1, 2, 3), set(1, 2, 3), map(1, "a"))
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE d=? AND b CONTAINS ? AND c CONTAINS ? ALLOW FILTERING", map(1, "a"), 2, 2),
-            row(0, list(1, 2, 3), set(1, 2, 3), map(1, "a"))
-        );
-
-        execute("DELETE d FROM %s WHERE a=? AND b=?", 0, list(1, 2, 3));
-        assertRows(execute("SELECT * FROM %s WHERE d=?", map(1, "a")),
-            row(0, list(4, 5, 6), set(1, 2, 3), map(1, "a"))
-        );
-    }
-
-    /** Test for CASSANDRA-8302 */
-    @Test
-    public void testClusteringColumnFiltering() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a int, b frozen<map<int, int>>, c int, d int, PRIMARY KEY (a, b, c))");
-        createIndex("CREATE INDEX c_index ON %s (c)");
-        createIndex("CREATE INDEX d_index ON %s (d)");
-
-        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, map(0, 0, 1, 1), 0, 0);
-        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, map(1, 1, 2, 2), 0, 0);
-        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 1, map(0, 0, 1, 1), 0, 0);
-        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 1, map(1, 1, 2, 2), 0, 0);
-
-        assertRows(execute("SELECT * FROM %s WHERE d=? AND b CONTAINS ? ALLOW FILTERING", 0, 0),
-                row(0, map(0, 0, 1, 1), 0, 0),
-                row(1, map(0, 0, 1, 1), 0, 0)
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE d=? AND b CONTAINS KEY ? ALLOW FILTERING", 0, 0),
-                row(0, map(0, 0, 1, 1), 0, 0),
-                row(1, map(0, 0, 1, 1), 0, 0)
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE a=? AND d=? AND b CONTAINS ? ALLOW FILTERING", 0, 0, 0),
-                row(0, map(0, 0, 1, 1), 0, 0)
-        );
-        assertRows(execute("SELECT * FROM %s WHERE a=? AND d=? AND b CONTAINS KEY ? ALLOW FILTERING", 0, 0, 0),
-                row(0, map(0, 0, 1, 1), 0, 0)
-        );
-
-        dropIndex("DROP INDEX %s.d_index");
-
-        assertRows(execute("SELECT * FROM %s WHERE c=? AND b CONTAINS ? ALLOW FILTERING", 0, 0),
-                row(0, map(0, 0, 1, 1), 0, 0),
-                row(1, map(0, 0, 1, 1), 0, 0)
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE c=? AND b CONTAINS KEY ? ALLOW FILTERING", 0, 0),
-                row(0, map(0, 0, 1, 1), 0, 0),
-                row(1, map(0, 0, 1, 1), 0, 0)
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE a=? AND c=? AND b CONTAINS ? ALLOW FILTERING", 0, 0, 0),
-                row(0, map(0, 0, 1, 1), 0, 0)
-        );
-        assertRows(execute("SELECT * FROM %s WHERE a=? AND c=? AND b CONTAINS KEY ? ALLOW FILTERING", 0, 0, 0),
-                row(0, map(0, 0, 1, 1), 0, 0)
-        );
-    }
-
-    @Test
-    public void testFrozenListInMap() throws Throwable
-    {
-        createTable("CREATE TABLE %s (k int primary key, m map<frozen<list<int>>, int>)");
-
-        execute("INSERT INTO %s (k, m) VALUES (1, {[1, 2, 3] : 1})");
-        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
-                row(1, map(list(1, 2, 3), 1)));
-
-        execute("UPDATE %s SET m[[1, 2, 3]]=2 WHERE k=1");
-        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
-                row(1, map(list(1, 2, 3), 2)));
-
-        execute("UPDATE %s SET m = m + ? WHERE k=1", map(list(4, 5, 6), 3));
-        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
-                row(1,
-                    map(list(1, 2, 3), 2,
-                        list(4, 5, 6), 3)));
-
-        execute("DELETE m[[1, 2, 3]] FROM %s WHERE k = 1");
-        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
-                row(1, map(list(4, 5, 6), 3)));
-    }
-
-    @Test
-    public void testFrozenListInSet() throws Throwable
-    {
-        createTable("CREATE TABLE %s (k int primary key, s set<frozen<list<int>>>)");
-
-        execute("INSERT INTO %s (k, s) VALUES (1, {[1, 2, 3]})");
-        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
-                row(1, set(list(1, 2, 3)))
-        );
-
-        execute("UPDATE %s SET s = s + ? WHERE k=1", set(list(4, 5, 6)));
-        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
-                row(1, set(list(1, 2, 3), list(4, 5, 6)))
-        );
-
-        execute("UPDATE %s SET s = s - ? WHERE k=1", set(list(4, 5, 6)));
-        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
-                row(1, set(list(1, 2, 3)))
-        );
-
-        execute("DELETE s[[1, 2, 3]] FROM %s WHERE k = 1");
-        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
-                row(1, null)
-        );
-    }
-
-    @Test
-    public void testFrozenListInList() throws Throwable
-    {
-        createTable("CREATE TABLE %s (k int primary key, l list<frozen<list<int>>>)");
-
-        execute("INSERT INTO %s (k, l) VALUES (1, [[1, 2, 3]])");
-        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
-                row(1, list(list(1, 2, 3)))
-        );
-
-        execute("UPDATE %s SET l[?]=? WHERE k=1", 0, list(4, 5, 6));
-        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
-                row(1, list(list(4, 5, 6)))
-        );
-
-        execute("UPDATE %s SET l = ? + l WHERE k=1", list(list(1, 2, 3)));
-        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
-                row(1, list(list(1, 2, 3), list(4, 5, 6)))
-        );
-
-        execute("UPDATE %s SET l = l + ? WHERE k=1", list(list(7, 8, 9)));
-        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
-                row(1, list(list(1, 2, 3), list(4, 5, 6), list(7, 8, 9)))
-        );
-
-        execute("UPDATE %s SET l = l - ? WHERE k=1", list(list(4, 5, 6)));
-        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
-                row(1, list(list(1, 2, 3), list(7, 8, 9)))
-        );
-
-        execute("DELETE l[0] FROM %s WHERE k = 1");
-        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
-                row(1, list(list(7, 8, 9)))
-        );
-    }
-
-    @Test
-    public void testFrozenMapInMap() throws Throwable
-    {
-        createTable("CREATE TABLE %s (k int primary key, m map<frozen<map<int, int>>, int>)");
-
-        execute("INSERT INTO %s (k, m) VALUES (1, {{1 : 1, 2 : 2} : 1})");
-        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
-                row(1, map(map(1, 1, 2, 2), 1)));
-
-        execute("UPDATE %s SET m[?]=2 WHERE k=1", map(1, 1, 2, 2));
-        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
-                row(1, map(map(1, 1, 2, 2), 2)));
-
-        execute("UPDATE %s SET m = m + ? WHERE k=1", map(map(3, 3, 4, 4), 3));
-        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
-                row(1,
-                    map(map(1, 1, 2, 2), 2,
-                        map(3, 3, 4, 4), 3)));
-
-        execute("DELETE m[?] FROM %s WHERE k = 1", map(1, 1, 2, 2));
-        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
-                row(1, map(map(3, 3, 4, 4), 3)));
-    }
-
-    @Test
-    public void testFrozenMapInSet() throws Throwable
-    {
-        createTable("CREATE TABLE %s (k int primary key, s set<frozen<map<int, int>>>)");
-
-        execute("INSERT INTO %s (k, s) VALUES (1, {{1 : 1, 2 : 2}})");
-
-        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
-                row(1, set(map(1, 1, 2, 2)))
-        );
-
-        execute("UPDATE %s SET s = s + ? WHERE k=1", set(map(3, 3, 4, 4)));
-        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
-                row(1, set(map(1, 1, 2, 2), map(3, 3, 4, 4)))
-        );
-
-        execute("UPDATE %s SET s = s - ? WHERE k=1", set(map(3, 3, 4, 4)));
-        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
-                row(1, set(map(1, 1, 2, 2)))
-        );
-
-        execute("DELETE s[?] FROM %s WHERE k = 1", map(1, 1, 2, 2));
-        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
-                row(1, null)
-        );
-    }
-
-    @Test
-    public void testFrozenMapInList() throws Throwable
-    {
-        createTable("CREATE TABLE %s (k int primary key, l list<frozen<map<int, int>>>)");
-
-        execute("INSERT INTO %s (k, l) VALUES (1, [{1 : 1, 2 : 2}])");
-        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
-                row(1, list(map(1, 1, 2, 2)))
-        );
-
-        execute("UPDATE %s SET l[?]=? WHERE k=1", 0, map(3, 3, 4, 4));
-        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
-                row(1, list(map(3, 3, 4, 4)))
-        );
-
-        execute("UPDATE %s SET l = ? + l WHERE k=1", list(map(1, 1, 2, 2)));
-        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
-                row(1, list(map(1, 1, 2, 2), map(3, 3, 4, 4)))
-        );
-
-        execute("UPDATE %s SET l = l + ? WHERE k=1", list(map(5, 5, 6, 6)));
-        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
-                row(1, list(map(1, 1, 2, 2), map(3, 3, 4, 4), map(5, 5, 6, 6)))
-        );
-
-        execute("UPDATE %s SET l = l - ? WHERE k=1", list(map(3, 3, 4, 4)));
-        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
-                row(1, list(map(1, 1, 2, 2), map(5, 5, 6, 6)))
-        );
-
-        execute("DELETE l[0] FROM %s WHERE k = 1");
-        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
-                row(1, list(map(5, 5, 6, 6)))
-        );
-    }
-
-    @Test
-    public void testFrozenSetInMap() throws Throwable
-    {
-        createTable("CREATE TABLE %s (k int primary key, m map<frozen<set<int>>, int>)");
-
-        execute("INSERT INTO %s (k, m) VALUES (1, {{1, 2, 3} : 1})");
-        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
-                row(1, map(set(1, 2, 3), 1)));
-
-        execute("UPDATE %s SET m[?]=2 WHERE k=1", set(1, 2, 3));
-        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
-                row(1, map(set(1, 2, 3), 2)));
-
-        execute("UPDATE %s SET m = m + ? WHERE k=1", map(set(4, 5, 6), 3));
-        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
-                row(1,
-                    map(set(1, 2, 3), 2,
-                        set(4, 5, 6), 3)));
-
-        execute("DELETE m[?] FROM %s WHERE k = 1", set(1, 2, 3));
-        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
-                row(1, map(set(4, 5, 6), 3)));
-    }
-
-    @Test
-    public void testFrozenSetInSet() throws Throwable
-    {
-        createTable("CREATE TABLE %s (k int primary key, s set<frozen<set<int>>>)");
-
-        execute("INSERT INTO %s (k, s) VALUES (1, {{1, 2, 3}})");
-
-        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
-                row(1, set(set(1, 2, 3)))
-        );
-
-        execute("UPDATE %s SET s = s + ? WHERE k=1", set(set(4, 5, 6)));
-        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
-                row(1, set(set(1, 2, 3), set(4, 5, 6)))
-        );
-
-        execute("UPDATE %s SET s = s - ? WHERE k=1", set(set(4, 5, 6)));
-        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
-                row(1, set(set(1, 2, 3)))
-        );
-
-        execute("DELETE s[?] FROM %s WHERE k = 1", set(1, 2, 3));
-        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
-                row(1, null)
-        );
-    }
-
-    @Test
-    public void testFrozenSetInList() throws Throwable
-    {
-        createTable("CREATE TABLE %s (k int primary key, l list<frozen<set<int>>>)");
-
-        execute("INSERT INTO %s (k, l) VALUES (1, [{1, 2, 3}])");
-        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
-                row(1, list(set(1, 2, 3)))
-        );
-
-        execute("UPDATE %s SET l[?]=? WHERE k=1", 0, set(4, 5, 6));
-        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
-                row(1, list(set(4, 5, 6)))
-        );
-
-        execute("UPDATE %s SET l = ? + l WHERE k=1", list(set(1, 2, 3)));
-        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
-                row(1, list(set(1, 2, 3), set(4, 5, 6)))
-        );
-
-        execute("UPDATE %s SET l = l + ? WHERE k=1", list(set(7, 8, 9)));
-        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
-                row(1, list(set(1, 2, 3), set(4, 5, 6), set(7, 8, 9)))
-        );
-
-        execute("UPDATE %s SET l = l - ? WHERE k=1", list(set(4, 5, 6)));
-        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
-                row(1, list(set(1, 2, 3), set(7, 8, 9)))
-        );
-
-        execute("DELETE l[0] FROM %s WHERE k = 1");
-        assertRows(execute("SELECT * FROM %s WHERE k = 1"),
-                row(1, list(set(7, 8, 9)))
-        );
-    }
-
-    @Test
-    public void testUserDefinedTypes() throws Throwable
-    {
-        String myType = createType("CREATE TYPE %s (a set<int>, b tuple<list<int>>)");
-        createTable("CREATE TABLE %s (k int PRIMARY KEY, v frozen<" + myType + ">)");
-        execute("INSERT INTO %s (k, v) VALUES (?, {a: ?, b: ?})", 0, set(1, 2, 3), tuple(list(1, 2, 3)));
-        assertRows(execute("SELECT v.a, v.b FROM %s WHERE k=?", 0),
-            row(set(1, 2, 3), tuple(list(1, 2, 3)))
-        );
-    }
-
-    private static String clean(String classname)
-    {
-        return StringUtils.remove(classname, "org.apache.cassandra.db.marshal.");
-    }
-
-    @Test
-    public void testToString()
-    {
-        // set<frozen<list<int>>>
-        SetType t = SetType.getInstance(ListType.getInstance(Int32Type.instance, false), true);
-        assertEquals("SetType(FrozenType(ListType(Int32Type)))", clean(t.toString()));
-        assertEquals("SetType(ListType(Int32Type))", clean(t.toString(true)));
-
-        // frozen<set<list<int>>>
-        t = SetType.getInstance(ListType.getInstance(Int32Type.instance, false), false);
-        assertEquals("FrozenType(SetType(ListType(Int32Type)))", clean(t.toString()));
-        assertEquals("SetType(ListType(Int32Type))", clean(t.toString(true)));
-
-        // map<frozen<list<int>>, int>
-        MapType m = MapType.getInstance(ListType.getInstance(Int32Type.instance, false), Int32Type.instance, true);
-        assertEquals("MapType(FrozenType(ListType(Int32Type)),Int32Type)", clean(m.toString()));
-        assertEquals("MapType(ListType(Int32Type),Int32Type)", clean(m.toString(true)));
-
-        // frozen<map<list<int>, int>>
-        m = MapType.getInstance(ListType.getInstance(Int32Type.instance, false), Int32Type.instance, false);
-        assertEquals("FrozenType(MapType(ListType(Int32Type),Int32Type))", clean(m.toString()));
-        assertEquals("MapType(ListType(Int32Type),Int32Type)", clean(m.toString(true)));
-
-        // tuple<set<int>>
-        List<AbstractType<?>> types = new ArrayList<>();
-        types.add(SetType.getInstance(Int32Type.instance, true));
-        TupleType tuple = new TupleType(types);
-        assertEquals("TupleType(SetType(Int32Type))", clean(tuple.toString()));
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/01115f72/test/unit/org/apache/cassandra/cql3/IndexedValuesValidationTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/IndexedValuesValidationTest.java b/test/unit/org/apache/cassandra/cql3/IndexedValuesValidationTest.java
deleted file mode 100644
index 487fbc2..0000000
--- a/test/unit/org/apache/cassandra/cql3/IndexedValuesValidationTest.java
+++ /dev/null
@@ -1,149 +0,0 @@
-/*
- *
- *  * 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;
-
-import java.nio.ByteBuffer;
-import java.util.HashMap;
-import java.util.Map;
-
-import org.junit.Test;
-
-import org.apache.cassandra.utils.FBUtilities;
-
-import static org.junit.Assert.fail;
-
-public class IndexedValuesValidationTest extends CQLTester
-{
-    private static final int TOO_BIG = 1024 * 65;
-    // CASSANDRA-8280/8081
-    // reject updates with indexed values where value > 64k
-    @Test
-    public void testIndexOnCompositeValueOver64k() throws Throwable
-    {
-        createTable("CREATE TABLE %s(a int, b int, c blob, PRIMARY KEY (a))");
-        createIndex("CREATE INDEX ON %s(c)");
-        failInsert("INSERT INTO %s (a, b, c) VALUES (0, 0, ?)", ByteBuffer.allocate(TOO_BIG));
-    }
-
-    @Test
-    public void testIndexOnClusteringColumnInsertPartitionKeyAndClusteringsOver64k() throws Throwable
-    {
-        createTable("CREATE TABLE %s(a blob, b blob, c blob, d int, PRIMARY KEY (a, b, c))");
-        createIndex("CREATE INDEX ON %s(b)");
-
-        // CompositeIndexOnClusteringKey creates index entries composed of the
-        // PK plus all of the non-indexed clustering columns from the primary row
-        // so we should reject where len(a) + len(c) > 65560 as this will form the
-        // total clustering in the index table
-        ByteBuffer a = ByteBuffer.allocate(100);
-        ByteBuffer b = ByteBuffer.allocate(10);
-        ByteBuffer c = ByteBuffer.allocate(FBUtilities.MAX_UNSIGNED_SHORT - 99);
-
-        failInsert("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, 0)", a, b, c);
-    }
-
-    @Test
-    public void testCompactTableWithValueOver64k() throws Throwable
-    {
-        createTable("CREATE TABLE %s(a int, b blob, PRIMARY KEY (a)) WITH COMPACT STORAGE");
-        createIndex("CREATE INDEX ON %s(b)");
-        failInsert("INSERT INTO %s (a, b) VALUES (0, ?)", ByteBuffer.allocate(TOO_BIG));
-    }
-
-    @Test
-    public void testIndexOnCollectionValueInsertPartitionKeyAndCollectionKeyOver64k() throws Throwable
-    {
-        createTable("CREATE TABLE %s(a blob , b map<blob, int>, PRIMARY KEY (a))");
-        createIndex("CREATE INDEX ON %s(b)");
-
-        // A collection key > 64k by itself will be rejected from
-        // the primary table.
-        // To test index validation we need to ensure that
-        // len(b) < 64k, but len(a) + len(b) > 64k as that will
-        // form the clustering in the index table
-        ByteBuffer a = ByteBuffer.allocate(100);
-        ByteBuffer b = ByteBuffer.allocate(FBUtilities.MAX_UNSIGNED_SHORT - 100);
-
-        failInsert("UPDATE %s SET b[?] = 0 WHERE a = ?", b, a);
-    }
-
-    @Test
-    public void testIndexOnCollectionKeyInsertPartitionKeyAndClusteringOver64k() throws Throwable
-    {
-        createTable("CREATE TABLE %s(a blob, b blob, c map<blob, int>, PRIMARY KEY (a, b))");
-        createIndex("CREATE INDEX ON %s(KEYS(c))");
-
-        // Basically the same as the case with non-collection clustering
-        // CompositeIndexOnCollectionKeyy creates index entries composed of the
-        // PK plus all of the clustering columns from the primary row, except the
-        // collection element - which becomes the partition key in the index table
-        ByteBuffer a = ByteBuffer.allocate(100);
-        ByteBuffer b = ByteBuffer.allocate(FBUtilities.MAX_UNSIGNED_SHORT - 100);
-        ByteBuffer c = ByteBuffer.allocate(10);
-
-        failInsert("UPDATE %s SET c[?] = 0 WHERE a = ? and b = ?", c, a, b);
-    }
-
-    @Test
-    public void testIndexOnPartitionKeyInsertValueOver64k() throws Throwable
-    {
-        createTable("CREATE TABLE %s(a int, b int, c blob, PRIMARY KEY ((a, b)))");
-        createIndex("CREATE INDEX ON %s(a)");
-        succeedInsert("INSERT INTO %s (a, b, c) VALUES (0, 0, ?)", ByteBuffer.allocate(TOO_BIG));
-    }
-
-    @Test
-    public void testIndexOnClusteringColumnInsertValueOver64k() throws Throwable
-    {
-        createTable("CREATE TABLE %s(a int, b int, c blob, PRIMARY KEY (a, b))");
-        createIndex("CREATE INDEX ON %s(b)");
-        succeedInsert("INSERT INTO %s (a, b, c) VALUES (0, 0, ?)", ByteBuffer.allocate(TOO_BIG));
-    }
-
-    @Test
-    public void testIndexOnFullCollectionEntryInsertCollectionValueOver64k() throws Throwable
-    {
-        createTable("CREATE TABLE %s(a int, b frozen<map<int, blob>>, PRIMARY KEY (a))");
-        createIndex("CREATE INDEX ON %s(full(b))");
-        Map<Integer, ByteBuffer> map = new HashMap();
-        map.put(0, ByteBuffer.allocate(1024 * 65));
-        failInsert("INSERT INTO %s (a, b) VALUES (0, ?)", map);
-    }
-
-    public void failInsert(String insertCQL, Object...args) throws Throwable
-    {
-        try
-        {
-            execute(insertCQL, args);
-            fail("Expected statement to fail validation");
-        }
-        catch (Exception e)
-        {
-            // as expected
-        }
-    }
-
-    public void succeedInsert(String insertCQL, Object...args) throws Throwable
-    {
-        execute(insertCQL, args);
-        flush();
-    }
-}


[20/32] cassandra git commit: 2.2 commit for CASSANDRA-9160

Posted by jm...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/01115f72/test/unit/org/apache/cassandra/cql3/validation/entities/JsonTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/JsonTest.java b/test/unit/org/apache/cassandra/cql3/validation/entities/JsonTest.java
new file mode 100644
index 0000000..7f8fa0b
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/JsonTest.java
@@ -0,0 +1,958 @@
+/*
+ * 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.validation.entities;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.cql3.Json;
+import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.dht.ByteOrderedPartitioner;
+import org.apache.cassandra.serializers.SimpleDateSerializer;
+import org.apache.cassandra.serializers.TimeSerializer;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.net.InetAddress;
+import java.text.SimpleDateFormat;
+import java.util.Date;
+import java.util.UUID;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+public class JsonTest extends CQLTester
+{
+    @BeforeClass
+    public static void setUp()
+    {
+        DatabaseDescriptor.setPartitioner(ByteOrderedPartitioner.instance);
+    }
+
+    @Test
+    public void testFromJsonFct() throws Throwable
+    {
+        String typeName = createType("CREATE TYPE %s (a int, b uuid, c set<text>)");
+        createTable("CREATE TABLE %s (" +
+                "k int PRIMARY KEY, " +
+                "asciival ascii, " +
+                "bigintval bigint, " +
+                "blobval blob, " +
+                "booleanval boolean, " +
+                "dateval date, " +
+                "decimalval decimal, " +
+                "doubleval double, " +
+                "floatval float, " +
+                "inetval inet, " +
+                "intval int, " +
+                "textval text, " +
+                "timeval time, " +
+                "timestampval timestamp, " +
+                "timeuuidval timeuuid, " +
+                "uuidval uuid," +
+                "varcharval varchar, " +
+                "varintval varint, " +
+                "listval list<int>, " +
+                "frozenlistval frozen<list<int>>, " +
+                "setval set<uuid>, " +
+                "frozensetval frozen<set<uuid>>, " +
+                "mapval map<ascii, int>," +
+                "frozenmapval frozen<map<ascii, int>>," +
+                "tupleval frozen<tuple<int, ascii, uuid>>," +
+                "udtval frozen<" + typeName + ">)");
+
+
+        // fromJson() can only be used when the receiver type is known
+        assertInvalidMessage("fromJson() cannot be used in the selection clause", "SELECT fromJson(asciival) FROM %s", 0, 0);
+
+        String func1 = createFunction(KEYSPACE, "int", "CREATE FUNCTION %s (a int) CALLED ON NULL INPUT RETURNS text LANGUAGE java AS $$ return a.toString(); $$");
+        createFunctionOverload(func1, "int", "CREATE FUNCTION %s (a text) CALLED ON NULL INPUT RETURNS text LANGUAGE java AS $$ return new String(a); $$");
+
+        assertInvalidMessage("Ambiguous call to function",
+                "INSERT INTO %s (k, textval) VALUES (?, " + func1 + "(fromJson(?)))", 0, "123");
+
+        // fails JSON parsing
+        assertInvalidMessage("Could not decode JSON string '\u038E\u0394\u03B4\u03E0'",
+                "INSERT INTO %s (k, asciival) VALUES (?, fromJson(?))", 0, "\u038E\u0394\u03B4\u03E0");
+
+        // handle nulls
+        execute("INSERT INTO %s (k, asciival) VALUES (?, fromJson(?))", 0, null);
+
+        // ================ ascii ================
+        execute("INSERT INTO %s (k, asciival) VALUES (?, fromJson(?))", 0, "\"ascii text\"");
+        assertRows(execute("SELECT k, asciival FROM %s WHERE k = ?", 0), row(0, "ascii text"));
+
+        execute("INSERT INTO %s (k, asciival) VALUES (?, fromJson(?))", 0, "\"ascii \\\" text\"");
+        assertRows(execute("SELECT k, asciival FROM %s WHERE k = ?", 0), row(0, "ascii \" text"));
+
+        assertInvalidMessage("Invalid ASCII character in string literal",
+                "INSERT INTO %s (k, asciival) VALUES (?, fromJson(?))", 0, "\"\\u1fff\\u2013\\u33B4\\u2014\"");
+
+        assertInvalidMessage("Expected an ascii string, but got a Integer",
+                "INSERT INTO %s (k, asciival) VALUES (?, fromJson(?))", 0, "123");
+
+        // test that we can use fromJson() in other valid places in queries
+        assertRows(execute("SELECT asciival FROM %s WHERE k = fromJson(?)", "0"), row("ascii \" text"));
+        execute("UPDATE %s SET asciival = fromJson(?) WHERE k = fromJson(?)", "\"ascii \\\" text\"", "0");
+        execute("DELETE FROM %s WHERE k = fromJson(?)", "0");
+
+        // ================ bigint ================
+        execute("INSERT INTO %s (k, bigintval) VALUES (?, fromJson(?))", 0, "123123123123");
+        assertRows(execute("SELECT k, bigintval FROM %s WHERE k = ?", 0), row(0, 123123123123L));
+
+        // strings are also accepted
+        execute("INSERT INTO %s (k, bigintval) VALUES (?, fromJson(?))", 0, "\"123123123123\"");
+        assertRows(execute("SELECT k, bigintval FROM %s WHERE k = ?", 0), row(0, 123123123123L));
+
+        // overflow (Long.MAX_VALUE + 1)
+        assertInvalidMessage("Expected a bigint value, but got a",
+                "INSERT INTO %s (k, bigintval) VALUES (?, fromJson(?))", 0, "9223372036854775808");
+
+        assertInvalidMessage("Expected a bigint value, but got a",
+                "INSERT INTO %s (k, bigintval) VALUES (?, fromJson(?))", 0, "123.456");
+
+        assertInvalidMessage("Unable to make long from",
+                "INSERT INTO %s (k, bigintval) VALUES (?, fromJson(?))", 0, "\"abc\"");
+
+        assertInvalidMessage("Expected a bigint value, but got a",
+                "INSERT INTO %s (k, bigintval) VALUES (?, fromJson(?))", 0, "[\"abc\"]");
+
+        // ================ blob ================
+        execute("INSERT INTO %s (k, blobval) VALUES (?, fromJson(?))", 0, "\"0x00000001\"");
+        assertRows(execute("SELECT k, blobval FROM %s WHERE k = ?", 0), row(0, ByteBufferUtil.bytes(1)));
+
+        assertInvalidMessage("Value 'xyzz' is not a valid blob representation",
+            "INSERT INTO %s (k, blobval) VALUES (?, fromJson(?))", 0, "\"xyzz\"");
+
+        assertInvalidMessage("String representation of blob is missing 0x prefix: 123",
+                "INSERT INTO %s (k, blobval) VALUES (?, fromJson(?))", 0, "\"123\"");
+
+        assertInvalidMessage("Value '0x123' is not a valid blob representation",
+                "INSERT INTO %s (k, blobval) VALUES (?, fromJson(?))", 0, "\"0x123\"");
+
+        assertInvalidMessage("Value '123' is not a valid blob representation",
+                "INSERT INTO %s (k, blobval) VALUES (?, fromJson(?))", 0, "123");
+
+        // ================ boolean ================
+        execute("INSERT INTO %s (k, booleanval) VALUES (?, fromJson(?))", 0, "true");
+        assertRows(execute("SELECT k, booleanval FROM %s WHERE k = ?", 0), row(0, true));
+
+        execute("INSERT INTO %s (k, booleanval) VALUES (?, fromJson(?))", 0, "false");
+        assertRows(execute("SELECT k, booleanval FROM %s WHERE k = ?", 0), row(0, false));
+
+        // strings are also accepted
+        execute("INSERT INTO %s (k, booleanval) VALUES (?, fromJson(?))", 0, "\"false\"");
+        assertRows(execute("SELECT k, booleanval FROM %s WHERE k = ?", 0), row(0, false));
+
+        assertInvalidMessage("Unable to make boolean from",
+                "INSERT INTO %s (k, booleanval) VALUES (?, fromJson(?))", 0, "\"abc\"");
+
+        assertInvalidMessage("Expected a boolean value, but got a Integer",
+                "INSERT INTO %s (k, booleanval) VALUES (?, fromJson(?))", 0, "123");
+
+        // ================ date ================
+        execute("INSERT INTO %s (k, dateval) VALUES (?, fromJson(?))", 0, "\"1987-03-23\"");
+        assertRows(execute("SELECT k, dateval FROM %s WHERE k = ?", 0), row(0, SimpleDateSerializer.dateStringToDays("1987-03-23")));
+
+        assertInvalidMessage("Expected a string representation of a date",
+                "INSERT INTO %s (k, dateval) VALUES (?, fromJson(?))", 0, "123");
+
+        assertInvalidMessage("Unable to coerce 'xyz' to a formatted date",
+                "INSERT INTO %s (k, dateval) VALUES (?, fromJson(?))", 0, "\"xyz\"");
+
+        // ================ decimal ================
+        execute("INSERT INTO %s (k, decimalval) VALUES (?, fromJson(?))", 0, "123123.123123");
+        assertRows(execute("SELECT k, decimalval FROM %s WHERE k = ?", 0), row(0, new BigDecimal("123123.123123")));
+
+        execute("INSERT INTO %s (k, decimalval) VALUES (?, fromJson(?))", 0, "123123");
+        assertRows(execute("SELECT k, decimalval FROM %s WHERE k = ?", 0), row(0, new BigDecimal("123123")));
+
+        // accept strings for numbers that cannot be represented as doubles
+        execute("INSERT INTO %s (k, decimalval) VALUES (?, fromJson(?))", 0, "\"123123.123123\"");
+        assertRows(execute("SELECT k, decimalval FROM %s WHERE k = ?", 0), row(0, new BigDecimal("123123.123123")));
+
+        execute("INSERT INTO %s (k, decimalval) VALUES (?, fromJson(?))", 0, "\"-1.23E-12\"");
+        assertRows(execute("SELECT k, decimalval FROM %s WHERE k = ?", 0), row(0, new BigDecimal("-1.23E-12")));
+
+        assertInvalidMessage("Value 'xyzz' is not a valid representation of a decimal value",
+                "INSERT INTO %s (k, decimalval) VALUES (?, fromJson(?))", 0, "\"xyzz\"");
+
+        assertInvalidMessage("Value 'true' is not a valid representation of a decimal value",
+                "INSERT INTO %s (k, decimalval) VALUES (?, fromJson(?))", 0, "true");
+
+        // ================ double ================
+        execute("INSERT INTO %s (k, doubleval) VALUES (?, fromJson(?))", 0, "123123.123123");
+        assertRows(execute("SELECT k, doubleval FROM %s WHERE k = ?", 0), row(0, 123123.123123d));
+
+        execute("INSERT INTO %s (k, doubleval) VALUES (?, fromJson(?))", 0, "123123");
+        assertRows(execute("SELECT k, doubleval FROM %s WHERE k = ?", 0), row(0, 123123.0d));
+
+        // strings are also accepted
+        execute("INSERT INTO %s (k, doubleval) VALUES (?, fromJson(?))", 0, "\"123123\"");
+        assertRows(execute("SELECT k, doubleval FROM %s WHERE k = ?", 0), row(0, 123123.0d));
+
+        assertInvalidMessage("Unable to make double from",
+                "INSERT INTO %s (k, doubleval) VALUES (?, fromJson(?))", 0, "\"xyzz\"");
+
+        assertInvalidMessage("Expected a double value, but got",
+                "INSERT INTO %s (k, doubleval) VALUES (?, fromJson(?))", 0, "true");
+
+        // ================ float ================
+        execute("INSERT INTO %s (k, floatval) VALUES (?, fromJson(?))", 0, "123123.123123");
+        assertRows(execute("SELECT k, floatval FROM %s WHERE k = ?", 0), row(0, 123123.123123f));
+
+        execute("INSERT INTO %s (k, floatval) VALUES (?, fromJson(?))", 0, "123123");
+        assertRows(execute("SELECT k, floatval FROM %s WHERE k = ?", 0), row(0, 123123.0f));
+
+        // strings are also accepted
+        execute("INSERT INTO %s (k, floatval) VALUES (?, fromJson(?))", 0, "\"123123.0\"");
+        assertRows(execute("SELECT k, floatval FROM %s WHERE k = ?", 0), row(0, 123123.0f));
+
+        assertInvalidMessage("Unable to make float from",
+                "INSERT INTO %s (k, floatval) VALUES (?, fromJson(?))", 0, "\"xyzz\"");
+
+        assertInvalidMessage("Expected a float value, but got a",
+                "INSERT INTO %s (k, floatval) VALUES (?, fromJson(?))", 0, "true");
+
+        // ================ inet ================
+        execute("INSERT INTO %s (k, inetval) VALUES (?, fromJson(?))", 0, "\"127.0.0.1\"");
+        assertRows(execute("SELECT k, inetval FROM %s WHERE k = ?", 0), row(0, InetAddress.getByName("127.0.0.1")));
+
+        execute("INSERT INTO %s (k, inetval) VALUES (?, fromJson(?))", 0, "\"::1\"");
+        assertRows(execute("SELECT k, inetval FROM %s WHERE k = ?", 0), row(0, InetAddress.getByName("::1")));
+
+        assertInvalidMessage("Unable to make inet address from 'xyzz'",
+                "INSERT INTO %s (k, inetval) VALUES (?, fromJson(?))", 0, "\"xyzz\"");
+
+        assertInvalidMessage("Expected a string representation of an inet value, but got a Integer",
+                "INSERT INTO %s (k, inetval) VALUES (?, fromJson(?))", 0, "123");
+
+        // ================ int ================
+        execute("INSERT INTO %s (k, intval) VALUES (?, fromJson(?))", 0, "123123");
+        assertRows(execute("SELECT k, intval FROM %s WHERE k = ?", 0), row(0, 123123));
+
+        // strings are also accepted
+        execute("INSERT INTO %s (k, intval) VALUES (?, fromJson(?))", 0, "\"123123\"");
+        assertRows(execute("SELECT k, intval FROM %s WHERE k = ?", 0), row(0, 123123));
+
+        // int overflow (2 ^ 32, or Integer.MAX_INT + 1)
+        assertInvalidMessage("Expected an int value, but got a",
+                "INSERT INTO %s (k, intval) VALUES (?, fromJson(?))", 0, "2147483648");
+
+        assertInvalidMessage("Expected an int value, but got a",
+                "INSERT INTO %s (k, intval) VALUES (?, fromJson(?))", 0, "123.456");
+
+        assertInvalidMessage("Unable to make int from",
+                "INSERT INTO %s (k, intval) VALUES (?, fromJson(?))", 0, "\"xyzz\"");
+
+        assertInvalidMessage("Expected an int value, but got a",
+                "INSERT INTO %s (k, intval) VALUES (?, fromJson(?))", 0, "true");
+
+        // ================ text (varchar) ================
+        execute("INSERT INTO %s (k, textval) VALUES (?, fromJson(?))", 0, "\"\"");
+        assertRows(execute("SELECT k, textval FROM %s WHERE k = ?", 0), row(0, ""));
+
+        execute("INSERT INTO %s (k, textval) VALUES (?, fromJson(?))", 0, "\"abcd\"");
+        assertRows(execute("SELECT k, textval FROM %s WHERE k = ?", 0), row(0, "abcd"));
+
+        execute("INSERT INTO %s (k, textval) VALUES (?, fromJson(?))", 0, "\"some \\\" text\"");
+        assertRows(execute("SELECT k, textval FROM %s WHERE k = ?", 0), row(0, "some \" text"));
+
+        execute("INSERT INTO %s (k, textval) VALUES (?, fromJson(?))", 0, "\"\\u2013\"");
+        assertRows(execute("SELECT k, textval FROM %s WHERE k = ?", 0), row(0, "\u2013"));
+
+        assertInvalidMessage("Expected a UTF-8 string, but got a Integer",
+                "INSERT INTO %s (k, textval) VALUES (?, fromJson(?))", 0, "123");
+
+        // ================ time ================
+        execute("INSERT INTO %s (k, timeval) VALUES (?, fromJson(?))", 0, "\"07:35:07.000111222\"");
+        assertRows(execute("SELECT k, timeval FROM %s WHERE k = ?", 0), row(0, TimeSerializer.timeStringToLong("07:35:07.000111222")));
+
+        assertInvalidMessage("Expected a string representation of a time value",
+                "INSERT INTO %s (k, timeval) VALUES (?, fromJson(?))", 0, "123456");
+
+        assertInvalidMessage("Unable to coerce 'xyz' to a formatted time",
+                "INSERT INTO %s (k, timeval) VALUES (?, fromJson(?))", 0, "\"xyz\"");
+
+        // ================ timestamp ================
+        execute("INSERT INTO %s (k, timestampval) VALUES (?, fromJson(?))", 0, "123123123123");
+        assertRows(execute("SELECT k, timestampval FROM %s WHERE k = ?", 0), row(0, new Date(123123123123L)));
+
+        execute("INSERT INTO %s (k, timestampval) VALUES (?, fromJson(?))", 0, "\"2014-01-01\"");
+        assertRows(execute("SELECT k, timestampval FROM %s WHERE k = ?", 0), row(0, new SimpleDateFormat("y-M-d").parse("2014-01-01")));
+
+        assertInvalidMessage("Expected a long or a datestring representation of a timestamp value, but got a Double",
+                "INSERT INTO %s (k, timestampval) VALUES (?, fromJson(?))", 0, "123.456");
+
+        assertInvalidMessage("Unable to coerce 'abcd' to a formatted date",
+                "INSERT INTO %s (k, timestampval) VALUES (?, fromJson(?))", 0, "\"abcd\"");
+
+        // ================ timeuuid ================
+        execute("INSERT INTO %s (k, timeuuidval) VALUES (?, fromJson(?))", 0, "\"6bddc89a-5644-11e4-97fc-56847afe9799\"");
+        assertRows(execute("SELECT k, timeuuidval FROM %s WHERE k = ?", 0), row(0, UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799")));
+
+        execute("INSERT INTO %s (k, timeuuidval) VALUES (?, fromJson(?))", 0, "\"6BDDC89A-5644-11E4-97FC-56847AFE9799\"");
+        assertRows(execute("SELECT k, timeuuidval FROM %s WHERE k = ?", 0), row(0, UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799")));
+
+        assertInvalidMessage("TimeUUID supports only version 1 UUIDs",
+                "INSERT INTO %s (k, timeuuidval) VALUES (?, fromJson(?))", 0, "\"00000000-0000-0000-0000-000000000000\"");
+
+        assertInvalidMessage("Expected a string representation of a timeuuid, but got a Integer",
+                "INSERT INTO %s (k, timeuuidval) VALUES (?, fromJson(?))", 0, "123");
+
+         // ================ uuidval ================
+        execute("INSERT INTO %s (k, uuidval) VALUES (?, fromJson(?))", 0, "\"6bddc89a-5644-11e4-97fc-56847afe9799\"");
+        assertRows(execute("SELECT k, uuidval FROM %s WHERE k = ?", 0), row(0, UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799")));
+
+        execute("INSERT INTO %s (k, uuidval) VALUES (?, fromJson(?))", 0, "\"6BDDC89A-5644-11E4-97FC-56847AFE9799\"");
+        assertRows(execute("SELECT k, uuidval FROM %s WHERE k = ?", 0), row(0, UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799")));
+
+        assertInvalidMessage("Unable to make UUID from",
+                "INSERT INTO %s (k, uuidval) VALUES (?, fromJson(?))", 0, "\"00000000-0000-0000-zzzz-000000000000\"");
+
+        assertInvalidMessage("Expected a string representation of a uuid, but got a Integer",
+                "INSERT INTO %s (k, uuidval) VALUES (?, fromJson(?))", 0, "123");
+
+        // ================ varint ================
+        execute("INSERT INTO %s (k, varintval) VALUES (?, fromJson(?))", 0, "123123123123");
+        assertRows(execute("SELECT k, varintval FROM %s WHERE k = ?", 0), row(0, new BigInteger("123123123123")));
+
+        // accept strings for numbers that cannot be represented as longs
+        execute("INSERT INTO %s (k, varintval) VALUES (?, fromJson(?))", 0, "\"1234567890123456789012345678901234567890\"");
+        assertRows(execute("SELECT k, varintval FROM %s WHERE k = ?", 0), row(0, new BigInteger("1234567890123456789012345678901234567890")));
+
+        assertInvalidMessage("Value '123123.123' is not a valid representation of a varint value",
+                "INSERT INTO %s (k, varintval) VALUES (?, fromJson(?))", 0, "123123.123");
+
+        assertInvalidMessage("Value 'xyzz' is not a valid representation of a varint value",
+                "INSERT INTO %s (k, varintval) VALUES (?, fromJson(?))", 0, "\"xyzz\"");
+
+        assertInvalidMessage("Value '' is not a valid representation of a varint value",
+                "INSERT INTO %s (k, varintval) VALUES (?, fromJson(?))", 0, "\"\"");
+
+        assertInvalidMessage("Value 'true' is not a valid representation of a varint value",
+                "INSERT INTO %s (k, varintval) VALUES (?, fromJson(?))", 0, "true");
+
+        // ================ lists ================
+        execute("INSERT INTO %s (k, listval) VALUES (?, fromJson(?))", 0, "[1, 2, 3]");
+        assertRows(execute("SELECT k, listval FROM %s WHERE k = ?", 0), row(0, list(1, 2, 3)));
+
+        execute("INSERT INTO %s (k, listval) VALUES (?, fromJson(?))", 0, "[]");
+        assertRows(execute("SELECT k, listval FROM %s WHERE k = ?", 0), row(0, null));
+
+        assertInvalidMessage("Expected a list, but got a Integer",
+                "INSERT INTO %s (k, listval) VALUES (?, fromJson(?))", 0, "123");
+
+        assertInvalidMessage("Unable to make int from",
+                "INSERT INTO %s (k, listval) VALUES (?, fromJson(?))", 0, "[\"abc\"]");
+
+        assertInvalidMessage("Invalid null element in list",
+                "INSERT INTO %s (k, listval) VALUES (?, fromJson(?))", 0, "[null]");
+
+        // frozen
+        execute("INSERT INTO %s (k, frozenlistval) VALUES (?, fromJson(?))", 0, "[1, 2, 3]");
+        assertRows(execute("SELECT k, frozenlistval FROM %s WHERE k = ?", 0), row(0, list(1, 2, 3)));
+
+        // ================ sets ================
+        execute("INSERT INTO %s (k, setval) VALUES (?, fromJson(?))",
+                0, "[\"6bddc89a-5644-11e4-97fc-56847afe9798\", \"6bddc89a-5644-11e4-97fc-56847afe9799\"]");
+        assertRows(execute("SELECT k, setval FROM %s WHERE k = ?", 0),
+                row(0, set(UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9798"), (UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799"))))
+        );
+
+        // duplicates are okay, just like in CQL
+        execute("INSERT INTO %s (k, setval) VALUES (?, fromJson(?))",
+                0, "[\"6bddc89a-5644-11e4-97fc-56847afe9798\", \"6bddc89a-5644-11e4-97fc-56847afe9798\", \"6bddc89a-5644-11e4-97fc-56847afe9799\"]");
+        assertRows(execute("SELECT k, setval FROM %s WHERE k = ?", 0),
+                row(0, set(UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9798"), (UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799"))))
+        );
+
+        execute("INSERT INTO %s (k, setval) VALUES (?, fromJson(?))", 0, "[]");
+        assertRows(execute("SELECT k, setval FROM %s WHERE k = ?", 0), row(0, null));
+
+        assertInvalidMessage("Expected a list (representing a set), but got a Integer",
+                "INSERT INTO %s (k, setval) VALUES (?, fromJson(?))", 0, "123");
+
+        assertInvalidMessage("Unable to make UUID from",
+                "INSERT INTO %s (k, setval) VALUES (?, fromJson(?))", 0, "[\"abc\"]");
+
+        assertInvalidMessage("Invalid null element in set",
+                "INSERT INTO %s (k, setval) VALUES (?, fromJson(?))", 0, "[null]");
+
+        // frozen
+        execute("INSERT INTO %s (k, frozensetval) VALUES (?, fromJson(?))",
+                0, "[\"6bddc89a-5644-11e4-97fc-56847afe9798\", \"6bddc89a-5644-11e4-97fc-56847afe9799\"]");
+        assertRows(execute("SELECT k, frozensetval FROM %s WHERE k = ?", 0),
+                row(0, set(UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9798"), (UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799"))))
+        );
+
+        execute("INSERT INTO %s (k, frozensetval) VALUES (?, fromJson(?))",
+                0, "[\"6bddc89a-5644-11e4-97fc-56847afe9799\", \"6bddc89a-5644-11e4-97fc-56847afe9798\"]");
+        assertRows(execute("SELECT k, frozensetval FROM %s WHERE k = ?", 0),
+                row(0, set(UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9798"), (UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799"))))
+        );
+
+        // ================ maps ================
+        execute("INSERT INTO %s (k, mapval) VALUES (?, fromJson(?))", 0, "{\"a\": 1, \"b\": 2}");
+        assertRows(execute("SELECT k, mapval FROM %s WHERE k = ?", 0), row(0, map("a", 1, "b", 2)));
+
+        execute("INSERT INTO %s (k, mapval) VALUES (?, fromJson(?))", 0, "{}");
+        assertRows(execute("SELECT k, mapval FROM %s WHERE k = ?", 0), row(0, null));
+
+        assertInvalidMessage("Expected a map, but got a Integer",
+                "INSERT INTO %s (k, mapval) VALUES (?, fromJson(?))", 0, "123");
+
+        assertInvalidMessage("Invalid ASCII character in string literal",
+                "INSERT INTO %s (k, mapval) VALUES (?, fromJson(?))", 0, "{\"\\u1fff\\u2013\\u33B4\\u2014\": 1}");
+
+        assertInvalidMessage("Invalid null value in map",
+                "INSERT INTO %s (k, mapval) VALUES (?, fromJson(?))", 0, "{\"a\": null}");
+
+        // frozen
+        execute("INSERT INTO %s (k, frozenmapval) VALUES (?, fromJson(?))", 0, "{\"a\": 1, \"b\": 2}");
+        assertRows(execute("SELECT k, frozenmapval FROM %s WHERE k = ?", 0), row(0, map("a", 1, "b", 2)));
+
+        execute("INSERT INTO %s (k, frozenmapval) VALUES (?, fromJson(?))", 0, "{\"b\": 2, \"a\": 1}");
+        assertRows(execute("SELECT k, frozenmapval FROM %s WHERE k = ?", 0), row(0, map("a", 1, "b", 2)));
+
+        // ================ tuples ================
+        execute("INSERT INTO %s (k, tupleval) VALUES (?, fromJson(?))", 0, "[1, \"foobar\", \"6bddc89a-5644-11e4-97fc-56847afe9799\"]");
+        assertRows(execute("SELECT k, tupleval FROM %s WHERE k = ?", 0),
+            row(0, tuple(1, "foobar", UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799")))
+        );
+
+        execute("INSERT INTO %s (k, tupleval) VALUES (?, fromJson(?))", 0, "[1, null, \"6bddc89a-5644-11e4-97fc-56847afe9799\"]");
+        assertRows(execute("SELECT k, tupleval FROM %s WHERE k = ?", 0),
+                row(0, tuple(1, null, UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799")))
+        );
+
+        assertInvalidMessage("Tuple contains extra items",
+                "INSERT INTO %s (k, tupleval) VALUES (?, fromJson(?))",
+                0, "[1, \"foobar\", \"6bddc89a-5644-11e4-97fc-56847afe9799\", 1, 2, 3]");
+
+        assertInvalidMessage("Tuple is missing items",
+                "INSERT INTO %s (k, tupleval) VALUES (?, fromJson(?))",
+                0, "[1, \"foobar\"]");
+
+        assertInvalidMessage("Unable to make int from",
+                "INSERT INTO %s (k, tupleval) VALUES (?, fromJson(?))",
+                0, "[\"not an int\", \"foobar\", \"6bddc89a-5644-11e4-97fc-56847afe9799\"]");
+
+        // ================ UDTs ================
+        execute("INSERT INTO %s (k, udtval) VALUES (?, fromJson(?))", 0, "{\"a\": 1, \"b\": \"6bddc89a-5644-11e4-97fc-56847afe9799\", \"c\": [\"foo\", \"bar\"]}");
+        assertRows(execute("SELECT k, udtval.a, udtval.b, udtval.c FROM %s WHERE k = ?", 0),
+                row(0, 1, UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799"), set("bar", "foo"))
+        );
+
+        // order of fields shouldn't matter
+        execute("INSERT INTO %s (k, udtval) VALUES (?, fromJson(?))", 0, "{\"b\": \"6bddc89a-5644-11e4-97fc-56847afe9799\", \"a\": 1, \"c\": [\"foo\", \"bar\"]}");
+        assertRows(execute("SELECT k, udtval.a, udtval.b, udtval.c FROM %s WHERE k = ?", 0),
+                row(0, 1, UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799"), set("bar", "foo"))
+        );
+
+        // test nulls
+        execute("INSERT INTO %s (k, udtval) VALUES (?, fromJson(?))", 0, "{\"a\": null, \"b\": \"6bddc89a-5644-11e4-97fc-56847afe9799\", \"c\": [\"foo\", \"bar\"]}");
+        assertRows(execute("SELECT k, udtval.a, udtval.b, udtval.c FROM %s WHERE k = ?", 0),
+                row(0, null, UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799"), set("bar", "foo"))
+        );
+
+        // test missing fields
+        execute("INSERT INTO %s (k, udtval) VALUES (?, fromJson(?))", 0, "{\"a\": 1, \"b\": \"6bddc89a-5644-11e4-97fc-56847afe9799\"}");
+        assertRows(execute("SELECT k, udtval.a, udtval.b, udtval.c FROM %s WHERE k = ?", 0),
+                row(0, 1, UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799"), null)
+        );
+
+        assertInvalidMessage("Unknown field", "INSERT INTO %s (k, udtval) VALUES (?, fromJson(?))", 0, "{\"xxx\": 1}");
+        assertInvalidMessage("Unable to make int from",
+                "INSERT INTO %s (k, udtval) VALUES (?, fromJson(?))", 0, "{\"a\": \"foobar\"}");
+    }
+
+    @Test
+    public void testToJsonFct() throws Throwable
+    {
+        String typeName = createType("CREATE TYPE %s (a int, b uuid, c set<text>)");
+        createTable("CREATE TABLE %s (" +
+                "k int PRIMARY KEY, " +
+                "asciival ascii, " +
+                "bigintval bigint, " +
+                "blobval blob, " +
+                "booleanval boolean, " +
+                "dateval date, " +
+                "decimalval decimal, " +
+                "doubleval double, " +
+                "floatval float, " +
+                "inetval inet, " +
+                "intval int, " +
+                "textval text, " +
+                "timeval time, " +
+                "timestampval timestamp, " +
+                "timeuuidval timeuuid, " +
+                "uuidval uuid," +
+                "varcharval varchar, " +
+                "varintval varint, " +
+                "listval list<int>, " +
+                "frozenlistval frozen<list<int>>, " +
+                "setval set<uuid>, " +
+                "frozensetval frozen<set<uuid>>, " +
+                "mapval map<ascii, int>, " +
+                "frozenmapval frozen<map<ascii, int>>, " +
+                "tupleval frozen<tuple<int, ascii, uuid>>," +
+                "udtval frozen<" + typeName + ">)");
+
+        // toJson() can only be used in selections
+        assertInvalidMessage("toJson() may only be used within the selection clause",
+                "INSERT INTO %s (k, asciival) VALUES (?, toJson(?))", 0, 0);
+        assertInvalidMessage("toJson() may only be used within the selection clause",
+                "UPDATE %s SET asciival = toJson(?) WHERE k = ?", 0, 0);
+        assertInvalidMessage("toJson() may only be used within the selection clause",
+                "DELETE FROM %s WHERE k = fromJson(toJson(?))", 0);
+
+        // ================ ascii ================
+        execute("INSERT INTO %s (k, asciival) VALUES (?, ?)", 0, "ascii text");
+        assertRows(execute("SELECT k, toJson(asciival) FROM %s WHERE k = ?", 0), row(0, "\"ascii text\""));
+
+        execute("INSERT INTO %s (k, asciival) VALUES (?, ?)", 0, "");
+        assertRows(execute("SELECT k, toJson(asciival) FROM %s WHERE k = ?", 0), row(0, "\"\""));
+
+        // ================ bigint ================
+        execute("INSERT INTO %s (k, bigintval) VALUES (?, ?)", 0, 123123123123L);
+        assertRows(execute("SELECT k, toJson(bigintval) FROM %s WHERE k = ?", 0), row(0, "123123123123"));
+
+        execute("INSERT INTO %s (k, bigintval) VALUES (?, ?)", 0, 0L);
+        assertRows(execute("SELECT k, toJson(bigintval) FROM %s WHERE k = ?", 0), row(0, "0"));
+
+        execute("INSERT INTO %s (k, bigintval) VALUES (?, ?)", 0, -123123123123L);
+        assertRows(execute("SELECT k, toJson(bigintval) FROM %s WHERE k = ?", 0), row(0, "-123123123123"));
+
+        // ================ blob ================
+        execute("INSERT INTO %s (k, blobval) VALUES (?, ?)", 0, ByteBufferUtil.bytes(1));
+        assertRows(execute("SELECT k, toJson(blobval) FROM %s WHERE k = ?", 0), row(0, "\"0x00000001\""));
+
+        execute("INSERT INTO %s (k, blobval) VALUES (?, ?)", 0, ByteBufferUtil.EMPTY_BYTE_BUFFER);
+        assertRows(execute("SELECT k, toJson(blobval) FROM %s WHERE k = ?", 0), row(0, "\"0x\""));
+
+        // ================ boolean ================
+        execute("INSERT INTO %s (k, booleanval) VALUES (?, ?)", 0, true);
+        assertRows(execute("SELECT k, toJson(booleanval) FROM %s WHERE k = ?", 0), row(0, "true"));
+
+        execute("INSERT INTO %s (k, booleanval) VALUES (?, ?)", 0, false);
+        assertRows(execute("SELECT k, toJson(booleanval) FROM %s WHERE k = ?", 0), row(0, "false"));
+
+        // ================ date ================
+        execute("INSERT INTO %s (k, dateval) VALUES (?, ?)", 0, SimpleDateSerializer.dateStringToDays("1987-03-23"));
+        assertRows(execute("SELECT k, toJson(dateval) FROM %s WHERE k = ?", 0), row(0, "\"1987-03-23\""));
+
+        // ================ decimal ================
+        execute("INSERT INTO %s (k, decimalval) VALUES (?, ?)", 0, new BigDecimal("123123.123123"));
+        assertRows(execute("SELECT k, toJson(decimalval) FROM %s WHERE k = ?", 0), row(0, "123123.123123"));
+
+        execute("INSERT INTO %s (k, decimalval) VALUES (?, ?)", 0, new BigDecimal("-1.23E-12"));
+        assertRows(execute("SELECT k, toJson(decimalval) FROM %s WHERE k = ?", 0), row(0, "-1.23E-12"));
+
+        // ================ double ================
+        execute("INSERT INTO %s (k, doubleval) VALUES (?, ?)", 0, 123123.123123d);
+        assertRows(execute("SELECT k, toJson(doubleval) FROM %s WHERE k = ?", 0), row(0, "123123.123123"));
+
+        execute("INSERT INTO %s (k, doubleval) VALUES (?, ?)", 0, 123123d);
+        assertRows(execute("SELECT k, toJson(doubleval) FROM %s WHERE k = ?", 0), row(0, "123123.0"));
+
+        // ================ float ================
+        execute("INSERT INTO %s (k, floatval) VALUES (?, ?)", 0, 123.123f);
+        assertRows(execute("SELECT k, toJson(floatval) FROM %s WHERE k = ?", 0), row(0, "123.123"));
+
+        execute("INSERT INTO %s (k, floatval) VALUES (?, ?)", 0, 123123f);
+        assertRows(execute("SELECT k, toJson(floatval) FROM %s WHERE k = ?", 0), row(0, "123123.0"));
+
+        // ================ inet ================
+        execute("INSERT INTO %s (k, inetval) VALUES (?, ?)", 0, InetAddress.getByName("127.0.0.1"));
+        assertRows(execute("SELECT k, toJson(inetval) FROM %s WHERE k = ?", 0), row(0, "\"127.0.0.1\""));
+
+        execute("INSERT INTO %s (k, inetval) VALUES (?, ?)", 0, InetAddress.getByName("::1"));
+        assertRows(execute("SELECT k, toJson(inetval) FROM %s WHERE k = ?", 0), row(0, "\"0:0:0:0:0:0:0:1\""));
+
+        // ================ int ================
+        execute("INSERT INTO %s (k, intval) VALUES (?, ?)", 0, 123123);
+        assertRows(execute("SELECT k, toJson(intval) FROM %s WHERE k = ?", 0), row(0, "123123"));
+
+        execute("INSERT INTO %s (k, intval) VALUES (?, ?)", 0, 0);
+        assertRows(execute("SELECT k, toJson(intval) FROM %s WHERE k = ?", 0), row(0, "0"));
+
+        execute("INSERT INTO %s (k, intval) VALUES (?, ?)", 0, -123123);
+        assertRows(execute("SELECT k, toJson(intval) FROM %s WHERE k = ?", 0), row(0, "-123123"));
+
+        // ================ text (varchar) ================
+        execute("INSERT INTO %s (k, textval) VALUES (?, ?)", 0, "");
+        assertRows(execute("SELECT k, toJson(textval) FROM %s WHERE k = ?", 0), row(0, "\"\""));
+
+        execute("INSERT INTO %s (k, textval) VALUES (?, ?)", 0, "abcd");
+        assertRows(execute("SELECT k, toJson(textval) FROM %s WHERE k = ?", 0), row(0, "\"abcd\""));
+
+        execute("INSERT INTO %s (k, textval) VALUES (?, ?)", 0, "\u8422");
+        assertRows(execute("SELECT k, toJson(textval) FROM %s WHERE k = ?", 0), row(0, "\"\u8422\""));
+
+        execute("INSERT INTO %s (k, textval) VALUES (?, ?)", 0, "\u0000");
+        assertRows(execute("SELECT k, toJson(textval) FROM %s WHERE k = ?", 0), row(0, "\"\\u0000\""));
+
+        // ================ timestamp ================
+        execute("INSERT INTO %s (k, timeval) VALUES (?, ?)", 0, 123L);
+        assertRows(execute("SELECT k, toJson(timeval) FROM %s WHERE k = ?", 0), row(0, "\"00:00:00.000000123\""));
+
+        // ================ timestamp ================
+        execute("INSERT INTO %s (k, timestampval) VALUES (?, ?)", 0, new SimpleDateFormat("y-M-d").parse("2014-01-01"));
+        assertRows(execute("SELECT k, toJson(timestampval) FROM %s WHERE k = ?", 0), row(0, "\"2014-01-01 00:00:00.000\""));
+
+        // ================ timeuuid ================
+        execute("INSERT INTO %s (k, timeuuidval) VALUES (?, ?)", 0, UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799"));
+        assertRows(execute("SELECT k, toJson(timeuuidval) FROM %s WHERE k = ?", 0), row(0, "\"6bddc89a-5644-11e4-97fc-56847afe9799\""));
+
+         // ================ uuidval ================
+        execute("INSERT INTO %s (k, uuidval) VALUES (?, ?)", 0, UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799"));
+        assertRows(execute("SELECT k, toJson(uuidval) FROM %s WHERE k = ?", 0), row(0, "\"6bddc89a-5644-11e4-97fc-56847afe9799\""));
+
+        // ================ varint ================
+        execute("INSERT INTO %s (k, varintval) VALUES (?, ?)", 0, new BigInteger("123123123123123123123"));
+        assertRows(execute("SELECT k, toJson(varintval) FROM %s WHERE k = ?", 0), row(0, "123123123123123123123"));
+
+        // ================ lists ================
+        execute("INSERT INTO %s (k, listval) VALUES (?, ?)", 0, list(1, 2, 3));
+        assertRows(execute("SELECT k, toJson(listval) FROM %s WHERE k = ?", 0), row(0, "[1, 2, 3]"));
+
+        execute("INSERT INTO %s (k, listval) VALUES (?, ?)", 0, list());
+        assertRows(execute("SELECT k, toJson(listval) FROM %s WHERE k = ?", 0), row(0, "null"));
+
+        // frozen
+        execute("INSERT INTO %s (k, frozenlistval) VALUES (?, ?)", 0, list(1, 2, 3));
+        assertRows(execute("SELECT k, toJson(frozenlistval) FROM %s WHERE k = ?", 0), row(0, "[1, 2, 3]"));
+
+        // ================ sets ================
+        execute("INSERT INTO %s (k, setval) VALUES (?, ?)",
+                0, set(UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9798"), (UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799"))));
+        assertRows(execute("SELECT k, toJson(setval) FROM %s WHERE k = ?", 0),
+                row(0, "[\"6bddc89a-5644-11e4-97fc-56847afe9798\", \"6bddc89a-5644-11e4-97fc-56847afe9799\"]")
+        );
+
+        execute("INSERT INTO %s (k, setval) VALUES (?, ?)", 0, set());
+        assertRows(execute("SELECT k, toJson(setval) FROM %s WHERE k = ?", 0), row(0, "null"));
+
+        // frozen
+        execute("INSERT INTO %s (k, frozensetval) VALUES (?, ?)",
+                0, set(UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9798"), (UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799"))));
+        assertRows(execute("SELECT k, toJson(frozensetval) FROM %s WHERE k = ?", 0),
+                row(0, "[\"6bddc89a-5644-11e4-97fc-56847afe9798\", \"6bddc89a-5644-11e4-97fc-56847afe9799\"]")
+        );
+
+        // ================ maps ================
+        execute("INSERT INTO %s (k, mapval) VALUES (?, ?)", 0, map("a", 1, "b", 2));
+        assertRows(execute("SELECT k, toJson(mapval) FROM %s WHERE k = ?", 0), row(0, "{\"a\": 1, \"b\": 2}"));
+
+        execute("INSERT INTO %s (k, mapval) VALUES (?, ?)", 0, map());
+        assertRows(execute("SELECT k, toJson(mapval) FROM %s WHERE k = ?", 0), row(0, "null"));
+
+        // frozen
+        execute("INSERT INTO %s (k, frozenmapval) VALUES (?, ?)", 0, map("a", 1, "b", 2));
+        assertRows(execute("SELECT k, toJson(frozenmapval) FROM %s WHERE k = ?", 0), row(0, "{\"a\": 1, \"b\": 2}"));
+
+        // ================ tuples ================
+        execute("INSERT INTO %s (k, tupleval) VALUES (?, ?)", 0, tuple(1, "foobar", UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799")));
+        assertRows(execute("SELECT k, toJson(tupleval) FROM %s WHERE k = ?", 0),
+            row(0, "[1, \"foobar\", \"6bddc89a-5644-11e4-97fc-56847afe9799\"]")
+        );
+
+        execute("INSERT INTO %s (k, tupleval) VALUES (?, ?)", 0, tuple(1, "foobar", null));
+        assertRows(execute("SELECT k, toJson(tupleval) FROM %s WHERE k = ?", 0),
+                row(0, "[1, \"foobar\", null]")
+        );
+
+        // ================ UDTs ================
+        execute("INSERT INTO %s (k, udtval) VALUES (?, {a: ?, b: ?, c: ?})", 0, 1, UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799"), set("foo", "bar"));
+        assertRows(execute("SELECT k, toJson(udtval) FROM %s WHERE k = ?", 0),
+                row(0, "{\"a\": 1, \"b\": \"6bddc89a-5644-11e4-97fc-56847afe9799\", \"c\": [\"bar\", \"foo\"]}")
+        );
+
+        execute("INSERT INTO %s (k, udtval) VALUES (?, {a: ?, b: ?})", 0, 1, UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799"));
+        assertRows(execute("SELECT k, toJson(udtval) FROM %s WHERE k = ?", 0),
+                row(0, "{\"a\": 1, \"b\": \"6bddc89a-5644-11e4-97fc-56847afe9799\", \"c\": null}")
+        );
+    }
+
+    @Test
+    public void testSelectJsonSyntax() throws Throwable
+    {
+        // tests SELECT JSON statements
+        createTable("CREATE TABLE %s (k int primary key, v int)");
+        execute("INSERT INTO %s (k, v) VALUES (0, 0)");
+        execute("INSERT INTO %s (k, v) VALUES (1, 1)");
+
+        assertRows(execute("SELECT JSON * FROM %s"),
+                row("{\"k\": 0, \"v\": 0}"),
+                row("{\"k\": 1, \"v\": 1}")
+        );
+
+        assertRows(execute("SELECT JSON k, v FROM %s"),
+                row("{\"k\": 0, \"v\": 0}"),
+                row("{\"k\": 1, \"v\": 1}")
+        );
+
+        assertRows(execute("SELECT JSON v, k FROM %s"),
+                row("{\"v\": 0, \"k\": 0}"),
+                row("{\"v\": 1, \"k\": 1}")
+        );
+
+        assertRows(execute("SELECT JSON v as foo, k as bar FROM %s"),
+                row("{\"foo\": 0, \"bar\": 0}"),
+                row("{\"foo\": 1, \"bar\": 1}")
+        );
+
+        assertRows(execute("SELECT JSON ttl(v), k FROM %s"),
+                row("{\"ttl(v)\": null, \"k\": 0}"),
+                row("{\"ttl(v)\": null, \"k\": 1}")
+        );
+
+        assertRows(execute("SELECT JSON ttl(v) as foo, k FROM %s"),
+                row("{\"foo\": null, \"k\": 0}"),
+                row("{\"foo\": null, \"k\": 1}")
+        );
+
+        assertRows(execute("SELECT JSON count(*) FROM %s"),
+                row("{\"count\": 2}")
+        );
+
+        assertRows(execute("SELECT JSON count(*) as foo FROM %s"),
+                row("{\"foo\": 2}")
+        );
+
+        assertRows(execute("SELECT JSON toJson(blobAsInt(intAsBlob(v))) FROM %s LIMIT 1"),
+                row("{\"system.tojson(system.blobasint(system.intasblob(v)))\": \"0\"}")
+        );
+    }
+
+    @Test
+    public void testInsertJsonSyntax() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int primary key, v int)");
+        execute("INSERT INTO %s JSON ?", "{\"k\": 0, \"v\": 0}");
+        assertRows(execute("SELECT * FROM %s"),
+                row(0, 0)
+        );
+
+        // without specifying column names
+        execute("INSERT INTO %s JSON ?", "{\"k\": 0, \"v\": 0}");
+        assertRows(execute("SELECT * FROM %s"),
+                row(0, 0)
+        );
+
+        execute("INSERT INTO %s JSON ?", "{\"k\": 0, \"v\": null}");
+        assertRows(execute("SELECT * FROM %s"),
+                row(0, null)
+        );
+
+        execute("INSERT INTO %s JSON ?", "{\"v\": 1, \"k\": 0}");
+        assertRows(execute("SELECT * FROM %s"),
+                row(0, 1)
+        );
+
+        execute("INSERT INTO %s JSON ?", "{\"k\": 0}");
+        assertRows(execute("SELECT * FROM %s"),
+                row(0, null)
+        );
+
+        if (USE_PREPARED_VALUES)
+            assertInvalidMessage("Got null for INSERT JSON values", "INSERT INTO %s JSON ?", new Object[]{null});
+
+        assertInvalidMessage("Got null for INSERT JSON values", "INSERT INTO %s JSON ?", "null");
+        assertInvalidMessage("Could not decode JSON string as a map", "INSERT INTO %s JSON ?", "\"notamap\"");
+        assertInvalidMessage("Could not decode JSON string as a map", "INSERT INTO %s JSON ?", "12.34");
+        assertInvalidMessage("JSON values map contains unrecognized column",
+                "INSERT INTO %s JSON ?",
+                "{\"k\": 0, \"v\": 0, \"zzz\": 0}");
+
+        assertInvalidMessage("Unable to make int from",
+                "INSERT INTO %s JSON ?",
+                "{\"k\": 0, \"v\": \"notanint\"}");
+    }
+
+    @Test
+    public void testCaseSensitivity() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int primary key, \"Foo\" int)");
+        execute("INSERT INTO %s JSON ?", "{\"k\": 0, \"\\\"Foo\\\"\": 0}");
+        execute("INSERT INTO %s JSON ?", "{\"K\": 0, \"\\\"Foo\\\"\": 0}");
+        execute("INSERT INTO %s JSON ?", "{\"\\\"k\\\"\": 0, \"\\\"Foo\\\"\": 0}");
+
+        // results should preserve and quote case-sensitive identifiers
+        assertRows(execute("SELECT JSON * FROM %s"), row("{\"k\": 0, \"\\\"Foo\\\"\": 0}"));
+        assertRows(execute("SELECT JSON k, \"Foo\" as foo FROM %s"), row("{\"k\": 0, \"foo\": 0}"));
+        assertRows(execute("SELECT JSON k, \"Foo\" as \"Bar\" FROM %s"), row("{\"k\": 0, \"\\\"Bar\\\"\": 0}"));
+
+        assertInvalid("INSERT INTO %s JSON ?", "{\"k\": 0, \"foo\": 0}");
+        assertInvalid("INSERT INTO %s JSON ?", "{\"k\": 0, \"\\\"foo\\\"\": 0}");
+
+        // user-defined types also need to handle case-sensitivity
+        String typeName = createType("CREATE TYPE %s (a int, \"Foo\" int)");
+        createTable("CREATE TABLE %s (k int primary key, v frozen<" + typeName + ">)");
+
+        execute("INSERT INTO %s JSON ?", "{\"k\": 0, \"v\": {\"a\": 0, \"\\\"Foo\\\"\": 0}}");
+        assertRows(execute("SELECT JSON k, v FROM %s"), row("{\"k\": 0, \"v\": {\"a\": 0, \"\\\"Foo\\\"\": 0}}"));
+
+        execute("INSERT INTO %s JSON ?", "{\"k\": 0, \"v\": {\"A\": 0, \"\\\"Foo\\\"\": 0}}");
+        assertRows(execute("SELECT JSON k, v FROM %s"), row("{\"k\": 0, \"v\": {\"a\": 0, \"\\\"Foo\\\"\": 0}}"));
+    }
+
+    @Test
+    public void testInsertJsonSyntaxWithCollections() throws Throwable
+    {
+        createTable("CREATE TABLE %s (" +
+                "k int PRIMARY KEY, " +
+                "m map<text, boolean>, " +
+                "mf frozen<map<text, boolean>>, " +
+                "s set<int>, " +
+                "sf frozen<set<int>>, " +
+                "l list<int>, " +
+                "lf frozen<list<int>>)");
+
+        // map
+        execute("INSERT INTO %s JSON ?", "{\"k\": 0, \"m\": {\"a\": true, \"b\": false}}");
+        assertRows(execute("SELECT k, m FROM %s"), row(0, map("a", true, "b", false)));
+
+        // frozen map
+        execute("INSERT INTO %s JSON ?", "{\"k\": 0, \"mf\": {\"a\": true, \"b\": false}}");
+        assertRows(execute("SELECT k, mf FROM %s"), row(0, map("a", true, "b", false)));
+
+        // set
+        execute("INSERT INTO %s JSON ?", "{\"k\": 0, \"s\": [3, 1, 2]}");
+        assertRows(execute("SELECT k, s FROM %s"), row(0, set(1, 2, 3)));
+
+        // frozen set
+        execute("INSERT INTO %s JSON ?", "{\"k\": 0, \"sf\": [3, 1, 2]}");
+        assertRows(execute("SELECT k, sf FROM %s"), row(0, set(1, 2, 3)));
+
+        // list
+        execute("INSERT INTO %s JSON ?", "{\"k\": 0, \"l\": [1, 2, 3]}");
+        assertRows(execute("SELECT k, l FROM %s"), row(0, list(1, 2, 3)));
+
+        // frozen list
+        execute("INSERT INTO %s JSON ?", "{\"k\": 0, \"lf\": [1, 2, 3]}");
+        assertRows(execute("SELECT k, lf FROM %s"), row(0, list(1, 2, 3)));
+    }
+
+    @Test
+    public void testInsertJsonSyntaxWithNonNativeMapKeys() throws Throwable
+    {
+        // JSON doesn't allow non-string keys, so we accept string representations of any type as map keys and
+        // return maps with string keys when necessary.
+
+        String typeName = createType("CREATE TYPE %s (a int)");
+        createTable("CREATE TABLE %s (" +
+                "k int PRIMARY KEY, " +
+                "intmap map<int, boolean>, " +
+                "bigintmap map<bigint, boolean>, " +
+                "varintmap map<varint, boolean>, " +
+                "booleanmap map<boolean, boolean>, " +
+                "floatmap map<float, boolean>, " +
+                "doublemap map<double, boolean>, " +
+                "decimalmap map<decimal, boolean>, " +
+                "tuplemap map<frozen<tuple<int, text>>, boolean>, " +
+                "udtmap map<frozen<" + typeName + ">, boolean>, " +
+                "setmap map<frozen<set<int>>, boolean>, " +
+                "listmap map<frozen<list<int>>, boolean>, " +
+                "textsetmap map<frozen<set<text>>, boolean>, " +
+                "nestedsetmap map<frozen<map<set<text>, text>>, boolean>, " +
+                "frozensetmap frozen<map<set<int>, boolean>>)");
+
+        // int keys
+        execute("INSERT INTO %s JSON ?", "{\"k\": 0, \"intmap\": {\"0\": true, \"1\": false}}");
+        assertRows(execute("SELECT JSON k, intmap FROM %s"), row("{\"k\": 0, \"intmap\": {\"0\": true, \"1\": false}}"));
+
+        // bigint keys
+        execute("INSERT INTO %s JSON ?", "{\"k\": 0, \"bigintmap\": {\"0\": true, \"1\": false}}");
+        assertRows(execute("SELECT JSON k, bigintmap FROM %s"), row("{\"k\": 0, \"bigintmap\": {\"0\": true, \"1\": false}}"));
+
+        // varint keys
+        execute("INSERT INTO %s JSON ?", "{\"k\": 0, \"varintmap\": {\"0\": true, \"1\": false}}");
+        assertRows(execute("SELECT JSON k, varintmap FROM %s"), row("{\"k\": 0, \"varintmap\": {\"0\": true, \"1\": false}}"));
+
+        // boolean keys
+        execute("INSERT INTO %s JSON ?", "{\"k\": 0, \"booleanmap\": {\"true\": true, \"false\": false}}");
+        assertRows(execute("SELECT JSON k, booleanmap FROM %s"), row("{\"k\": 0, \"booleanmap\": {\"false\": false, \"true\": true}}"));
+
+        // float keys
+        execute("INSERT INTO %s JSON ?", "{\"k\": 0, \"floatmap\": {\"1.23\": true, \"4.56\": false}}");
+        assertRows(execute("SELECT JSON k, floatmap FROM %s"), row("{\"k\": 0, \"floatmap\": {\"1.23\": true, \"4.56\": false}}"));
+
+        // double keys
+        execute("INSERT INTO %s JSON ?", "{\"k\": 0, \"doublemap\": {\"1.23\": true, \"4.56\": false}}");
+        assertRows(execute("SELECT JSON k, doublemap FROM %s"), row("{\"k\": 0, \"doublemap\": {\"1.23\": true, \"4.56\": false}}"));
+
+        // decimal keys
+        execute("INSERT INTO %s JSON ?", "{\"k\": 0, \"decimalmap\": {\"1.23\": true, \"4.56\": false}}");
+        assertRows(execute("SELECT JSON k, decimalmap FROM %s"), row("{\"k\": 0, \"decimalmap\": {\"1.23\": true, \"4.56\": false}}"));
+
+        // tuple<int, text> keys
+        execute("INSERT INTO %s JSON ?", "{\"k\": 0, \"tuplemap\": {\"[0, \\\"a\\\"]\": true, \"[1, \\\"b\\\"]\": false}}");
+        assertRows(execute("SELECT JSON k, tuplemap FROM %s"), row("{\"k\": 0, \"tuplemap\": {\"[0, \\\"a\\\"]\": true, \"[1, \\\"b\\\"]\": false}}"));
+
+        // UDT keys
+        execute("INSERT INTO %s JSON ?", "{\"k\": 0, \"udtmap\": {\"{\\\"a\\\": 0}\": true, \"{\\\"a\\\": 1}\": false}}");
+        assertRows(execute("SELECT JSON k, udtmap FROM %s"), row("{\"k\": 0, \"udtmap\": {\"{\\\"a\\\": 0}\": true, \"{\\\"a\\\": 1}\": false}}"));
+
+        // set<int> keys
+        execute("INSERT INTO %s JSON ?", "{\"k\": 0, \"setmap\": {\"[0, 1, 2]\": true, \"[3, 4, 5]\": false}}");
+        assertRows(execute("SELECT JSON k, setmap FROM %s"), row("{\"k\": 0, \"setmap\": {\"[0, 1, 2]\": true, \"[3, 4, 5]\": false}}"));
+
+        // list<int> keys
+        execute("INSERT INTO %s JSON ?", "{\"k\": 0, \"listmap\": {\"[0, 1, 2]\": true, \"[3, 4, 5]\": false}}");
+        assertRows(execute("SELECT JSON k, listmap FROM %s"), row("{\"k\": 0, \"listmap\": {\"[0, 1, 2]\": true, \"[3, 4, 5]\": false}}"));
+
+        // set<text> keys
+        execute("INSERT INTO %s JSON ?", "{\"k\": 0, \"textsetmap\": {\"[\\\"0\\\", \\\"1\\\"]\": true, \"[\\\"3\\\", \\\"4\\\"]\": false}}");
+        assertRows(execute("SELECT JSON k, textsetmap FROM %s"), row("{\"k\": 0, \"textsetmap\": {\"[\\\"0\\\", \\\"1\\\"]\": true, \"[\\\"3\\\", \\\"4\\\"]\": false}}"));
+
+        // map<set<text>, text> keys
+        String innerKey1 = "[\"0\", \"1\"]";
+        String fullKey1 = String.format("{\"%s\": \"%s\"}", new String(Json.JSON_STRING_ENCODER.quoteAsString(innerKey1)), "a");
+        String stringKey1 = new String(Json.JSON_STRING_ENCODER.quoteAsString(fullKey1));
+        String innerKey2 = "[\"3\", \"4\"]";
+        String fullKey2 = String.format("{\"%s\": \"%s\"}", new String(Json.JSON_STRING_ENCODER.quoteAsString(innerKey2)), "b");
+        String stringKey2 = new String(Json.JSON_STRING_ENCODER.quoteAsString(fullKey2));
+        execute("INSERT INTO %s JSON ?", "{\"k\": 0, \"nestedsetmap\": {\"" + stringKey1 + "\": true, \"" + stringKey2 + "\": false}}");
+        assertRows(execute("SELECT JSON k, nestedsetmap FROM %s"), row("{\"k\": 0, \"nestedsetmap\": {\"" + stringKey1 + "\": true, \"" + stringKey2 + "\": false}}"));
+
+        // set<int> keys in a frozen map
+        execute("INSERT INTO %s JSON ?", "{\"k\": 0, \"frozensetmap\": {\"[0, 1, 2]\": true, \"[3, 4, 5]\": false}}");
+        assertRows(execute("SELECT JSON k, frozensetmap FROM %s"), row("{\"k\": 0, \"frozensetmap\": {\"[0, 1, 2]\": true, \"[3, 4, 5]\": false}}"));
+    }
+
+    @Test
+    public void testInsertJsonSyntaxWithTuplesAndUDTs() throws Throwable
+    {
+        String typeName = createType("CREATE TYPE %s (a int, b frozen<set<int>>, c tuple<int, int>)");
+        createTable("CREATE TABLE %s (" +
+                "k int PRIMARY KEY, " +
+                "a frozen<" + typeName + ">, " +
+                "b tuple<int, boolean>)");
+
+        execute("INSERT INTO %s JSON ?", "{\"k\": 0, \"a\": {\"a\": 0, \"b\": [1, 2, 3], \"c\": [0, 1]}, \"b\": [0, true]}");
+        assertRows(execute("SELECT k, a.a, a.b, a.c, b FROM %s"), row(0, 0, set(1, 2, 3), tuple(0, 1), tuple(0, true)));
+
+        execute("INSERT INTO %s JSON ?", "{\"k\": 0, \"a\": {\"a\": 0, \"b\": [1, 2, 3], \"c\": null}, \"b\": null}");
+        assertRows(execute("SELECT k, a.a, a.b, a.c, b FROM %s"), row(0, 0, set(1, 2, 3), null, null));
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/01115f72/test/unit/org/apache/cassandra/cql3/validation/entities/SecondaryIndexOnMapEntriesTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/SecondaryIndexOnMapEntriesTest.java b/test/unit/org/apache/cassandra/cql3/validation/entities/SecondaryIndexOnMapEntriesTest.java
new file mode 100644
index 0000000..fb0d027
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/SecondaryIndexOnMapEntriesTest.java
@@ -0,0 +1,348 @@
+/*
+ * 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.validation.entities;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.cql3.UntypedResultSet;
+import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.dht.ByteOrderedPartitioner;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.commons.lang3.StringUtils;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+public class SecondaryIndexOnMapEntriesTest extends CQLTester
+{
+    @BeforeClass
+    public static void setUp()
+    {
+        DatabaseDescriptor.setPartitioner(ByteOrderedPartitioner.instance);
+    }
+
+    @Test
+    public void testShouldNotCreateIndexOnFrozenMaps() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k TEXT PRIMARY KEY, v FROZEN<MAP<TEXT, TEXT>>)");
+        assertIndexInvalidForColumn("v");
+    }
+
+    @Test
+    public void testShouldNotCreateIndexOnNonMapTypes() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k TEXT PRIMARY KEY, i INT, t TEXT, b BLOB, s SET<TEXT>, l LIST<TEXT>, tu TUPLE<TEXT>)");
+        assertIndexInvalidForColumn("i");
+        assertIndexInvalidForColumn("t");
+        assertIndexInvalidForColumn("b");
+        assertIndexInvalidForColumn("s");
+        assertIndexInvalidForColumn("l");
+        assertIndexInvalidForColumn("tu");
+    }
+
+    @Test
+    public void testShouldValidateMapKeyAndValueTypes() throws Throwable
+    {
+        createSimpleTableAndIndex();
+
+        String query = "SELECT * FROM %s WHERE v[?] = ?";
+        Object validKey = "valid key";
+        Object invalidKey = 31415;
+        Object validValue = 31415;
+        Object invalidValue = "invalid value";
+        assertInvalid(query, invalidKey, invalidValue);
+        assertInvalid(query, invalidKey, validValue);
+        assertInvalid(query, validKey, invalidValue);
+        assertReturnsNoRows(query, validKey, validValue);
+    }
+
+    @Test
+    public void testShouldFindRowsMatchingSingleEqualityRestriction() throws Throwable
+    {
+        createSimpleTableAndIndex();
+        Object[] foo = insertIntoSimpleTable("foo", map("a", 1,
+                                                        "c", 3));
+        Object[] bar = insertIntoSimpleTable("bar", map("a", 1,
+                                                        "b", 2));
+        Object[] baz = insertIntoSimpleTable("baz", map("b", 2,
+                                                        "c", 5,
+                                                        "d", 4));
+        Object[] qux = insertIntoSimpleTable("qux", map("b", 2,
+                                                        "d", 4));
+
+        assertRowsForConditions(entry("a", 1), bar, foo);
+        assertRowsForConditions(entry("b", 2), bar, baz, qux);
+        assertRowsForConditions(entry("c", 3), foo);
+        assertRowsForConditions(entry("c", 5), baz);
+        assertRowsForConditions(entry("d", 4), baz, qux);
+    }
+
+    @Test
+    public void testRequireFilteringDirectiveIfMultipleRestrictionsSpecified() throws Throwable
+    {
+        createSimpleTableAndIndex();
+        String baseQuery = "SELECT * FROM %s WHERE v['foo'] = 31415 AND v['baz'] = 31416";
+        assertInvalid(baseQuery);
+        assertReturnsNoRows(baseQuery + " ALLOW FILTERING");
+    }
+
+    @Test
+    public void testShouldFindRowsMatchingMultipleEqualityRestrictions() throws Throwable
+    {
+        createSimpleTableAndIndex();
+
+        Object[] foo = insertIntoSimpleTable("foo", map("k1", 1));
+        Object[] bar = insertIntoSimpleTable("bar", map("k1", 1,
+                                                        "k2", 2));
+        Object[] baz = insertIntoSimpleTable("baz", map("k2", 2,
+                                                        "k3", 3));
+        Object[] qux = insertIntoSimpleTable("qux", map("k2", 2,
+                                                        "k3", 3,
+                                                        "k4", 4));
+
+        assertRowsForConditions(entry("k1", 1),
+                                bar, foo);
+        assertRowsForConditions(entry("k1", 1).entry("k2", 2),
+                                bar);
+        assertNoRowsForConditions(entry("k1", 1).entry("k2", 2).entry("k3", 3));
+        assertRowsForConditions(entry("k2", 2).entry("k3", 3),
+                                baz, qux);
+        assertRowsForConditions(entry("k2", 2).entry("k3", 3).entry("k4", 4),
+                                qux);
+        assertRowsForConditions(entry("k3", 3).entry("k4", 4),
+                                qux);
+        assertNoRowsForConditions(entry("k3", 3).entry("k4", 4).entry("k5", 5));
+    }
+
+    @Test
+    public void testShouldFindRowsMatchingEqualityAndContainsRestrictions() throws Throwable
+    {
+        createSimpleTableAndIndex();
+
+        Object[] foo = insertIntoSimpleTable("foo", map("common", 31415,
+                                                        "k1", 1,
+                                                        "k2", 2,
+                                                        "k3", 3));
+        Object[] bar = insertIntoSimpleTable("bar", map("common", 31415,
+                                                        "k3", 3,
+                                                        "k4", 4,
+                                                        "k5", 5));
+        Object[] baz = insertIntoSimpleTable("baz", map("common", 31415,
+                                                        "k5", 5,
+                                                        "k6", 6,
+                                                        "k7", 7));
+
+        assertRowsForConditions(entry("common", 31415),
+                                bar, baz, foo);
+        assertRowsForConditions(entry("common", 31415).key("k1"),
+                                foo);
+        assertRowsForConditions(entry("common", 31415).key("k2"),
+                                foo);
+        assertRowsForConditions(entry("common", 31415).key("k3"),
+                                bar, foo);
+        assertRowsForConditions(entry("common", 31415).key("k3").value(2),
+                                foo);
+        assertRowsForConditions(entry("common", 31415).key("k3").value(3),
+                                bar, foo);
+        assertRowsForConditions(entry("common", 31415).key("k3").value(4),
+                                bar);
+        assertRowsForConditions(entry("common", 31415).key("k3").key("k5"),
+                                bar);
+        assertRowsForConditions(entry("common", 31415).key("k5"),
+                                bar, baz);
+        assertRowsForConditions(entry("common", 31415).key("k5").value(4),
+                                bar);
+        assertRowsForConditions(entry("common", 31415).key("k5").value(5),
+                                bar, baz);
+        assertRowsForConditions(entry("common", 31415).key("k5").value(6),
+                                baz);
+        assertNoRowsForConditions(entry("common", 31415).key("k5").value(8));
+    }
+
+    @Test
+    public void testShouldNotAcceptUnsupportedRelationsOnEntries() throws Throwable
+    {
+        createSimpleTableAndIndex();
+        assertInvalidRelation("< 31415");
+        assertInvalidRelation("<= 31415");
+        assertInvalidRelation("> 31415");
+        assertInvalidRelation(">= 31415");
+        assertInvalidRelation("IN (31415, 31416, 31417)");
+        assertInvalidRelation("CONTAINS 31415");
+        assertInvalidRelation("CONTAINS KEY 'foo'");
+    }
+
+    @Test
+    public void testShouldRecognizeAlteredOrDeletedMapEntries() throws Throwable
+    {
+        createSimpleTableAndIndex();
+        Object[] foo = insertIntoSimpleTable("foo", map("common", 31415,
+                                                        "target", 8192));
+        Object[] bar = insertIntoSimpleTable("bar", map("common", 31415,
+                                                        "target", 8192));
+        Object[] baz = insertIntoSimpleTable("baz", map("common", 31415,
+                                                        "target", 8192));
+
+        assertRowsForConditions(entry("target", 8192),
+                                bar, baz, foo);
+        baz = updateMapInSimpleTable(baz, "target", 4096);
+        assertRowsForConditions(entry("target", 8192),
+                                bar, foo);
+        bar = updateMapInSimpleTable(bar, "target", null);
+        assertRowsForConditions(entry("target", 8192),
+                                foo);
+        execute("DELETE FROM %s WHERE k = 'foo'");
+        assertNoRowsForConditions(entry("target", 8192));
+        assertRowsForConditions(entry("common", 31415),
+                                bar, baz);
+        assertRowsForConditions(entry("target", 4096),
+                                baz);
+    }
+
+    @Test
+    public void testShouldRejectQueriesForNullEntries() throws Throwable
+    {
+        createSimpleTableAndIndex();
+        assertInvalid("SELECT * FROM %s WHERE v['somekey'] = null");
+    }
+
+    @Test
+    public void testShouldTreatQueriesAgainstFrozenMapIndexesAsInvalid() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k TEXT PRIMARY KEY, v FROZEN<MAP<TEXT, TEXT>>)");
+        createIndex("CREATE INDEX ON %s(FULL(V))");
+
+        try
+        {
+            execute("SELECT * FROM %s WHERE v['somekey'] = 'somevalue'");
+            fail("Expected index query to fail");
+        }
+        catch (InvalidRequestException e)
+        {
+            String expectedMessage = "Map-entry equality predicates on frozen map column v are not supported";
+            assertTrue("Expected error message to contain '" + expectedMessage + "' but got '" +
+                       e.getMessage() + "'", e.getMessage().contains(expectedMessage));
+        }
+    }
+
+    private void assertIndexInvalidForColumn(String colname) throws Throwable
+    {
+        String query = String.format("CREATE INDEX ON %%s(ENTRIES(%s))", colname);
+        assertInvalid(query);
+    }
+
+    private void assertReturnsNoRows(String query, Object... params) throws Throwable
+    {
+        assertRows(execute(query, params));
+    }
+
+    private void createSimpleTableAndIndex() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k TEXT PRIMARY KEY, v MAP<TEXT, INT>)");
+        createIndex("CREATE INDEX ON %s(ENTRIES(v))");
+    }
+
+    private Object[] insertIntoSimpleTable(String key, Object value) throws Throwable
+    {
+        String query = "INSERT INTO %s (k, v) VALUES (?, ?)";
+        execute(query, key, value);
+        return row(key, value);
+    }
+
+    private void assertRowsForConditions(IndexWhereClause whereClause, Object[]... rows) throws Throwable
+    {
+        assertRows(execute("SELECT * FROM %s WHERE " + whereClause.text(), whereClause.params()), rows);
+    }
+
+    private void assertNoRowsForConditions(IndexWhereClause whereClause) throws Throwable
+    {
+        assertRowsForConditions(whereClause);
+    }
+
+    private void assertInvalidRelation(String rel) throws Throwable
+    {
+        String query = "SELECT * FROM %s WHERE v " + rel;
+        assertInvalid(query);
+    }
+
+    private Object[] updateMapInSimpleTable(Object[] row, String mapKey, Integer mapValue) throws Throwable
+    {
+        execute("UPDATE %s SET v[?] = ? WHERE k = ?", mapKey, mapValue, row[0]);
+        UntypedResultSet rawResults = execute("SELECT * FROM %s WHERE k = ?", row[0]);
+        Map<Object, Object> value = (Map<Object, Object>)row[1];
+        if (mapValue == null)
+        {
+            value.remove(mapKey);
+        }
+        else
+        {
+            value.put(mapKey, mapValue);
+        }
+        return row;
+    }
+
+    private IndexWhereClause entry(Object key, Object value)
+    {
+        return (new IndexWhereClause()).entry(key, value);
+    }
+
+    private static final class IndexWhereClause
+    {
+        private final List<String> preds = new ArrayList<>();
+        private final List<Object> params = new ArrayList<>();
+
+        public IndexWhereClause entry(Object key, Object value)
+        {
+            preds.add("v[?] = ?");
+            params.add(key);
+            params.add(value);
+            return this;
+        }
+
+        public IndexWhereClause key(Object key)
+        {
+            preds.add("v CONTAINS KEY ?");
+            params.add(key);
+            return this;
+        }
+
+        public IndexWhereClause value(Object value)
+        {
+            preds.add("v CONTAINS ?");
+            params.add(value);
+            return this;
+        }
+
+        public String text()
+        {
+            if (preds.size() == 1)
+                return preds.get(0);
+            return StringUtils.join(preds, " AND ") + " ALLOW FILTERING";
+        }
+
+        public Object[] params()
+        {
+            return params.toArray();
+        }
+    }
+}


[32/32] cassandra git commit: Merge branch 'cassandra-2.2' into trunk

Posted by jm...@apache.org.
Merge branch 'cassandra-2.2' into trunk


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

Branch: refs/heads/trunk
Commit: bc9b0be321b34c55366a61418b94e9bae6ddb26b
Parents: 7392fb9 01115f7
Author: Josh McKenzie <jo...@datastax.com>
Authored: Wed Jun 24 12:13:15 2015 -0400
Committer: Josh McKenzie <jo...@datastax.com>
Committed: Wed Jun 24 12:13:15 2015 -0400

----------------------------------------------------------------------
 .../cassandra/config/DatabaseDescriptor.java    |    6 +
 .../org/apache/cassandra/cql3/ResultSet.java    |    9 +
 .../apache/cassandra/cql3/UntypedResultSet.java |    2 +-
 .../cql3/statements/BatchStatement.java         |   69 +-
 .../cql3/statements/CQL3CasRequest.java         |    8 +-
 .../cql3/statements/ModificationStatement.java  |   70 +-
 .../cql3/statements/SelectStatement.java        |   84 +-
 .../cql3/statements/TruncateStatement.java      |   13 +-
 .../apache/cassandra/service/StorageProxy.java  |    4 +-
 .../org/apache/cassandra/utils/UUIDGen.java     |   16 +-
 .../org/apache/cassandra/cql3/ManyRowsTest.java |   92 +
 .../apache/cassandra/cql3/AggregationTest.java  | 1479 ----------
 .../org/apache/cassandra/cql3/AliasTest.java    |   40 -
 .../apache/cassandra/cql3/AlterTableTest.java   |  113 -
 .../org/apache/cassandra/cql3/CQLTester.java    |  172 +-
 .../apache/cassandra/cql3/CollectionsTest.java  |  340 ---
 .../cassandra/cql3/ContainsRelationTest.java    |  283 --
 .../cassandra/cql3/CrcCheckChanceTest.java      |  159 --
 .../cql3/CreateAndAlterKeyspaceTest.java        |   37 -
 .../cql3/CreateIndexStatementTest.java          |  101 -
 .../apache/cassandra/cql3/CreateTableTest.java  |   69 -
 .../cql3/CreateTriggerStatementTest.java        |  121 -
 .../cassandra/cql3/FrozenCollectionsTest.java   | 1101 --------
 .../cql3/IndexedValuesValidationTest.java       |  149 -
 .../org/apache/cassandra/cql3/JsonTest.java     |  947 -------
 .../apache/cassandra/cql3/ModificationTest.java |  112 -
 .../cassandra/cql3/MultiColumnRelationTest.java |  936 -------
 .../org/apache/cassandra/cql3/PgStringTest.java |   76 -
 .../cassandra/cql3/RangeDeletionTest.java       |   35 -
 .../apache/cassandra/cql3/RoleSyntaxTest.java   |   51 -
 .../cql3/SSTableMetadataTrackingTest.java       |  160 --
 .../cql3/SecondaryIndexOnMapEntriesTest.java    |  337 ---
 .../cql3/SelectWithTokenFunctionTest.java       |  233 --
 .../cassandra/cql3/SelectionOrderingTest.java   |  233 --
 .../cql3/SingleColumnRelationTest.java          |  553 ----
 .../SliceQueryFilterWithTombstonesTest.java     |  170 --
 .../cassandra/cql3/StaticColumnsQueryTest.java  |  280 --
 .../cassandra/cql3/ThriftCompatibilityTest.java |    1 +
 .../apache/cassandra/cql3/TimestampTest.java    |   36 -
 .../apache/cassandra/cql3/TupleTypeTest.java    |  114 -
 .../org/apache/cassandra/cql3/TypeCastTest.java |   54 -
 .../org/apache/cassandra/cql3/TypeTest.java     |   89 -
 .../org/apache/cassandra/cql3/UFAuthTest.java   |  724 -----
 .../cassandra/cql3/UFIdentificationTest.java    |  376 ---
 test/unit/org/apache/cassandra/cql3/UFTest.java | 2585 -----------------
 .../apache/cassandra/cql3/UseStatementTest.java |   29 -
 .../apache/cassandra/cql3/UserTypesTest.java    |  334 ---
 .../selection/SelectionColumnMappingTest.java   |    9 +
 .../validation/entities/CollectionsTest.java    |  588 ++++
 .../cql3/validation/entities/CountersTest.java  |  115 +
 .../cql3/validation/entities/DateTypeTest.java  |   39 +
 .../entities/FrozenCollectionsTest.java         | 1111 ++++++++
 .../cql3/validation/entities/JsonTest.java      |  958 +++++++
 .../SecondaryIndexOnMapEntriesTest.java         |  348 +++
 .../validation/entities/SecondaryIndexTest.java |  645 +++++
 .../validation/entities/StaticColumnsTest.java  |  271 ++
 .../cql3/validation/entities/TimestampTest.java |  155 ++
 .../cql3/validation/entities/TimeuuidTest.java  |   81 +
 .../cql3/validation/entities/TupleTypeTest.java |  171 ++
 .../cql3/validation/entities/TypeTest.java      |   92 +
 .../cql3/validation/entities/UFAuthTest.java    |  728 +++++
 .../entities/UFIdentificationTest.java          |  380 +++
 .../cql3/validation/entities/UFTest.java        | 2596 ++++++++++++++++++
 .../cql3/validation/entities/UserTypesTest.java |  404 +++
 .../miscellaneous/CrcCheckChanceTest.java       |  160 ++
 .../validation/miscellaneous/OverflowTest.java  |  331 +++
 .../validation/miscellaneous/PgStringTest.java  |   77 +
 .../miscellaneous/RoleSyntaxTest.java           |   53 +
 .../SSTableMetadataTrackingTest.java            |  161 ++
 .../miscellaneous/TombstonesTest.java           |  171 ++
 .../validation/operations/AggregationTest.java  | 1481 ++++++++++
 .../cql3/validation/operations/AlterTest.java   |  203 ++
 .../cql3/validation/operations/BatchTest.java   |  106 +
 .../cql3/validation/operations/CreateTest.java  |  498 ++++
 .../cql3/validation/operations/DeleteTest.java  |  329 +++
 .../cql3/validation/operations/InsertTest.java  |   59 +
 .../operations/InsertUpdateIfCondition.java     |  861 ++++++
 .../validation/operations/SelectLimitTest.java  |  112 +
 .../SelectMultiColumnRelationTest.java          |  962 +++++++
 .../operations/SelectOrderByTest.java           |  504 ++++
 .../SelectOrderedPartitionerTest.java           |  481 ++++
 .../SelectSingleColumnRelationTest.java         |  555 ++++
 .../cql3/validation/operations/SelectTest.java  | 1336 +++++++++
 .../cql3/validation/operations/UpdateTest.java  |   86 +
 .../cql3/validation/operations/UseTest.java     |   31 +
 .../cassandra/service/ClientWarningsTest.java   |    5 +-
 .../cassandra/transport/MessagePayloadTest.java |    6 +-
 .../stress/generate/values/TimeUUIDs.java       |    2 +-
 88 files changed, 17720 insertions(+), 12543 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/bc9b0be3/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
----------------------------------------------------------------------


[16/32] cassandra git commit: 2.2 commit for CASSANDRA-9160

Posted by jm...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/01115f72/test/unit/org/apache/cassandra/cql3/validation/entities/UserTypesTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/UserTypesTest.java b/test/unit/org/apache/cassandra/cql3/validation/entities/UserTypesTest.java
new file mode 100644
index 0000000..7274cd4
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/UserTypesTest.java
@@ -0,0 +1,404 @@
+/*
+ * 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.validation.entities;
+
+import java.util.UUID;
+
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.dht.ByteOrderedPartitioner;
+
+public class UserTypesTest extends CQLTester
+{
+    @BeforeClass
+    public static void setUpClass()
+    {
+        DatabaseDescriptor.setPartitioner(new ByteOrderedPartitioner());
+    }
+
+    @Test
+    public void testInvalidField() throws Throwable
+    {
+        String myType = createType("CREATE TYPE %s (f int)");
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, v frozen<" + myType + ">)");
+
+        // 's' is not a field of myType
+        assertInvalid("INSERT INTO %s (k, v) VALUES (?, {s : ?})", 0, 1);
+    }
+
+    @Test
+    public void testCassandra8105() throws Throwable
+    {
+        String ut1 = createType("CREATE TYPE %s (a int, b int)");
+        String ut2 = createType("CREATE TYPE %s (j frozen<" + KEYSPACE + "." + ut1 + ">, k int)");
+        createTable("CREATE TABLE %s (x int PRIMARY KEY, y set<frozen<" + KEYSPACE + "." + ut2 + ">>)");
+        execute("INSERT INTO %s (x, y) VALUES (1, { { k: 1 } })");
+
+        String ut3 = createType("CREATE TYPE %s (a int, b int)");
+        String ut4 = createType("CREATE TYPE %s (j frozen<" + KEYSPACE + "." + ut3 + ">, k int)");
+        createTable("CREATE TABLE %s (x int PRIMARY KEY, y list<frozen<" + KEYSPACE + "." + ut4 + ">>)");
+        execute("INSERT INTO %s (x, y) VALUES (1, [ { k: 1 } ])");
+
+        String ut5 = createType("CREATE TYPE %s (a int, b int)");
+        String ut6 = createType("CREATE TYPE %s (i int, j frozen<" + KEYSPACE + "." + ut5 + ">)");
+        createTable("CREATE TABLE %s (x int PRIMARY KEY, y set<frozen<" + KEYSPACE + "." + ut6 + ">>)");
+        execute("INSERT INTO %s (x, y) VALUES (1, { { i: 1 } })");
+    }
+
+    @Test
+    public void testFor7684() throws Throwable
+    {
+        String myType = createType("CREATE TYPE %s (x double)");
+        createTable("CREATE TABLE %s (k int, v frozen<" + myType + ">, b boolean static, PRIMARY KEY (k, v))");
+
+        execute("INSERT INTO %s(k, v) VALUES (?, {x:?})", 1, -104.99251);
+        execute("UPDATE %s SET b = ? WHERE k = ?", true, 1);
+
+        assertRows(execute("SELECT v.x FROM %s WHERE k = ? AND v = {x:?}", 1, -104.99251),
+            row(-104.99251)
+        );
+
+        flush();
+
+        assertRows(execute("SELECT v.x FROM %s WHERE k = ? AND v = {x:?}", 1, -104.99251),
+                   row(-104.99251)
+        );
+    }
+
+    @Test
+    public void testCreateInvalidTablesWithUDT() throws Throwable
+    {
+        String myType = createType("CREATE TYPE %s (f int)");
+
+        // Using a UDT without frozen shouldn't work
+        assertInvalidMessage("Non-frozen User-Defined types are not supported, please use frozen<>",
+                             "CREATE TABLE " + KEYSPACE + ".wrong (k int PRIMARY KEY, v " + KEYSPACE + '.' + myType + ")");
+
+        assertInvalidMessage("Statement on keyspace " + KEYSPACE + " cannot refer to a user type in keyspace otherkeyspace;" +
+                             " user types can only be used in the keyspace they are defined in",
+                             "CREATE TABLE " + KEYSPACE + ".wrong (k int PRIMARY KEY, v frozen<otherKeyspace.myType>)");
+
+        assertInvalidMessage("Unknown type " + KEYSPACE + ".unknowntype",
+                             "CREATE TABLE " + KEYSPACE + ".wrong (k int PRIMARY KEY, v frozen<" + KEYSPACE + '.' + "unknownType>)");
+    }
+
+    @Test
+    public void testAlterUDT() throws Throwable
+    {
+        String myType = KEYSPACE + '.' + createType("CREATE TYPE %s (a int)");
+        createTable("CREATE TABLE %s (a int PRIMARY KEY, b frozen<" + myType + ">)");
+        execute("INSERT INTO %s (a, b) VALUES (1, {a: 1})");
+
+        assertRows(execute("SELECT b.a FROM %s"), row(1));
+
+        flush();
+
+        execute("ALTER TYPE " + myType + " ADD b int");
+        execute("INSERT INTO %s (a, b) VALUES (2, {a: 2, b :2})");
+
+        assertRows(execute("SELECT b.a, b.b FROM %s"),
+                   row(1, null),
+                   row(2, 2));
+
+        flush();
+
+        assertRows(execute("SELECT b.a, b.b FROM %s"),
+                   row(1, null),
+                   row(2, 2));
+    }
+
+    @Test
+    public void testUDTWithUnsetValues() throws Throwable
+    {
+        // set up
+        String myType = createType("CREATE TYPE %s (x int, y int)");
+        String myOtherType = createType("CREATE TYPE %s (a frozen<" + myType + ">)");
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, v frozen<" + myType + ">, z frozen<" + myOtherType + ">)");
+
+        assertInvalidMessage("Invalid unset value for field 'y' of user defined type " + myType,
+                             "INSERT INTO %s (k, v) VALUES (10, {x:?, y:?})", 1, unset());
+
+        assertInvalidMessage("Invalid unset value for field 'y' of user defined type " + myType,
+                             "INSERT INTO %s (k, v, z) VALUES (10, {x:?, y:?}, {a:{x: ?, y: ?}})", 1, 1, 1, unset());
+    }
+
+    @Test
+    public void testAlteringUserTypeNestedWithinMap() throws Throwable
+    {
+        // test frozen and non-frozen collections
+        String[] columnTypePrefixes = {"frozen<map<text, ", "map<text, frozen<"};
+        for (String columnTypePrefix : columnTypePrefixes)
+        {
+            String ut1 = createType("CREATE TYPE %s (a int)");
+            String columnType = columnTypePrefix + KEYSPACE + "." + ut1 + ">>";
+
+            createTable("CREATE TABLE %s (x int PRIMARY KEY, y " + columnType + ")");
+
+            execute("INSERT INTO %s (x, y) VALUES(1, {'firstValue':{a:1}})");
+            assertRows(execute("SELECT * FROM %s"), row(1, map("firstValue", userType(1))));
+            flush();
+
+            execute("ALTER TYPE " + KEYSPACE + "." + ut1 + " ADD b int");
+            execute("INSERT INTO %s (x, y) VALUES(2, {'secondValue':{a:2, b:2}})");
+            execute("INSERT INTO %s (x, y) VALUES(3, {'thirdValue':{a:3}})");
+            execute("INSERT INTO %s (x, y) VALUES(4, {'fourthValue':{b:4}})");
+
+            assertRows(execute("SELECT * FROM %s"),
+                    row(1, map("firstValue", userType(1))),
+                    row(2, map("secondValue", userType(2, 2))),
+                    row(3, map("thirdValue", userType(3, null))),
+                    row(4, map("fourthValue", userType(null, 4))));
+
+            flush();
+
+            assertRows(execute("SELECT * FROM %s"),
+                    row(1, map("firstValue", userType(1))),
+                    row(2, map("secondValue", userType(2, 2))),
+                    row(3, map("thirdValue", userType(3, null))),
+                    row(4, map("fourthValue", userType(null, 4))));
+        }
+    }
+
+    @Test
+    public void testAlteringUserTypeNestedWithinSet() throws Throwable
+    {
+        // test frozen and non-frozen collections
+        String[] columnTypePrefixes = {"frozen<set<", "set<frozen<"};
+        for (String columnTypePrefix : columnTypePrefixes)
+        {
+            String ut1 = createType("CREATE TYPE %s (a int)");
+            String columnType = columnTypePrefix + KEYSPACE + "." + ut1 + ">>";
+
+            createTable("CREATE TABLE %s (x int PRIMARY KEY, y " + columnType + ")");
+
+            execute("INSERT INTO %s (x, y) VALUES(1, {1} )");
+            assertRows(execute("SELECT * FROM %s"), row(1, set(userType(1))));
+            flush();
+
+            execute("ALTER TYPE " + KEYSPACE + "." + ut1 + " ADD b int");
+            execute("INSERT INTO %s (x, y) VALUES(2, {{a:2, b:2}})");
+            execute("INSERT INTO %s (x, y) VALUES(3, {{a:3}})");
+            execute("INSERT INTO %s (x, y) VALUES(4, {{b:4}})");
+
+            assertRows(execute("SELECT * FROM %s"),
+                    row(1, set(userType(1))),
+                    row(2, set(userType(2, 2))),
+                    row(3, set(userType(3, null))),
+                    row(4, set(userType(null, 4))));
+
+            flush();
+
+            assertRows(execute("SELECT * FROM %s"),
+                    row(1, set(userType(1))),
+                    row(2, set(userType(2, 2))),
+                    row(3, set(userType(3, null))),
+                    row(4, set(userType(null, 4))));
+        }
+    }
+
+    @Test
+    public void testAlteringUserTypeNestedWithinList() throws Throwable
+    {
+        // test frozen and non-frozen collections
+        String[] columnTypePrefixes = {"frozen<list<", "list<frozen<"};
+        for (String columnTypePrefix : columnTypePrefixes)
+        {
+            String ut1 = createType("CREATE TYPE %s (a int)");
+            String columnType = columnTypePrefix + KEYSPACE + "." + ut1 + ">>";
+
+            createTable("CREATE TABLE %s (x int PRIMARY KEY, y " + columnType + ")");
+
+            execute("INSERT INTO %s (x, y) VALUES(1, [1] )");
+            assertRows(execute("SELECT * FROM %s"), row(1, list(userType(1))));
+            flush();
+
+            execute("ALTER TYPE " + KEYSPACE + "." + ut1 + " ADD b int");
+            execute("INSERT INTO %s (x, y) VALUES(2, [{a:2, b:2}])");
+            execute("INSERT INTO %s (x, y) VALUES(3, [{a:3}])");
+            execute("INSERT INTO %s (x, y) VALUES(4, [{b:4}])");
+
+            assertRows(execute("SELECT * FROM %s"),
+                    row(1, list(userType(1))),
+                    row(2, list(userType(2, 2))),
+                    row(3, list(userType(3, null))),
+                    row(4, list(userType(null, 4))));
+
+            flush();
+
+            assertRows(execute("SELECT * FROM %s"),
+                    row(1, list(userType(1))),
+                    row(2, list(userType(2, 2))),
+                    row(3, list(userType(3, null))),
+                    row(4, list(userType(null, 4))));
+        }
+    }
+
+    @Test
+    public void testAlteringUserTypeNestedWithinTuple() throws Throwable
+    {
+        String type = createType("CREATE TYPE %s (a int, b int)");
+
+        createTable("CREATE TABLE %s (a int PRIMARY KEY, b frozen<tuple<int, " + KEYSPACE + "." + type + ">>)");
+
+        execute("INSERT INTO %s (a, b) VALUES(1, (1, {a:1, b:1}))");
+        assertRows(execute("SELECT * FROM %s"), row(1, tuple(1, userType(1, 1))));
+        flush();
+
+        execute("ALTER TYPE " + KEYSPACE + "." + type + " ADD c int");
+        execute("INSERT INTO %s (a, b) VALUES(2, (2, {a: 2, b: 2, c: 2}))");
+        execute("INSERT INTO %s (a, b) VALUES(3, (3, {a: 3, b: 3}))");
+        execute("INSERT INTO %s (a, b) VALUES(4, (4, {b:4}))");
+
+        assertRows(execute("SELECT * FROM %s"),
+                   row(1, tuple(1, userType(1, 1))),
+                   row(2, tuple(2, userType(2, 2, 2))),
+                   row(3, tuple(3, userType(3, 3, null))),
+                   row(4, tuple(4, userType(null, 4, null))));
+
+        flush();
+
+        assertRows(execute("SELECT * FROM %s"),
+                   row(1, tuple(1, userType(1, 1))),
+                   row(2, tuple(2, userType(2, 2, 2))),
+                   row(3, tuple(3, userType(3, 3, null))),
+                   row(4, tuple(4, userType(null, 4, null))));
+    }
+
+    @Test
+    public void testAlteringUserTypeNestedWithinNestedTuple() throws Throwable
+    {
+        String type = createType("CREATE TYPE %s (a int, b int)");
+
+        createTable("CREATE TABLE %s (a int PRIMARY KEY, b frozen<tuple<int, tuple<int, " + KEYSPACE + "." + type + ">>>)");
+
+        execute("INSERT INTO %s (a, b) VALUES(1, (1, (1, {a:1, b:1})))");
+        assertRows(execute("SELECT * FROM %s"), row(1, tuple(1, tuple(1, userType(1, 1)))));
+        flush();
+
+        execute("ALTER TYPE " + KEYSPACE + "." + type + " ADD c int");
+        execute("INSERT INTO %s (a, b) VALUES(2, (2, (1, {a: 2, b: 2, c: 2})))");
+        execute("INSERT INTO %s (a, b) VALUES(3, (3, (1, {a: 3, b: 3})))");
+        execute("INSERT INTO %s (a, b) VALUES(4, (4, (1, {b:4})))");
+
+        assertRows(execute("SELECT * FROM %s"),
+                   row(1, tuple(1, tuple(1, userType(1, 1)))),
+                   row(2, tuple(2, tuple(1, userType(2, 2, 2)))),
+                   row(3, tuple(3, tuple(1, userType(3, 3, null)))),
+                   row(4, tuple(4, tuple(1, userType(null, 4, null)))));
+
+        flush();
+
+        assertRows(execute("SELECT * FROM %s"),
+                   row(1, tuple(1, tuple(1, userType(1, 1)))),
+                   row(2, tuple(2, tuple(1, userType(2, 2, 2)))),
+                   row(3, tuple(3, tuple(1, userType(3, 3, null)))),
+                   row(4, tuple(4, tuple(1, userType(null, 4, null)))));
+    }
+
+    @Test
+    public void testAlteringUserTypeNestedWithinUserType() throws Throwable
+    {
+        String type = createType("CREATE TYPE %s (a int, b int)");
+        String otherType = createType("CREATE TYPE %s (x frozen<" + KEYSPACE + "." + type + ">)");
+
+        createTable("CREATE TABLE %s (a int PRIMARY KEY, b frozen<" + KEYSPACE + "." + otherType + ">)");
+
+        execute("INSERT INTO %s (a, b) VALUES(1, {x: {a:1, b:1}})");
+        assertRows(execute("SELECT b.x.a, b.x.b FROM %s"), row(1, 1));
+        flush();
+
+        execute("ALTER TYPE " + KEYSPACE + "." + type + " ADD c int");
+        execute("INSERT INTO %s (a, b) VALUES(2, {x: {a: 2, b: 2, c: 2}})");
+        execute("INSERT INTO %s (a, b) VALUES(3, {x: {a: 3, b: 3}})");
+        execute("INSERT INTO %s (a, b) VALUES(4, {x: {b:4}})");
+
+        assertRows(execute("SELECT b.x.a, b.x.b, b.x.c FROM %s"),
+                   row(1, 1, null),
+                   row(2, 2, 2),
+                   row(3, 3, null),
+                   row(null, 4, null));
+
+        flush();
+
+        assertRows(execute("SELECT b.x.a, b.x.b, b.x.c FROM %s"),
+                   row(1, 1, null),
+                   row(2, 2, 2),
+                   row(3, 3, null),
+                   row(null, 4, null));
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.user_types_test()
+     */
+    @Test
+    public void testUserTypes() throws Throwable
+    {
+        UUID userID_1 = UUID.fromString("550e8400-e29b-41d4-a716-446655440000");
+
+        String addressType = createType("CREATE TYPE %s (street text, city text, zip_code int, phones set<text >)");
+
+        String nameType = createType("CREATE TYPE %s (firstname text, lastname text)");
+
+        createTable("CREATE TABLE %s (id uuid PRIMARY KEY, name frozen < " + nameType + " >, addresses map < text, frozen < " + addressType + " >> )");
+
+        execute("INSERT INTO %s (id, name) VALUES(?, { firstname: 'Paul', lastname: 'smith' } )", userID_1);
+
+        assertRows(execute("SELECT name.firstname FROM %s WHERE id = ?", userID_1), row("Paul"));
+
+        execute("UPDATE %s SET addresses = addresses + { 'home': { street: '...', city:'SF', zip_code:94102, phones:{ } } } WHERE id = ?", userID_1);
+
+        // TODO: deserialize the value here and check it 's right.
+        execute("SELECT addresses FROM %s WHERE id = ? ", userID_1);
+    }
+
+    /**
+     * Test user type test that does a little more nesting,
+     * migrated from cql_tests.py:TestCQL.more_user_types_test()
+     */
+    @Test
+    public void testNestedUserTypes() throws Throwable
+    {
+        String type1 = createType("CREATE TYPE %s ( s set<text>, m map<text, text>, l list<text>)");
+
+        String type2 = createType("CREATE TYPE %s ( s set < frozen < " + type1 + " >>,)");
+
+        createTable("CREATE TABLE %s (id int PRIMARY KEY, val frozen<" + type2 + ">)");
+
+        execute("INSERT INTO %s (id, val) VALUES (0, { s : {{ s : {'foo', 'bar'}, m : { 'foo' : 'bar' }, l : ['foo', 'bar']} }})");
+
+        // TODO: check result once we have an easy way to do it. For now we just check it doesn't crash
+        execute("SELECT * FROM %s");
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.add_field_to_udt_test()
+     */
+    @Test
+    public void testAddFieldToUdt() throws Throwable
+    {
+        String typeName = createType("CREATE TYPE %s (fooint int, fooset set <text>)");
+        createTable("CREATE TABLE %s (key int PRIMARY KEY, data frozen <" + typeName + ">)");
+
+        execute("INSERT INTO %s (key, data) VALUES (1, {fooint: 1, fooset: {'2'}})");
+        execute("ALTER TYPE " + keyspace() + "." + typeName + " ADD foomap map <int,text>");
+        execute("INSERT INTO %s (key, data) VALUES (1, {fooint: 1, fooset: {'2'}, foomap: {3 : 'bar'}})");
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/01115f72/test/unit/org/apache/cassandra/cql3/validation/miscellaneous/CrcCheckChanceTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/miscellaneous/CrcCheckChanceTest.java b/test/unit/org/apache/cassandra/cql3/validation/miscellaneous/CrcCheckChanceTest.java
new file mode 100644
index 0000000..98d7d70
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/validation/miscellaneous/CrcCheckChanceTest.java
@@ -0,0 +1,160 @@
+/*
+ * 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.validation.miscellaneous;
+
+import java.util.List;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+
+import junit.framework.Assert;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.db.compaction.CompactionInterruptedException;
+import org.apache.cassandra.db.compaction.CompactionManager;
+import org.apache.cassandra.utils.FBUtilities;
+
+import org.junit.Test;
+
+
+public class CrcCheckChanceTest extends CQLTester
+{
+    @Test
+    public void testChangingCrcCheckChance() throws Throwable
+    {
+        //Start with crc_check_chance of 99%
+        createTable("CREATE TABLE %s (p text, c text, v text, s text static, PRIMARY KEY (p, c)) WITH compression = {'sstable_compression': 'LZ4Compressor', 'crc_check_chance' : 0.99}");
+
+        execute("CREATE INDEX foo ON %s(v)");
+
+        execute("INSERT INTO %s(p, c, v, s) values (?, ?, ?, ?)", "p1", "k1", "v1", "sv1");
+        execute("INSERT INTO %s(p, c, v) values (?, ?, ?)", "p1", "k2", "v2");
+        execute("INSERT INTO %s(p, s) values (?, ?)", "p2", "sv2");
+
+
+        ColumnFamilyStore cfs = Keyspace.open(CQLTester.KEYSPACE).getColumnFamilyStore(currentTable());
+        ColumnFamilyStore indexCfs = cfs.indexManager.getIndexesBackedByCfs().iterator().next();
+        cfs.forceBlockingFlush();
+
+        Assert.assertEquals(0.99, cfs.metadata.compressionParameters.getCrcCheckChance());
+        Assert.assertEquals(0.99, cfs.getSSTables().iterator().next().getCompressionMetadata().parameters.getCrcCheckChance());
+        Assert.assertEquals(0.99, indexCfs.metadata.compressionParameters.getCrcCheckChance());
+        Assert.assertEquals(0.99, indexCfs.getSSTables().iterator().next().getCompressionMetadata().parameters.getCrcCheckChance());
+
+        //Test for stack overflow
+        cfs.setCrcCheckChance(0.99);
+
+        assertRows(execute("SELECT * FROM %s WHERE p=?", "p1"),
+                row("p1", "k1", "sv1", "v1"),
+                row("p1", "k2", "sv1", "v2")
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE v=?", "v1"),
+                row("p1", "k1", "sv1", "v1")
+        );
+
+
+
+        //Write a few SSTables then Compact
+
+        execute("INSERT INTO %s(p, c, v, s) values (?, ?, ?, ?)", "p1", "k1", "v1", "sv1");
+        execute("INSERT INTO %s(p, c, v) values (?, ?, ?)", "p1", "k2", "v2");
+        execute("INSERT INTO %s(p, s) values (?, ?)", "p2", "sv2");
+
+        cfs.forceBlockingFlush();
+
+
+        execute("INSERT INTO %s(p, c, v, s) values (?, ?, ?, ?)", "p1", "k1", "v1", "sv1");
+        execute("INSERT INTO %s(p, c, v) values (?, ?, ?)", "p1", "k2", "v2");
+        execute("INSERT INTO %s(p, s) values (?, ?)", "p2", "sv2");
+
+        cfs.forceBlockingFlush();
+
+        execute("INSERT INTO %s(p, c, v, s) values (?, ?, ?, ?)", "p1", "k1", "v1", "sv1");
+        execute("INSERT INTO %s(p, c, v) values (?, ?, ?)", "p1", "k2", "v2");
+        execute("INSERT INTO %s(p, s) values (?, ?)", "p2", "sv2");
+
+        cfs.forceBlockingFlush();
+
+        cfs.forceMajorCompaction();
+
+        //Verify when we alter the value the live sstable readers hold the new one
+        alterTable("ALTER TABLE %s WITH compression = {'sstable_compression': 'LZ4Compressor', 'crc_check_chance': 0.01}");
+
+        Assert.assertEquals( 0.01, cfs.metadata.compressionParameters.getCrcCheckChance());
+        Assert.assertEquals( 0.01, cfs.getSSTables().iterator().next().getCompressionMetadata().parameters.getCrcCheckChance());
+        Assert.assertEquals( 0.01, indexCfs.metadata.compressionParameters.getCrcCheckChance());
+        Assert.assertEquals( 0.01, indexCfs.getSSTables().iterator().next().getCompressionMetadata().parameters.getCrcCheckChance());
+
+        assertRows(execute("SELECT * FROM %s WHERE p=?", "p1"),
+                row("p1", "k1", "sv1", "v1"),
+                row("p1", "k2", "sv1", "v2")
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE v=?", "v1"),
+                row("p1", "k1", "sv1", "v1")
+        );
+
+
+        //Verify the call used by JMX still works
+        cfs.setCrcCheckChance(0.03);
+        Assert.assertEquals( 0.03, cfs.metadata.compressionParameters.getCrcCheckChance());
+        Assert.assertEquals( 0.03, cfs.getSSTables().iterator().next().getCompressionMetadata().parameters.getCrcCheckChance());
+        Assert.assertEquals( 0.03, indexCfs.metadata.compressionParameters.getCrcCheckChance());
+        Assert.assertEquals( 0.03, indexCfs.getSSTables().iterator().next().getCompressionMetadata().parameters.getCrcCheckChance());
+
+    }
+
+
+    @Test
+    public void testDropDuringCompaction() throws Throwable
+    {
+        CompactionManager.instance.disableAutoCompaction();
+
+        //Start with crc_check_chance of 99%
+        createTable("CREATE TABLE %s (p text, c text, v text, s text static, PRIMARY KEY (p, c)) WITH compression = {'sstable_compression': 'LZ4Compressor', 'crc_check_chance' : 0.99}");
+
+        ColumnFamilyStore cfs = Keyspace.open(CQLTester.KEYSPACE).getColumnFamilyStore(currentTable());
+
+        //Write a few SSTables then Compact, and drop
+        for (int i = 0; i < 100; i++)
+        {
+            execute("INSERT INTO %s(p, c, v, s) values (?, ?, ?, ?)", "p1", "k1", "v1", "sv1");
+            execute("INSERT INTO %s(p, c, v) values (?, ?, ?)", "p1", "k2", "v2");
+            execute("INSERT INTO %s(p, s) values (?, ?)", "p2", "sv2");
+
+            cfs.forceBlockingFlush();
+        }
+
+        DatabaseDescriptor.setCompactionThroughputMbPerSec(1);
+        List<Future<?>> futures = CompactionManager.instance.submitMaximal(cfs, CompactionManager.getDefaultGcBefore(cfs), false); 
+        execute("DROP TABLE %s");
+
+        try
+        {
+            FBUtilities.waitOnFutures(futures);
+        }
+        catch (Throwable t)
+        {
+            if (!(t.getCause() instanceof ExecutionException) || !(t.getCause().getCause() instanceof CompactionInterruptedException))
+                throw t;
+        }
+    }
+}
+

http://git-wip-us.apache.org/repos/asf/cassandra/blob/01115f72/test/unit/org/apache/cassandra/cql3/validation/miscellaneous/OverflowTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/miscellaneous/OverflowTest.java b/test/unit/org/apache/cassandra/cql3/validation/miscellaneous/OverflowTest.java
new file mode 100644
index 0000000..5b43599
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/validation/miscellaneous/OverflowTest.java
@@ -0,0 +1,331 @@
+/*
+ * 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.validation.miscellaneous;
+
+import java.math.BigInteger;
+
+import org.junit.Test;
+
+import static junit.framework.Assert.assertFalse;
+import static junit.framework.Assert.assertNull;
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.exceptions.SyntaxException;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+/**
+ * Any tests that do not fit in any other category,
+ * migrated from python dtests, CASSANDRA-9160
+ **/
+public class OverflowTest extends CQLTester
+{
+    /**
+     * Test support for nulls
+     * migrated from cql_tests.py:TestCQL.null_support_test()
+     */
+    @Test
+    public void testNullSupport() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int, c int, v1 int, v2 set<text>, PRIMARY KEY (k, c))");
+
+        execute("INSERT INTO %s (k, c, v1, v2) VALUES (0, 0, null, {'1', '2'})");
+        execute("INSERT INTO %s (k, c, v1) VALUES (0, 1, 1)");
+
+        assertRows(execute("SELECT * FROM %s"),
+                   row(0, 0, null, set("1", "2")),
+                   row(0, 1, 1, null));
+
+        execute("INSERT INTO %s (k, c, v1) VALUES (0, 1, null)");
+        execute("INSERT INTO %s (k, c, v2) VALUES (0, 0, null)");
+
+        assertRows(execute("SELECT * FROM %s"),
+                   row(0, 0, null, null),
+                   row(0, 1, null, null));
+
+        assertInvalid("INSERT INTO %s (k, c, v2) VALUES (0, 2, {1, null})");
+        assertInvalid("SELECT * FROM %s WHERE k = null");
+        assertInvalid("INSERT INTO %s (k, c, v2) VALUES (0, 0, { 'foo', 'bar', null })");
+    }
+
+    /**
+     * Test reserved keywords
+     * migrated from cql_tests.py:TestCQL.reserved_keyword_test()
+     */
+    @Test
+    public void testReservedKeywords() throws Throwable
+    {
+        createTable("CREATE TABLE %s (key text PRIMARY KEY, count counter)");
+
+        String tableName = createTableName();
+        assertInvalidThrow(SyntaxException.class, String.format("CREATE TABLE %s.%s (select text PRIMARY KEY, x int)", keyspace(), tableName));
+    }
+
+    /**
+     * Test identifiers
+     * migrated from cql_tests.py:TestCQL.identifier_test()
+     */
+    @Test
+    public void testIdentifiers() throws Throwable
+    {
+        createTable("CREATE TABLE %s (key_23 int PRIMARY KEY, CoLuMn int)");
+
+        execute("INSERT INTO %s (Key_23, Column) VALUES (0, 0)");
+        execute("INSERT INTO %s (KEY_23, COLUMN) VALUES (0, 0)");
+
+        assertInvalid("INSERT INTO %s (key_23, column, column) VALUES (0, 0, 0)");
+        assertInvalid("INSERT INTO %s (key_23, column, COLUMN) VALUES (0, 0, 0)");
+        assertInvalid("INSERT INTO %s (key_23, key_23, column) VALUES (0, 0, 0)");
+        assertInvalid("INSERT INTO %s (key_23, KEY_23, column) VALUES (0, 0, 0)");
+
+        String tableName = createTableName();
+        assertInvalidThrow(SyntaxException.class, String.format("CREATE TABLE %s.%s (select int PRIMARY KEY, column int)", keyspace(), tableName));
+    }
+
+    /**
+     * Test table options
+     * migrated from cql_tests.py:TestCQL.table_options_test()
+     */
+    @Test
+    public void testTableOptions() throws Throwable
+    {
+        createTable("CREATE TABLE %s ( k int PRIMARY KEY, c int ) WITH "
+                    + "comment = 'My comment' "
+                    + "AND read_repair_chance = 0.5 "
+                    + "AND dclocal_read_repair_chance = 0.5 "
+                    + "AND gc_grace_seconds = 4 "
+                    + "AND bloom_filter_fp_chance = 0.01 "
+                    + "AND compaction = { 'class' : 'LeveledCompactionStrategy', 'sstable_size_in_mb' : 10 } "
+                    + "AND compression = { 'sstable_compression' : '' } "
+                    + "AND caching = 'all' ");
+
+        execute("ALTER TABLE %s WITH "
+                + "comment = 'other comment' "
+                + "AND read_repair_chance = 0.3 "
+                + "AND dclocal_read_repair_chance = 0.3 "
+                + "AND gc_grace_seconds = 100 "
+                + "AND bloom_filter_fp_chance = 0.1 "
+                + "AND compaction = { 'class': 'SizeTieredCompactionStrategy', 'min_sstable_size' : 42 } "
+                + "AND compression = { 'sstable_compression' : 'SnappyCompressor' } "
+                + "AND caching = 'rows_only' ");
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.unescaped_string_test()
+     */
+    @Test
+    public void testUnescapedString() throws Throwable
+    {
+        createTable("CREATE TABLE %s ( k text PRIMARY KEY, c text, )");
+
+        //The \ in this query string is not forwarded to cassandra.
+        //The ' is being escaped in python, but only ' is forwarded
+        //over the wire instead of \'.
+        assertInvalidThrow(SyntaxException.class, "INSERT INTO %s (k, c) VALUES ('foo', 'CQL is cassandra\'s best friend')");
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.boolean_test()
+     */
+    @Test
+    public void testBoolean() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k boolean PRIMARY KEY, b boolean)");
+
+        execute("INSERT INTO %s (k, b) VALUES (true, false)");
+        assertRows(execute("SELECT * FROM %s WHERE k = true"),
+                   row(true, false));
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.float_with_exponent_test()
+     */
+    @Test
+    public void testFloatWithExponent() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, d double, f float)");
+
+        execute("INSERT INTO %s (k, d, f) VALUES (0, 3E+10, 3.4E3)");
+        execute("INSERT INTO %s (k, d, f) VALUES (1, 3.E10, -23.44E-3)");
+        execute("INSERT INTO %s (k, d, f) VALUES (2, 3, -2)");
+    }
+
+    /**
+     * Test regression from #5189,
+     * migrated from cql_tests.py:TestCQL.compact_metadata_test()
+     */
+    @Test
+    public void testCompactMetadata() throws Throwable
+    {
+        createTable("CREATE TABLE %s (id int primary key, i int ) WITH COMPACT STORAGE");
+
+        execute("INSERT INTO %s (id, i) VALUES (1, 2)");
+        assertRows(execute("SELECT * FROM %s"),
+                   row(1, 2));
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.conversion_functions_test()
+     */
+    @Test
+    public void testConversionFunctions() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, i varint, b blob)");
+
+        execute("INSERT INTO %s (k, i, b) VALUES (0, blobAsVarint(bigintAsBlob(3)), textAsBlob('foobar'))");
+        assertRows(execute("SELECT i, blobAsText(b) FROM %s WHERE k = 0"),
+                   row(BigInteger.valueOf(3), "foobar"));
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.empty_blob_test()
+     */
+    @Test
+    public void testEmptyBlob() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, b blob)");
+        execute("INSERT INTO %s (k, b) VALUES (0, 0x)");
+
+        assertRows(execute("SELECT * FROM %s"),
+                   row(0, ByteBufferUtil.bytes("")));
+    }
+
+    private Object[][] fill() throws Throwable
+    {
+        for (int i = 0; i < 2; i++)
+            for (int j = 0; j < 2; j++)
+                execute("INSERT INTO %s (k1, k2, v) VALUES (?, ?, ?)", i, j, i + j);
+
+        return getRows(execute("SELECT * FROM %s"));
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.empty_in_test()
+     */
+    @Test
+    public void testEmpty() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k1 int, k2 int, v int, PRIMARY KEY (k1, k2))");
+
+        // Inserts a few rows to make sure we don 't actually query something
+        Object[][] rows = fill();
+
+        // Test empty IN() in SELECT
+        assertEmpty(execute("SELECT v FROM %s WHERE k1 IN ()"));
+        assertEmpty(execute("SELECT v FROM %s WHERE k1 = 0 AND k2 IN ()"));
+
+        // Test empty IN() in DELETE
+        execute("DELETE FROM %s WHERE k1 IN ()");
+        assertArrayEquals(rows, getRows(execute("SELECT * FROM %s")));
+
+        // Test empty IN() in UPDATE
+        execute("UPDATE %s SET v = 3 WHERE k1 IN () AND k2 = 2");
+        assertArrayEquals(rows, getRows(execute("SELECT * FROM %s")));
+
+        // Same test, but for compact
+        createTable("CREATE TABLE %s (k1 int, k2 int, v int, PRIMARY KEY (k1, k2)) WITH COMPACT STORAGE");
+
+        rows = fill();
+
+        assertEmpty(execute("SELECT v FROM %s WHERE k1 IN ()"));
+        assertEmpty(execute("SELECT v FROM %s WHERE k1 = 0 AND k2 IN ()"));
+
+        // Test empty IN() in DELETE
+        execute("DELETE FROM %s WHERE k1 IN ()");
+        assertArrayEquals(rows, getRows(execute("SELECT * FROM %s")));
+
+        // Test empty IN() in UPDATE
+        execute("UPDATE %s SET v = 3 WHERE k1 IN () AND k2 = 2");
+        assertArrayEquals(rows, getRows(execute("SELECT * FROM %s")));
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.function_with_null_test()
+     */
+    @Test
+    public void testFunctionWithNull() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, t timeuuid,)");
+
+        execute("INSERT INTO %s (k) VALUES (0)");
+        Object[][] rows = getRows(execute("SELECT dateOf(t) FROM %s WHERE k=0"));
+        assertNull(rows[0][0]);
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.column_name_validation_test()
+     */
+    @Test
+    public void testColumnNameValidation() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k text, c int, v timeuuid, PRIMARY KEY (k, c))");
+
+        assertInvalid("INSERT INTO %s (k, c) VALUES ('', 0)");
+
+        // Insert a value that don't fit 'int'
+        assertInvalid("INSERT INTO %s (k, c) VALUES (0, 10000000000)");
+
+        // Insert a non-version 1 uuid
+        assertInvalid("INSERT INTO %s (k, c, v) VALUES (0, 0, 550e8400-e29b-41d4-a716-446655440000)");
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.nan_infinity_test()
+     */
+    @Test
+    public void testNanInfinityValues() throws Throwable
+    {
+        createTable("CREATE TABLE %s (f float PRIMARY KEY)");
+
+        execute("INSERT INTO %s (f) VALUES (NaN)");
+        execute("INSERT INTO %s (f) VALUES (-NaN)");
+        execute("INSERT INTO %s (f) VALUES (Infinity)");
+        execute("INSERT INTO %s (f) VALUES (-Infinity)");
+
+        Object[][] selected = getRows(execute("SELECT * FROM %s"));
+
+        // selected should be[[nan],[inf],[-inf]],
+        // but assert element - wise because NaN!=NaN
+        assertEquals(3, selected.length);
+        assertEquals(1, selected[0].length);
+        assertTrue(Float.isNaN((Float) selected[0][0]));
+
+        assertTrue(Float.isInfinite((Float) selected[1][0])); //inf
+        assertTrue(((Float) selected[1][0]) > 0);
+
+        assertTrue(Float.isInfinite((Float) selected[2][0])); //-inf
+        assertTrue(((Float) selected[2][0]) < 0);
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.blobAs_functions_test()
+     */
+    @Test
+    public void testBlobAsFunction() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, v int)");
+
+        //  A blob that is not 4 bytes should be rejected
+        assertInvalid("INSERT INTO %s (k, v) VALUES (0, blobAsInt(0x01))");
+
+        execute("INSERT INTO %s (k, v) VALUES (0, blobAsInt(0x00000001))");
+        assertRows(execute("select v from %s where k=0"), row(1));
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/01115f72/test/unit/org/apache/cassandra/cql3/validation/miscellaneous/PgStringTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/miscellaneous/PgStringTest.java b/test/unit/org/apache/cassandra/cql3/validation/miscellaneous/PgStringTest.java
new file mode 100644
index 0000000..0d03b94
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/validation/miscellaneous/PgStringTest.java
@@ -0,0 +1,77 @@
+/*
+ * 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.validation.miscellaneous;
+
+import org.junit.Test;
+
+import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.exceptions.SyntaxException;
+
+public class PgStringTest extends CQLTester
+{
+    @Test
+    public void testPgSyleFunction() throws Throwable
+    {
+        execute("create or replace function "+KEYSPACE+".pgfun1 ( input double ) called on null input returns text language java\n" +
+                "AS $$return \"foobar\";$$");
+    }
+
+    @Test
+    public void testPgSyleInsert() throws Throwable
+    {
+        createTable("CREATE TABLE %s (key ascii primary key, val text)");
+
+        // some non-terminated pg-strings
+        assertInvalidSyntax("INSERT INTO %s (key, val) VALUES ($ $key_empty$$, $$'' value for empty$$)");
+        assertInvalidSyntax("INSERT INTO %s (key, val) VALUES ($$key_empty$$, $$'' value for empty$ $)");
+        assertInvalidSyntax("INSERT INTO %s (key, val) VALUES ($$key_empty$ $, $$'' value for empty$$)");
+
+        // different pg-style markers for multiple strings
+        execute("INSERT INTO %s (key, val) VALUES ($$prim$ $ $key$$, $$some '' arbitrary value$$)");
+        // same empty pg-style marker for multiple strings
+        execute("INSERT INTO %s (key, val) VALUES ($$key_empty$$, $$'' value for empty$$)");
+        // stange but valid pg-style
+        execute("INSERT INTO %s (key, val) VALUES ($$$foo$_$foo$$, $$$'' value for empty$$)");
+        // these are conventional quoted strings
+        execute("INSERT INTO %s (key, val) VALUES ('$txt$key$$$$txt$', '$txt$'' other value$txt$')");
+
+        assertRows(execute("SELECT key, val FROM %s WHERE key='prim$ $ $key'"),
+                   row("prim$ $ $key", "some '' arbitrary value")
+        );
+        assertRows(execute("SELECT key, val FROM %s WHERE key='key_empty'"),
+                   row("key_empty", "'' value for empty")
+        );
+        assertRows(execute("SELECT key, val FROM %s WHERE key='$foo$_$foo'"),
+                   row("$foo$_$foo", "$'' value for empty")
+        );
+        assertRows(execute("SELECT key, val FROM %s WHERE key='$txt$key$$$$txt$'"),
+                   row("$txt$key$$$$txt$", "$txt$' other value$txt$")
+        );
+
+        // invalid syntax
+        assertInvalidSyntax("INSERT INTO %s (key, val) VALUES ($ascii$prim$$$key$invterm$, $txt$some '' arbitrary value$txt$)");
+    }
+
+    @Test(expected = SyntaxException.class)
+    public void testMarkerPgFail() throws Throwable
+    {
+        // must throw SyntaxException - not StringIndexOutOfBoundsException or similar
+        execute("create function "+KEYSPACE+".pgfun1 ( input double ) called on null input returns bigint language java\n" +
+                "AS $javasrc$return 0L;$javasrc$;");
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/01115f72/test/unit/org/apache/cassandra/cql3/validation/miscellaneous/RoleSyntaxTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/miscellaneous/RoleSyntaxTest.java b/test/unit/org/apache/cassandra/cql3/validation/miscellaneous/RoleSyntaxTest.java
new file mode 100644
index 0000000..9dfa47b
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/validation/miscellaneous/RoleSyntaxTest.java
@@ -0,0 +1,53 @@
+/*
+ * 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.validation.miscellaneous;
+
+import org.junit.Test;
+
+import org.apache.cassandra.cql3.CQLTester;
+
+public class RoleSyntaxTest extends CQLTester
+{
+    @Test
+    public void standardOptionsSyntaxTest() throws Throwable
+    {
+        assertValidSyntax("CREATE ROLE r WITH LOGIN = true AND SUPERUSER = false AND PASSWORD = 'foo'");
+        assertValidSyntax("CREATE ROLE r WITH PASSWORD = 'foo' AND LOGIN = true AND SUPERUSER = false");
+        assertValidSyntax("CREATE ROLE r WITH SUPERUSER = true AND PASSWORD = 'foo' AND LOGIN = false");
+        assertValidSyntax("CREATE ROLE r WITH LOGIN = true AND PASSWORD = 'foo' AND SUPERUSER = false");
+        assertValidSyntax("CREATE ROLE r WITH SUPERUSER = true AND PASSWORD = 'foo' AND LOGIN = false");
+
+        assertValidSyntax("ALTER ROLE r WITH LOGIN = true AND SUPERUSER = false AND PASSWORD = 'foo'");
+        assertValidSyntax("ALTER ROLE r WITH PASSWORD = 'foo' AND LOGIN = true AND SUPERUSER = false");
+        assertValidSyntax("ALTER ROLE r WITH SUPERUSER = true AND PASSWORD = 'foo' AND LOGIN = false");
+        assertValidSyntax("ALTER ROLE r WITH LOGIN = true AND PASSWORD = 'foo' AND SUPERUSER = false");
+        assertValidSyntax("ALTER ROLE r WITH SUPERUSER = true AND PASSWORD = 'foo' AND LOGIN = false");
+    }
+
+    @Test
+    public void customOptionsSyntaxTestl() throws Throwable
+    {
+        assertValidSyntax("CREATE ROLE r WITH OPTIONS = {'a':'b', 'b':1}");
+        assertInvalidSyntax("CREATE ROLE r WITH OPTIONS = 'term'");
+        assertInvalidSyntax("CREATE ROLE r WITH OPTIONS = 99");
+
+        assertValidSyntax("ALTER ROLE r WITH OPTIONS = {'a':'b', 'b':1}");
+        assertInvalidSyntax("ALTER ROLE r WITH OPTIONS = 'term'");
+        assertInvalidSyntax("ALTER ROLE r WITH OPTIONS = 99");
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/01115f72/test/unit/org/apache/cassandra/cql3/validation/miscellaneous/SSTableMetadataTrackingTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/miscellaneous/SSTableMetadataTrackingTest.java b/test/unit/org/apache/cassandra/cql3/validation/miscellaneous/SSTableMetadataTrackingTest.java
new file mode 100644
index 0000000..2a2ca7b
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/validation/miscellaneous/SSTableMetadataTrackingTest.java
@@ -0,0 +1,161 @@
+/*
+ * 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.validation.miscellaneous;
+
+import org.junit.Test;
+
+import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.io.sstable.metadata.StatsMetadata;
+import static org.junit.Assert.assertEquals;
+
+public class SSTableMetadataTrackingTest extends CQLTester
+{
+    @Test
+    public void baseCheck() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int, b int, c text, PRIMARY KEY (a, b))");
+        ColumnFamilyStore cfs = Keyspace.open(keyspace()).getColumnFamilyStore(currentTable());
+        execute("INSERT INTO %s (a,b,c) VALUES (1,1,'1') using timestamp 9999");
+        cfs.forceBlockingFlush();
+        StatsMetadata metadata = cfs.getSSTables().iterator().next().getSSTableMetadata();
+        assertEquals(9999, metadata.minTimestamp);
+        assertEquals(Integer.MAX_VALUE, metadata.maxLocalDeletionTime);
+        cfs.forceMajorCompaction();
+        metadata = cfs.getSSTables().iterator().next().getSSTableMetadata();
+        assertEquals(9999, metadata.minTimestamp);
+        assertEquals(Integer.MAX_VALUE, metadata.maxLocalDeletionTime);
+    }
+
+    @Test
+    public void testMinMaxtimestampRange() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int, b int, c text, PRIMARY KEY (a, b))");
+        ColumnFamilyStore cfs = Keyspace.open(keyspace()).getColumnFamilyStore(currentTable());
+        execute("INSERT INTO %s (a,b,c) VALUES (1,1,'1') using timestamp 10000");
+        execute("DELETE FROM %s USING TIMESTAMP 9999 WHERE a = 1 and b = 1");
+        cfs.forceBlockingFlush();
+        StatsMetadata metadata = cfs.getSSTables().iterator().next().getSSTableMetadata();
+        assertEquals(9999, metadata.minTimestamp);
+        assertEquals(10000, metadata.maxTimestamp);
+        assertEquals(Integer.MAX_VALUE, metadata.maxLocalDeletionTime, 5);
+        cfs.forceMajorCompaction();
+        metadata = cfs.getSSTables().iterator().next().getSSTableMetadata();
+        assertEquals(9999, metadata.minTimestamp);
+        assertEquals(10000, metadata.maxTimestamp);
+        assertEquals(Integer.MAX_VALUE, metadata.maxLocalDeletionTime, 5);
+    }
+
+    @Test
+    public void testMinMaxtimestampRow() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int, b int, c text, PRIMARY KEY (a, b))");
+        ColumnFamilyStore cfs = Keyspace.open(keyspace()).getColumnFamilyStore(currentTable());
+        execute("INSERT INTO %s (a,b,c) VALUES (1,1,'1') using timestamp 10000");
+        execute("DELETE FROM %s USING TIMESTAMP 9999 WHERE a = 1");
+        cfs.forceBlockingFlush();
+        StatsMetadata metadata = cfs.getSSTables().iterator().next().getSSTableMetadata();
+        assertEquals(9999, metadata.minTimestamp);
+        assertEquals(10000, metadata.maxTimestamp);
+        assertEquals(Integer.MAX_VALUE, metadata.maxLocalDeletionTime, 5);
+        cfs.forceMajorCompaction();
+        metadata = cfs.getSSTables().iterator().next().getSSTableMetadata();
+        assertEquals(9999, metadata.minTimestamp);
+        assertEquals(10000, metadata.maxTimestamp);
+        assertEquals(Integer.MAX_VALUE, metadata.maxLocalDeletionTime, 5);
+    }
+
+
+    @Test
+    public void testTrackMetadata_rangeTombstone() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int, b int, c text, PRIMARY KEY (a, b)) WITH gc_grace_seconds = 10000");
+        ColumnFamilyStore cfs = Keyspace.open(keyspace()).getColumnFamilyStore(currentTable());
+        execute("DELETE FROM %s USING TIMESTAMP 9999 WHERE a = 1 and b = 1");
+        cfs.forceBlockingFlush();
+        assertEquals(1, cfs.getSSTables().size());
+        StatsMetadata metadata = cfs.getSSTables().iterator().next().getSSTableMetadata();
+        assertEquals(9999, metadata.minTimestamp);
+        assertEquals(9999, metadata.maxTimestamp);
+        assertEquals(System.currentTimeMillis()/1000, metadata.maxLocalDeletionTime, 5);
+        cfs.forceMajorCompaction();
+        StatsMetadata metadata2 = cfs.getSSTables().iterator().next().getSSTableMetadata();
+        assertEquals(metadata.maxLocalDeletionTime, metadata2.maxLocalDeletionTime);
+        assertEquals(metadata.minTimestamp, metadata2.minTimestamp);
+        assertEquals(metadata.maxTimestamp, metadata2.maxTimestamp);
+    }
+
+    @Test
+    public void testTrackMetadata_rowTombstone() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int, b int, c text, PRIMARY KEY (a, b))");
+        ColumnFamilyStore cfs = Keyspace.open(keyspace()).getColumnFamilyStore(currentTable());
+        execute("DELETE FROM %s USING TIMESTAMP 9999 WHERE a = 1");
+
+        cfs.forceBlockingFlush();
+        assertEquals(1, cfs.getSSTables().size());
+        StatsMetadata metadata = cfs.getSSTables().iterator().next().getSSTableMetadata();
+        assertEquals(9999, metadata.minTimestamp);
+        assertEquals(9999, metadata.maxTimestamp);
+        assertEquals(System.currentTimeMillis()/1000, metadata.maxLocalDeletionTime, 5);
+        cfs.forceMajorCompaction();
+        StatsMetadata metadata2 = cfs.getSSTables().iterator().next().getSSTableMetadata();
+        assertEquals(metadata.maxLocalDeletionTime, metadata2.maxLocalDeletionTime);
+        assertEquals(metadata.minTimestamp, metadata2.minTimestamp);
+        assertEquals(metadata.maxTimestamp, metadata2.maxTimestamp);
+    }
+
+    @Test
+    public void testTrackMetadata_rowMarker() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int, PRIMARY KEY (a))");
+        ColumnFamilyStore cfs = Keyspace.open(keyspace()).getColumnFamilyStore(currentTable());
+        execute("INSERT INTO %s (a) VALUES (1) USING TIMESTAMP 9999");
+
+        cfs.forceBlockingFlush();
+        assertEquals(1, cfs.getSSTables().size());
+        StatsMetadata metadata = cfs.getSSTables().iterator().next().getSSTableMetadata();
+        assertEquals(9999, metadata.minTimestamp);
+        assertEquals(9999, metadata.maxTimestamp);
+        assertEquals(Integer.MAX_VALUE, metadata.maxLocalDeletionTime);
+        cfs.forceMajorCompaction();
+        StatsMetadata metadata2 = cfs.getSSTables().iterator().next().getSSTableMetadata();
+        assertEquals(metadata.maxLocalDeletionTime, metadata2.maxLocalDeletionTime);
+        assertEquals(metadata.minTimestamp, metadata2.minTimestamp);
+        assertEquals(metadata.maxTimestamp, metadata2.maxTimestamp);
+    }
+    @Test
+    public void testTrackMetadata_rowMarkerDelete() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int, PRIMARY KEY (a))");
+        ColumnFamilyStore cfs = Keyspace.open(keyspace()).getColumnFamilyStore(currentTable());
+        execute("DELETE FROM %s USING TIMESTAMP 9999 WHERE a=1");
+        cfs.forceBlockingFlush();
+        assertEquals(1, cfs.getSSTables().size());
+        StatsMetadata metadata = cfs.getSSTables().iterator().next().getSSTableMetadata();
+        assertEquals(9999, metadata.minTimestamp);
+        assertEquals(9999, metadata.maxTimestamp);
+        assertEquals(System.currentTimeMillis()/1000, metadata.maxLocalDeletionTime, 5);
+        cfs.forceMajorCompaction();
+        StatsMetadata metadata2 = cfs.getSSTables().iterator().next().getSSTableMetadata();
+        assertEquals(metadata.maxLocalDeletionTime, metadata2.maxLocalDeletionTime);
+        assertEquals(metadata.minTimestamp, metadata2.minTimestamp);
+        assertEquals(metadata.maxTimestamp, metadata2.maxTimestamp);
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/01115f72/test/unit/org/apache/cassandra/cql3/validation/miscellaneous/TombstonesTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/miscellaneous/TombstonesTest.java b/test/unit/org/apache/cassandra/cql3/validation/miscellaneous/TombstonesTest.java
new file mode 100644
index 0000000..5980372
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/validation/miscellaneous/TombstonesTest.java
@@ -0,0 +1,171 @@
+/*
+ * 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.validation.miscellaneous;
+
+import java.util.concurrent.TimeUnit;
+
+import com.google.common.base.Throwables;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.db.filter.TombstoneOverwhelmingException;
+
+import static junit.framework.Assert.assertTrue;
+import static junit.framework.Assert.fail;
+
+/**
+ * Test that TombstoneOverwhelmingException gets thrown when it should be and doesn't when it shouldn't be.
+ */
+public class TombstonesTest extends CQLTester
+{
+    static final int ORIGINAL_THRESHOLD = DatabaseDescriptor.getTombstoneFailureThreshold();
+    static final int THRESHOLD = 100;
+
+    @BeforeClass
+    public static void setUp() throws Throwable
+    {
+        DatabaseDescriptor.setTombstoneFailureThreshold(THRESHOLD);
+    }
+
+    @AfterClass
+    public static void tearDown()
+    {
+        DatabaseDescriptor.setTombstoneFailureThreshold(ORIGINAL_THRESHOLD);
+    }
+
+    @Test
+    public void testBelowThresholdSelect() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a text, b text, c text, PRIMARY KEY (a, b));");
+
+        // insert exactly the amount of tombstones that shouldn't trigger an exception
+        for (int i = 0; i < THRESHOLD; i++)
+            execute("INSERT INTO %s (a, b, c) VALUES ('key', 'column" + i + "', null);");
+
+        try
+        {
+            execute("SELECT * FROM %s WHERE a = 'key';");
+        }
+        catch (Throwable e)
+        {
+            fail("SELECT with tombstones below the threshold should not have failed, but has: " + e);
+        }
+    }
+
+    @Test
+    public void testBeyondThresholdSelect() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a text, b text, c text, PRIMARY KEY (a, b));");
+
+        // insert exactly the amount of tombstones that *SHOULD* trigger an exception
+        for (int i = 0; i < THRESHOLD + 1; i++)
+            execute("INSERT INTO %s (a, b, c) VALUES ('key', 'column" + i + "', null);");
+
+        try
+        {
+            execute("SELECT * FROM %s WHERE a = 'key';");
+            fail("SELECT with tombstones beyond the threshold should have failed, but hasn't");
+        }
+        catch (Throwable e)
+        {
+            String error = "Expected exception instanceof TombstoneOverwhelmingException instead got "
+                          + System.lineSeparator()
+                          + Throwables.getStackTraceAsString(e);
+            assertTrue(error, e instanceof TombstoneOverwhelmingException);
+        }
+    }
+
+    @Test
+    public void testAllShadowedSelect() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a text, b text, c text, PRIMARY KEY (a, b));");
+
+        // insert exactly the amount of tombstones that *SHOULD* normally trigger an exception
+        for (int i = 0; i < THRESHOLD + 1; i++)
+            execute("INSERT INTO %s (a, b, c) VALUES ('key', 'column" + i + "', null);");
+
+        // delete all with a partition level tombstone
+        execute("DELETE FROM %s WHERE a = 'key'");
+
+        try
+        {
+            execute("SELECT * FROM %s WHERE a = 'key';");
+        }
+        catch (Throwable e)
+        {
+            fail("SELECT with tombstones shadowed by a partition tombstone should not have failed, but has: " + e);
+        }
+    }
+
+    @Test
+    public void testLiveShadowedCellsSelect() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a text, b text, c text, PRIMARY KEY (a, b));");
+
+        for (int i = 0; i < THRESHOLD + 1; i++)
+            execute("INSERT INTO %s (a, b, c) VALUES ('key', 'column" + i + "', 'column');");
+
+        // delete all with a partition level tombstone
+        execute("DELETE FROM %s WHERE a = 'key'");
+
+        try
+        {
+            execute("SELECT * FROM %s WHERE a = 'key';");
+        }
+        catch (Throwable e)
+        {
+            fail("SELECT with regular cells shadowed by a partition tombstone should not have failed, but has: " + e);
+        }
+    }
+
+    @Test
+    public void testExpiredTombstones() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a text, b text, c text, PRIMARY KEY (a, b)) WITH gc_grace_seconds = 1;");
+
+        for (int i = 0; i < THRESHOLD + 1; i++)
+            execute("INSERT INTO %s (a, b, c) VALUES ('key', 'column" + i + "', null);");
+
+        // not yet past gc grace - must throw a TOE
+        try
+        {
+            execute("SELECT * FROM %s WHERE a = 'key';");
+            fail("SELECT with tombstones beyond the threshold should have failed, but hasn't");
+        }
+        catch (Throwable e)
+        {
+            assertTrue(e instanceof TombstoneOverwhelmingException);
+        }
+
+        // sleep past gc grace
+        TimeUnit.SECONDS.sleep(2);
+
+        // past gc grace - must not throw a TOE now
+        try
+        {
+            execute("SELECT * FROM %s WHERE a = 'key';");
+        }
+        catch (Throwable e)
+        {
+            fail("SELECT with expired tombstones beyond the threshold should not have failed, but has: " + e);
+        }
+    }
+}


[09/32] cassandra git commit: Migrate CQL tests from dtest to unit tests

Posted by jm...@apache.org.
Migrate CQL tests from dtest to unit tests

Patch by stefania; reviewed by jmckenzie for CASSANDRA-9160


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

Branch: refs/heads/trunk
Commit: f797bfa4da53315b49f8d97b784047f33ba1bf5f
Parents: 3caf0e0
Author: Stefania Alborghetti <st...@datastax.com>
Authored: Wed Jun 24 12:10:29 2015 -0400
Committer: Josh McKenzie <jo...@datastax.com>
Committed: Wed Jun 24 12:10:29 2015 -0400

----------------------------------------------------------------------
 .../cassandra/config/DatabaseDescriptor.java    |    6 +
 .../org/apache/cassandra/cql3/ResultSet.java    |    9 +
 .../apache/cassandra/cql3/UntypedResultSet.java |    7 +-
 .../cql3/statements/BatchStatement.java         |   69 +-
 .../cql3/statements/CQL3CasRequest.java         |    8 +-
 .../cql3/statements/ModificationStatement.java  |   80 +-
 .../cql3/statements/SelectStatement.java        |   39 +-
 .../cql3/statements/TruncateStatement.java      |   17 +-
 .../apache/cassandra/db/ColumnFamilyStore.java  |    5 +
 .../apache/cassandra/service/StorageProxy.java  |    4 +-
 .../org/apache/cassandra/utils/UUIDGen.java     |   16 +-
 .../org/apache/cassandra/cql3/ManyRowsTest.java |   92 ++
 .../apache/cassandra/cql3/AlterTableTest.java   |  113 --
 .../org/apache/cassandra/cql3/CQLTester.java    |  216 ++-
 .../apache/cassandra/cql3/CollectionsTest.java  |  240 ----
 .../cassandra/cql3/ContainsRelationTest.java    |  269 ----
 .../cassandra/cql3/CrcCheckChanceTest.java      |  159 ---
 .../cql3/CreateAndAlterKeyspaceTest.java        |   37 -
 .../cql3/CreateIndexStatementTest.java          |  101 --
 .../apache/cassandra/cql3/CreateTableTest.java  |   32 -
 .../cql3/CreateTriggerStatementTest.java        |  121 --
 .../cassandra/cql3/FrozenCollectionsTest.java   | 1101 ----------------
 .../cql3/IndexedValuesValidationTest.java       |  149 ---
 .../cassandra/cql3/MultiColumnRelationTest.java |  808 ------------
 .../cassandra/cql3/RangeDeletionTest.java       |   35 -
 .../cql3/SSTableMetadataTrackingTest.java       |  160 ---
 .../cql3/SelectWithTokenFunctionTest.java       |   77 --
 .../cassandra/cql3/SelectionOrderingTest.java   |  233 ----
 .../cql3/SingleColumnRelationTest.java          |  107 --
 .../SliceQueryFilterWithTombstonesTest.java     |  166 ---
 .../cassandra/cql3/StaticColumnsQueryTest.java  |  280 ----
 .../cassandra/cql3/ThriftCompatibilityTest.java |    1 +
 .../apache/cassandra/cql3/TimestampTest.java    |   36 -
 .../apache/cassandra/cql3/TupleTypeTest.java    |  101 --
 .../org/apache/cassandra/cql3/TypeTest.java     |   86 --
 .../apache/cassandra/cql3/UseStatementTest.java |   29 -
 .../apache/cassandra/cql3/UserTypesTest.java    |  319 -----
 .../statements/SelectionColumnMappingTest.java  |    9 +
 .../validation/entities/CollectionsTest.java    |  488 +++++++
 .../cql3/validation/entities/CountersTest.java  |   85 ++
 .../cql3/validation/entities/DateTypeTest.java  |   39 +
 .../entities/FrozenCollectionsTest.java         | 1111 ++++++++++++++++
 .../validation/entities/SecondaryIndexTest.java |  644 +++++++++
 .../validation/entities/StaticColumnsTest.java  |  271 ++++
 .../cql3/validation/entities/TimestampTest.java |  147 +++
 .../cql3/validation/entities/TimeuuidTest.java  |   81 ++
 .../cql3/validation/entities/TupleTypeTest.java |  158 +++
 .../cql3/validation/entities/TypeTest.java      |   88 ++
 .../cql3/validation/entities/UserTypesTest.java |  389 ++++++
 .../miscellaneous/CrcCheckChanceTest.java       |  160 +++
 .../validation/miscellaneous/OverflowTest.java  |  331 +++++
 .../SSTableMetadataTrackingTest.java            |  161 +++
 .../miscellaneous/TombstonesTest.java           |  167 +++
 .../cql3/validation/operations/AlterTest.java   |  201 +++
 .../cql3/validation/operations/BatchTest.java   |   88 ++
 .../cql3/validation/operations/CreateTest.java  |  462 +++++++
 .../cql3/validation/operations/DeleteTest.java  |  329 +++++
 .../operations/InsertUpdateIfCondition.java     |  861 ++++++++++++
 .../validation/operations/SelectLimitTest.java  |  111 ++
 .../SelectMultiColumnRelationTest.java          |  982 ++++++++++++++
 .../operations/SelectOrderByTest.java           |  503 +++++++
 .../SelectOrderedPartitionerTest.java           |  325 +++++
 .../SelectSingleColumnRelationTest.java         |  109 ++
 .../cql3/validation/operations/SelectTest.java  | 1238 ++++++++++++++++++
 .../cql3/validation/operations/UpdateTest.java  |   59 +
 .../cql3/validation/operations/UseTest.java     |   31 +
 .../stress/generate/values/TimeUUIDs.java       |    2 +-
 67 files changed, 10137 insertions(+), 4821 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/f797bfa4/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
index 803591f..a2a9e48 100644
--- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
+++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
@@ -1565,6 +1565,12 @@ public class DatabaseDescriptor
         return conf.row_cache_size_in_mb;
     }
 
+    @VisibleForTesting
+    public static void setRowCacheSizeInMB(long val)
+    {
+        conf.row_cache_size_in_mb = val;
+    }
+
     public static int getRowCacheSavePeriod()
     {
         return conf.row_cache_save_period;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f797bfa4/src/java/org/apache/cassandra/cql3/ResultSet.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/ResultSet.java b/src/java/org/apache/cassandra/cql3/ResultSet.java
index 4ff513b..62857c9 100644
--- a/src/java/org/apache/cassandra/cql3/ResultSet.java
+++ b/src/java/org/apache/cassandra/cql3/ResultSet.java
@@ -269,6 +269,15 @@ public class ResultSet
             return new Metadata(EnumSet.copyOf(flags), names, columnCount, pagingState);
         }
 
+        /**
+         * Return only the column names requested by the user, excluding those added for post-query re-orderings,
+         * see definition of names and columnCount.
+         **/
+        public List<ColumnSpecification> requestNames()
+        {
+            return names.subList(0, columnCount);
+        }
+
         // The maximum number of values that the ResultSet can hold. This can be bigger than columnCount due to CASSANDRA-4911
         public int valueCount()
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f797bfa4/src/java/org/apache/cassandra/cql3/UntypedResultSet.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/UntypedResultSet.java b/src/java/org/apache/cassandra/cql3/UntypedResultSet.java
index fef70fb..81482ef 100644
--- a/src/java/org/apache/cassandra/cql3/UntypedResultSet.java
+++ b/src/java/org/apache/cassandra/cql3/UntypedResultSet.java
@@ -96,7 +96,7 @@ public abstract class UntypedResultSet implements Iterable<UntypedResultSet.Row>
 
         public List<ColumnSpecification> metadata()
         {
-            return cqlRows.metadata.names;
+            return cqlRows.metadata.requestNames();
         }
     }
 
@@ -219,6 +219,11 @@ public abstract class UntypedResultSet implements Iterable<UntypedResultSet.Row>
             return data.get(column) != null;
         }
 
+        public ByteBuffer getBlob(String column)
+        {
+            return data.get(column);
+        }
+
         public String getString(String column)
         {
             return UTF8Type.instance.compose(data.get(column));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f797bfa4/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java b/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
index 6d4d3a1..2005521 100644
--- a/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
@@ -39,6 +39,7 @@ import org.apache.cassandra.service.QueryState;
 import org.apache.cassandra.service.StorageProxy;
 import org.apache.cassandra.transport.messages.ResultMessage;
 import org.apache.cassandra.utils.NoSpamLogger;
+import org.apache.cassandra.utils.Pair;
 
 /**
  * A <code>BATCH</code> statement parsed from a CQL query.
@@ -318,10 +319,32 @@ public class BatchStatement implements CQLStatement
     private ResultMessage executeWithConditions(BatchQueryOptions options, QueryState state)
     throws RequestExecutionException, RequestValidationException
     {
+        Pair<CQL3CasRequest, Set<ColumnDefinition>> p = makeCasRequest(options, state);
+        CQL3CasRequest casRequest = p.left;
+        Set<ColumnDefinition> columnsWithConditions = p.right;
+
+        ColumnFamily result = StorageProxy.cas(casRequest.cfm.ksName,
+                                               casRequest.cfm.cfName,
+                                               casRequest.key,
+                                               casRequest,
+                                               options.getSerialConsistency(),
+                                               options.getConsistency(),
+                                               state.getClientState());
+
+        return new ResultMessage.Rows(ModificationStatement.buildCasResultSet(casRequest.cfm.ksName,
+                                                                              casRequest.key,
+                                                                              casRequest.cfm.cfName,
+                                                                              result,
+                                                                              columnsWithConditions,
+                                                                              true,
+                                                                              options.forStatement(0)));
+    }
+
+    private Pair<CQL3CasRequest,Set<ColumnDefinition>> makeCasRequest(BatchQueryOptions options, QueryState state)
+    throws InvalidRequestException
+    {
         long now = state.getTimestamp();
         ByteBuffer key = null;
-        String ksName = null;
-        String cfName = null;
         CQL3CasRequest casRequest = null;
         Set<ColumnDefinition> columnsWithConditions = new LinkedHashSet<>();
 
@@ -336,8 +359,6 @@ public class BatchStatement implements CQLStatement
             if (key == null)
             {
                 key = pks.get(0);
-                ksName = statement.cfm.ksName;
-                cfName = statement.cfm.cfName;
                 casRequest = new CQL3CasRequest(statement.cfm, key, true);
             }
             else if (!key.equals(pks.get(0)))
@@ -358,23 +379,49 @@ public class BatchStatement implements CQLStatement
             casRequest.addRowUpdate(clusteringPrefix, statement, statementOptions, timestamp);
         }
 
-        ColumnFamily result = StorageProxy.cas(ksName, cfName, key, casRequest, options.getSerialConsistency(), options.getConsistency(), state.getClientState());
-
-        return new ResultMessage.Rows(ModificationStatement.buildCasResultSet(ksName, key, cfName, result, columnsWithConditions, true, options.forStatement(0)));
+        return Pair.create(casRequest, columnsWithConditions);
     }
 
     public ResultMessage executeInternal(QueryState queryState, QueryOptions options) throws RequestValidationException, RequestExecutionException
     {
-        assert !hasConditions;
+        if (hasConditions)
+            return executeInternalWithConditions(BatchQueryOptions.withoutPerStatementVariables(options), queryState);
+
+        executeInternalWithoutCondition(queryState, options);
+        return new ResultMessage.Void();
+    }
+
+    private ResultMessage executeInternalWithoutCondition(QueryState queryState, QueryOptions options) throws RequestValidationException, RequestExecutionException
+    {
         for (IMutation mutation : getMutations(BatchQueryOptions.withoutPerStatementVariables(options), true, queryState.getTimestamp()))
         {
-            // We don't use counters internally.
-            assert mutation instanceof Mutation;
-            ((Mutation) mutation).apply();
+            assert mutation instanceof Mutation || mutation instanceof CounterMutation;
+
+            if (mutation instanceof Mutation)
+                ((Mutation) mutation).apply();
+            else if (mutation instanceof CounterMutation)
+                ((CounterMutation) mutation).apply();
         }
         return null;
     }
 
+    private ResultMessage executeInternalWithConditions(BatchQueryOptions options, QueryState state) throws RequestExecutionException, RequestValidationException
+    {
+        Pair<CQL3CasRequest, Set<ColumnDefinition>> p = makeCasRequest(options, state);
+        CQL3CasRequest request = p.left;
+        Set<ColumnDefinition> columnsWithConditions = p.right;
+
+        ColumnFamily result = ModificationStatement.casInternal(request, state);
+
+        return new ResultMessage.Rows(ModificationStatement.buildCasResultSet(request.cfm.ksName,
+                                                                              request.key,
+                                                                              request.cfm.cfName,
+                                                                              result,
+                                                                              columnsWithConditions,
+                                                                              true,
+                                                                              options.forStatement(0)));
+    }
+
     public interface BatchVariables
     {
         public List<ByteBuffer> getVariablesForStatement(int statementInBatch);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f797bfa4/src/java/org/apache/cassandra/cql3/statements/CQL3CasRequest.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/CQL3CasRequest.java b/src/java/org/apache/cassandra/cql3/statements/CQL3CasRequest.java
index eb29012..3ea7cd8 100644
--- a/src/java/org/apache/cassandra/cql3/statements/CQL3CasRequest.java
+++ b/src/java/org/apache/cassandra/cql3/statements/CQL3CasRequest.java
@@ -37,10 +37,10 @@ import org.apache.cassandra.utils.Pair;
  */
 public class CQL3CasRequest implements CASRequest
 {
-    private final CFMetaData cfm;
-    private final ByteBuffer key;
-    private final long now;
-    private final boolean isBatch;
+    final CFMetaData cfm;
+    final ByteBuffer key;
+    final long now;
+    final boolean isBatch;
 
     // We index RowCondition by the prefix of the row they applied to for 2 reasons:
     //   1) this allows to keep things sorted to build the ColumnSlice array below

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f797bfa4/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java b/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
index 60558b4..6b0901a 100644
--- a/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
@@ -26,6 +26,7 @@ import com.google.common.collect.Iterables;
 import org.apache.cassandra.auth.Permission;
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.cql3.*;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.composites.CBuilder;
@@ -37,9 +38,12 @@ import org.apache.cassandra.exceptions.*;
 import org.apache.cassandra.service.ClientState;
 import org.apache.cassandra.service.QueryState;
 import org.apache.cassandra.service.StorageProxy;
+import org.apache.cassandra.service.paxos.Commit;
 import org.apache.cassandra.thrift.ThriftValidation;
 import org.apache.cassandra.transport.messages.ResultMessage;
+import org.apache.cassandra.triggers.TriggerExecutor;
 import org.apache.cassandra.utils.Pair;
+import org.apache.cassandra.utils.UUIDGen;
 
 /*
  * Abstract parent class of individual modifications, i.e. INSERT, UPDATE and DELETE.
@@ -494,6 +498,21 @@ public abstract class ModificationStatement implements CQLStatement
     public ResultMessage executeWithCondition(QueryState queryState, QueryOptions options)
     throws RequestExecutionException, RequestValidationException
     {
+        CQL3CasRequest request = makeCasRequest(queryState, options);
+
+        ColumnFamily result = StorageProxy.cas(keyspace(),
+                                               columnFamily(),
+                                               request.key,
+                                               request,
+                                               options.getSerialConsistency(),
+                                               options.getConsistency(),
+                                               queryState.getClientState());
+        return new ResultMessage.Rows(buildCasResultSet(request.key, result, options));
+    }
+
+    private CQL3CasRequest makeCasRequest(QueryState queryState, QueryOptions options)
+    throws InvalidRequestException
+    {
         List<ByteBuffer> keys = buildPartitionKeyNames(options);
         // We don't support IN for CAS operation so far
         if (keys.size() > 1)
@@ -506,15 +525,7 @@ public abstract class ModificationStatement implements CQLStatement
         CQL3CasRequest request = new CQL3CasRequest(cfm, key, false);
         addConditions(prefix, request, options);
         request.addRowUpdate(prefix, this, options, now);
-
-        ColumnFamily result = StorageProxy.cas(keyspace(),
-                                               columnFamily(),
-                                               key,
-                                               request,
-                                               options.getSerialConsistency(),
-                                               options.getConsistency(),
-                                               queryState.getClientState());
-        return new ResultMessage.Rows(buildCasResultSet(key, result, options));
+        return request;
     }
 
     public void addConditions(Composite clusteringPrefix, CQL3CasRequest request, QueryOptions options) throws InvalidRequestException
@@ -615,16 +626,57 @@ public abstract class ModificationStatement implements CQLStatement
 
     public ResultMessage executeInternal(QueryState queryState, QueryOptions options) throws RequestValidationException, RequestExecutionException
     {
-        if (hasConditions())
-            throw new UnsupportedOperationException();
+        return hasConditions()
+               ? executeInternalWithCondition(queryState, options)
+               : executeInternalWithoutCondition(queryState, options);
+    }
 
+    public ResultMessage executeInternalWithoutCondition(QueryState queryState, QueryOptions options) throws RequestValidationException, RequestExecutionException
+    {
         for (IMutation mutation : getMutations(options, true, queryState.getTimestamp()))
         {
-            // We don't use counters internally.
-            assert mutation instanceof Mutation;
+            assert mutation instanceof Mutation || mutation instanceof CounterMutation;
+
+            if (mutation instanceof Mutation)
+                ((Mutation) mutation).apply();
+            else if (mutation instanceof CounterMutation)
+                ((CounterMutation) mutation).apply();
+        }
+        return null;
+    }
+
+    public ResultMessage executeInternalWithCondition(QueryState state, QueryOptions options) throws RequestValidationException, RequestExecutionException
+    {
+        CQL3CasRequest request = makeCasRequest(state, options);
+        ColumnFamily result = casInternal(request, state);
+        return new ResultMessage.Rows(buildCasResultSet(request.key, result, options));
+    }
+
+    static ColumnFamily casInternal(CQL3CasRequest request, QueryState state)
+    throws InvalidRequestException
+    {
+        long millis = state.getTimestamp() / 1000;
+        long nanos = ((state.getTimestamp() - (millis * 1000)) + 1) * 10;
+        UUID ballot = UUIDGen.getTimeUUID(millis, nanos);
+        CFMetaData metadata = Schema.instance.getCFMetaData(request.cfm.ksName, request.cfm.cfName);
 
-            ((Mutation) mutation).apply();
+        ReadCommand readCommand = ReadCommand.create(request.cfm.ksName, request.key, request.cfm.cfName, request.now, request.readFilter());
+        Keyspace keyspace = Keyspace.open(request.cfm.ksName);
+
+        Row row = readCommand.getRow(keyspace);
+        ColumnFamily current = row.cf;
+        if (!request.appliesTo(current))
+        {
+            if (current == null)
+                current = ArrayBackedSortedColumns.factory.create(metadata);
+            return current;
         }
+
+        ColumnFamily updates = request.makeUpdates(current);
+        updates = TriggerExecutor.instance.execute(request.key, updates);
+
+        Commit proposal = Commit.newProposal(request.key, ballot, updates);
+        proposal.makeMutation().apply();
         return null;
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f797bfa4/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
index d0566eb..540cb8c 100644
--- a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
@@ -315,13 +315,40 @@ public class SelectStatement implements CQLStatement
         int limit = getLimit(options);
         long now = System.currentTimeMillis();
         Pageable command = getPageableCommand(options, limit, now);
-        List<Row> rows = command == null
-                       ? Collections.<Row>emptyList()
-                       : (command instanceof Pageable.ReadCommands
-                          ? readLocally(keyspace(), ((Pageable.ReadCommands)command).commands)
-                          : ((RangeSliceCommand)command).executeLocally());
 
-        return processResults(rows, options, limit, now);
+        int pageSize = options.getPageSize();
+        if (parameters.isCount && pageSize <= 0)
+            pageSize = DEFAULT_COUNT_PAGE_SIZE;
+
+        if (pageSize <= 0 || command == null || !QueryPagers.mayNeedPaging(command, pageSize))
+        {
+            List<Row> rows = command == null
+                             ? Collections.<Row>emptyList()
+                             : (command instanceof Pageable.ReadCommands
+                                ? readLocally(keyspace(), ((Pageable.ReadCommands)command).commands)
+                                : ((RangeSliceCommand)command).executeLocally());
+
+            return processResults(rows, options, limit, now);
+        }
+        else
+        {
+            QueryPager pager = QueryPagers.localPager(command);
+            if (parameters.isCount)
+                return pageCountQuery(pager, options, pageSize, now, limit);
+
+            // We can't properly do post-query ordering if we page (see #6722)
+            if (needsPostQueryOrdering())
+                throw new InvalidRequestException("Cannot page queries with both ORDER BY and a IN restriction on the partition key; you must either remove the "
+                                                  + "ORDER BY or the IN and sort client side, or disable paging for this query");
+
+            List<Row> page = pager.fetchPage(pageSize);
+            ResultMessage.Rows msg = processResults(page, options, limit, now);
+
+            if (!pager.isExhausted())
+                msg.result.metadata.setHasMorePages(pager.state());
+
+            return msg;
+        }
     }
 
     public ResultSet process(List<Row> rows) throws InvalidRequestException

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f797bfa4/src/java/org/apache/cassandra/cql3/statements/TruncateStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/TruncateStatement.java b/src/java/org/apache/cassandra/cql3/statements/TruncateStatement.java
index ef1c4a4..09275e8 100644
--- a/src/java/org/apache/cassandra/cql3/statements/TruncateStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/TruncateStatement.java
@@ -22,6 +22,8 @@ import java.util.concurrent.TimeoutException;
 
 import org.apache.cassandra.auth.Permission;
 import org.apache.cassandra.cql3.*;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.exceptions.*;
 import org.apache.cassandra.transport.messages.ResultMessage;
 import org.apache.cassandra.service.ClientState;
@@ -56,7 +58,8 @@ public class TruncateStatement extends CFStatement implements CQLStatement
         ThriftValidation.validateColumnFamily(keyspace(), columnFamily());
     }
 
-    public ResultMessage execute(QueryState state, QueryOptions options) throws InvalidRequestException, TruncateException
+    public ResultMessage execute(QueryState state, QueryOptions options)
+    throws InvalidRequestException, TruncateException
     {
         try
         {
@@ -78,7 +81,17 @@ public class TruncateStatement extends CFStatement implements CQLStatement
     }
 
     public ResultMessage executeInternal(QueryState state, QueryOptions options)
+    throws TruncateException
     {
-        throw new UnsupportedOperationException();
+        try
+        {
+            ColumnFamilyStore cfs = Keyspace.open(keyspace()).getColumnFamilyStore(columnFamily());
+            cfs.truncateBlocking();
+        }
+        catch (Exception e)
+        {
+            throw new TruncateException(e);
+        }
+        return null;
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f797bfa4/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
index a01bb49..65bf289 100644
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
@@ -806,6 +806,11 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
         logger.info("Done loading load new SSTables for {}/{}", keyspace.getName(), name);
     }
 
+    public void rebuildSecondaryIndex(String idxName)
+    {
+        rebuildSecondaryIndex(keyspace.getName(), metadata.cfName, idxName);
+    }
+
     public static void rebuildSecondaryIndex(String ksName, String cfName, String... idxNames)
     {
         ColumnFamilyStore cfs = Keyspace.open(ksName).getColumnFamilyStore(cfName);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f797bfa4/src/java/org/apache/cassandra/service/StorageProxy.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/StorageProxy.java b/src/java/org/apache/cassandra/service/StorageProxy.java
index 1536e46..b76c231 100644
--- a/src/java/org/apache/cassandra/service/StorageProxy.java
+++ b/src/java/org/apache/cassandra/service/StorageProxy.java
@@ -229,7 +229,9 @@ public class StorageProxy implements StorageProxyMBean
                 Tracing.trace("Reading existing values for CAS precondition");
                 long timestamp = System.currentTimeMillis();
                 ReadCommand readCommand = ReadCommand.create(keyspaceName, key, cfName, timestamp, request.readFilter());
-                List<Row> rows = read(Arrays.asList(readCommand), consistencyForPaxos == ConsistencyLevel.LOCAL_SERIAL ? ConsistencyLevel.LOCAL_QUORUM : ConsistencyLevel.QUORUM);
+                List<Row> rows = read(Arrays.asList(readCommand), consistencyForPaxos == ConsistencyLevel.LOCAL_SERIAL
+                                                                  ? ConsistencyLevel.LOCAL_QUORUM
+                                                                  : ConsistencyLevel.QUORUM);
                 ColumnFamily current = rows.get(0).cf;
                 if (!request.appliesTo(current))
                 {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f797bfa4/src/java/org/apache/cassandra/utils/UUIDGen.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/UUIDGen.java b/src/java/org/apache/cassandra/utils/UUIDGen.java
index 53293b2..54347ff 100644
--- a/src/java/org/apache/cassandra/utils/UUIDGen.java
+++ b/src/java/org/apache/cassandra/utils/UUIDGen.java
@@ -82,10 +82,15 @@ public class UUIDGen
         return new UUID(createTime(fromUnixTimestamp(when)), clockSeqAndNode);
     }
 
+    public static UUID getTimeUUID(long when, long nanos)
+    {
+        return new UUID(createTime(fromUnixTimestamp(when, nanos)), clockSeqAndNode);
+    }
+
     @VisibleForTesting
-    public static UUID getTimeUUID(long when, long clockSeqAndNode)
+    public static UUID getTimeUUID(long when, long nanos, long clockSeqAndNode)
     {
-        return new UUID(createTime(fromUnixTimestamp(when)), clockSeqAndNode);
+        return new UUID(createTime(fromUnixTimestamp(when, nanos)), clockSeqAndNode);
     }
 
     /** creates a type 1 uuid from raw bytes. */
@@ -169,7 +174,12 @@ public class UUIDGen
      * @return
      */
     private static long fromUnixTimestamp(long timestamp) {
-        return (timestamp - START_EPOCH) * 10000;
+        return fromUnixTimestamp(timestamp, 0L);
+    }
+
+    private static long fromUnixTimestamp(long timestamp, long nanos)
+    {
+        return ((timestamp - START_EPOCH) * 10000) + nanos;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f797bfa4/test/long/org/apache/cassandra/cql3/ManyRowsTest.java
----------------------------------------------------------------------
diff --git a/test/long/org/apache/cassandra/cql3/ManyRowsTest.java b/test/long/org/apache/cassandra/cql3/ManyRowsTest.java
new file mode 100644
index 0000000..82eeabd
--- /dev/null
+++ b/test/long/org/apache/cassandra/cql3/ManyRowsTest.java
@@ -0,0 +1,92 @@
+/*
+ * 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;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Random;
+import java.util.Set;
+
+import org.junit.Test;
+
+public class ManyRowsTest extends CQLTester
+{
+    /**
+     * Migrated from cql_tests.py:TestCQL.large_count_test()
+     */
+    @Test
+    public void testLargeCount() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int, v int, PRIMARY KEY (k))");
+
+        // We know we page at 10K, so test counting just before, at 10K, just after and
+        // a bit after that.
+        for (int k = 1; k < 10000; k++)
+            execute("INSERT INTO %s (k) VALUES (?)", k);
+
+        assertRows(execute("SELECT COUNT(*) FROM %s"), row(9999L));
+
+        execute("INSERT INTO %s (k) VALUES (?)", 10000);
+
+        assertRows(execute("SELECT COUNT(*) FROM %s"), row(10000L));
+
+        execute("INSERT INTO %s (k) VALUES (?)", 10001);
+
+        assertRows(execute("SELECT COUNT(*) FROM %s"), row(10001L));
+
+        for (int k = 10002; k < 15001; k++)
+            execute("INSERT INTO %s (k) VALUES (?)", k);
+
+        assertRows(execute("SELECT COUNT(*) FROM %s"), row(15000L));
+    }
+
+    /**
+     * Test for CASSANDRA-8410,
+     * migrated from cql_tests.py:TestCQL.large_clustering_in_test()
+     */
+    @Test
+    public void testLargeClustering() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int, c int, v int, PRIMARY KEY (k, c) )");
+
+        execute("INSERT INTO %s (k, c, v) VALUES (?, ?, ?)", 0, 0, 0);
+
+        // try to fetch one existing row and 9999 non-existing rows
+        List<Integer> inValues = new ArrayList(10000);
+        for (int i = 0; i < 10000; i++)
+            inValues.add(i);
+
+        assertRows(execute("SELECT * FROM %s WHERE k=? AND c IN ?", 0, inValues),
+                   row(0, 0, 0));
+
+        // insert approximately 1000 random rows between 0 and 10k
+        Random rnd = new Random();
+        Set<Integer> clusteringValues = new HashSet<>();
+        for (int i = 0; i < 1000; i++)
+            clusteringValues.add(rnd.nextInt(10000));
+
+        clusteringValues.add(0);
+
+        for (int i : clusteringValues) // TODO - this was done in parallel by dtests
+            execute("INSERT INTO %s (k, c, v) VALUES (?, ?, ?)", 0, i, i);
+
+        assertRowCount(execute("SELECT * FROM %s WHERE k=? AND c IN ?", 0, inValues), clusteringValues.size());
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f797bfa4/test/unit/org/apache/cassandra/cql3/AlterTableTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/AlterTableTest.java b/test/unit/org/apache/cassandra/cql3/AlterTableTest.java
deleted file mode 100644
index 4540f3c..0000000
--- a/test/unit/org/apache/cassandra/cql3/AlterTableTest.java
+++ /dev/null
@@ -1,113 +0,0 @@
-/*
- * 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;
-
-import org.apache.cassandra.db.ColumnFamilyStore;
-import org.apache.cassandra.db.Keyspace;
-import org.junit.Test;
-
-import static org.junit.Assert.assertEquals;
-
-public class AlterTableTest extends CQLTester
-{
-    @Test
-    public void testAddList() throws Throwable
-    {
-        createTable("CREATE TABLE %s (id text PRIMARY KEY, content text);");
-        execute("ALTER TABLE %s ADD myCollection list<text>;");
-        execute("INSERT INTO %s (id, content , myCollection) VALUES ('test', 'first test', ['first element']);");
-
-        assertRows(execute("SELECT * FROM %s;"), row("test", "first test", list("first element")));
-    }
-
-    @Test
-    public void testDropList() throws Throwable
-    {
-        createTable("CREATE TABLE %s (id text PRIMARY KEY, content text, myCollection list<text>);");
-        execute("INSERT INTO %s (id, content , myCollection) VALUES ('test', 'first test', ['first element']);");
-        execute("ALTER TABLE %s DROP myCollection;");
-
-        assertRows(execute("SELECT * FROM %s;"), row("test", "first test"));
-    }
-    @Test
-    public void testAddMap() throws Throwable
-    {
-        createTable("CREATE TABLE %s (id text PRIMARY KEY, content text);");
-        execute("ALTER TABLE %s ADD myCollection map<text, text>;");
-        execute("INSERT INTO %s (id, content , myCollection) VALUES ('test', 'first test', { '1' : 'first element'});");
-
-        assertRows(execute("SELECT * FROM %s;"), row("test", "first test", map("1", "first element")));
-    }
-
-    @Test
-    public void testDropMap() throws Throwable
-    {
-        createTable("CREATE TABLE %s (id text PRIMARY KEY, content text, myCollection map<text, text>);");
-        execute("INSERT INTO %s (id, content , myCollection) VALUES ('test', 'first test', { '1' : 'first element'});");
-        execute("ALTER TABLE %s DROP myCollection;");
-
-        assertRows(execute("SELECT * FROM %s;"), row("test", "first test"));
-    }
-
-    @Test
-    public void testDropListAndAddListWithSameName() throws Throwable
-    {
-        createTable("CREATE TABLE %s (id text PRIMARY KEY, content text, myCollection list<text>);");
-        execute("INSERT INTO %s (id, content , myCollection) VALUES ('test', 'first test', ['first element']);");
-        execute("ALTER TABLE %s DROP myCollection;");
-        execute("ALTER TABLE %s ADD myCollection list<text>;");
-
-        assertRows(execute("SELECT * FROM %s;"), row("test", "first test", null));
-        execute("UPDATE %s set myCollection = ['second element'] WHERE id = 'test';");
-        assertRows(execute("SELECT * FROM %s;"), row("test", "first test", list("second element")));
-    }
-    @Test
-    public void testDropListAndAddMapWithSameName() throws Throwable
-    {
-        createTable("CREATE TABLE %s (id text PRIMARY KEY, content text, myCollection list<text>);");
-        execute("INSERT INTO %s (id, content , myCollection) VALUES ('test', 'first test', ['first element']);");
-        execute("ALTER TABLE %s DROP myCollection;");
-
-        assertInvalid("ALTER TABLE %s ADD myCollection map<int, int>;");
-    }
-
-    @Test
-    public void testChangeStrategyWithUnquotedAgrument() throws Throwable
-    {
-        createTable("CREATE TABLE %s (id text PRIMARY KEY);");
-
-        assertInvalidSyntaxMessage("no viable alternative at input '}'",
-                                   "ALTER TABLE %s WITH caching = {'keys' : 'all', 'rows_per_partition' : ALL};");
-    }
-
-    @Test
-    // tests CASSANDRA-7976
-    public void testAlterIndexInterval() throws Throwable
-    {
-        String tableName = createTable("CREATE TABLE IF NOT EXISTS %s (id uuid, album text, artist text, data blob, PRIMARY KEY (id))");
-        ColumnFamilyStore cfs = Keyspace.open(KEYSPACE).getColumnFamilyStore(tableName);
-
-        alterTable("ALTER TABLE %s WITH min_index_interval=256 AND max_index_interval=512");
-        assertEquals(256, cfs.metadata.getMinIndexInterval());
-        assertEquals(512, cfs.metadata.getMaxIndexInterval());
-
-        alterTable("ALTER TABLE %s WITH caching = 'none'");
-        assertEquals(256, cfs.metadata.getMinIndexInterval());
-        assertEquals(512, cfs.metadata.getMaxIndexInterval());
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f797bfa4/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 8b24e69..8e620af 100644
--- a/test/unit/org/apache/cassandra/cql3/CQLTester.java
+++ b/test/unit/org/apache/cassandra/cql3/CQLTester.java
@@ -18,6 +18,9 @@
 package org.apache.cassandra.cql3;
 
 import java.io.File;
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.net.InetAddress;
 import java.nio.ByteBuffer;
 import java.util.*;
 import java.util.concurrent.CountDownLatch;
@@ -30,17 +33,21 @@ import com.google.common.collect.ImmutableSet;
 import org.junit.AfterClass;
 import org.junit.After;
 import org.junit.Assert;
+import org.junit.Before;
 import org.junit.BeforeClass;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import static junit.framework.Assert.assertNotNull;
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.concurrent.ScheduledExecutors;
 import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.db.Directories;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.marshal.*;
+import org.apache.cassandra.dht.Murmur3Partitioner;
 import org.apache.cassandra.exceptions.*;
 import org.apache.cassandra.io.util.FileUtils;
 import org.apache.cassandra.serializers.TypeSerializer;
@@ -54,6 +61,7 @@ public abstract class CQLTester
 
     public static final String KEYSPACE = "cql_test_keyspace";
     private static final boolean USE_PREPARED_VALUES = Boolean.valueOf(System.getProperty("cassandra.test.use_prepared", "true"));
+    protected static final long ROW_CACHE_SIZE_IN_MB = Integer.valueOf(System.getProperty("cassandra.test.row_cache_size_in_mb", "0"));
     private static final AtomicInteger seqNumber = new AtomicInteger();
 
     static
@@ -66,9 +74,12 @@ public abstract class CQLTester
     private List<String> types = new ArrayList<>();
 
     @BeforeClass
-    public static void setUpClass() throws Throwable
+    public static void setUpClass()
     {
-        schemaChange(String.format("CREATE KEYSPACE IF NOT EXISTS %s WITH replication = {'class': 'SimpleStrategy', 'replication_factor': '1'}", KEYSPACE));
+        if (ROW_CACHE_SIZE_IN_MB > 0)
+            DatabaseDescriptor.setRowCacheSizeInMB(ROW_CACHE_SIZE_IN_MB);
+
+        DatabaseDescriptor.setPartitioner(new Murmur3Partitioner());
     }
 
     @AfterClass
@@ -76,6 +87,12 @@ public abstract class CQLTester
     {
     }
 
+    @Before
+    public void beforeTest() throws Throwable
+    {
+        schemaChange(String.format("CREATE KEYSPACE IF NOT EXISTS %s WITH replication = {'class': 'SimpleStrategy', 'replication_factor': '1'}", KEYSPACE));
+    }
+
     @After
     public void afterTest() throws Throwable
     {
@@ -91,10 +108,10 @@ public abstract class CQLTester
             {
                 try
                 {
-                    for (int i = tablesToDrop.size() - 1; i >=0; i--)
+                    for (int i = tablesToDrop.size() - 1; i >= 0; i--)
                         schemaChange(String.format("DROP TABLE IF EXISTS %s.%s", KEYSPACE, tablesToDrop.get(i)));
 
-                    for (int i = typesToDrop.size() - 1; i >=0; i--)
+                    for (int i = typesToDrop.size() - 1; i >= 0; i--)
                         schemaChange(String.format("DROP TYPE IF EXISTS %s.%s", KEYSPACE, typesToDrop.get(i)));
 
                     // Dropping doesn't delete the sstables. It's not a huge deal but it's cleaner to cleanup after us
@@ -105,10 +122,10 @@ public abstract class CQLTester
                     final CountDownLatch latch = new CountDownLatch(1);
                     ScheduledExecutors.nonPeriodicTasks.execute(new Runnable()
                     {
-                            public void run()
-                            {
-                                latch.countDown();
-                            }
+                        public void run()
+                        {
+                            latch.countDown();
+                        }
                     });
                     latch.await(2, TimeUnit.SECONDS);
 
@@ -149,9 +166,25 @@ public abstract class CQLTester
         }
     }
 
-    public boolean usePrepared()
+    public void compact()
+    {
+        try
+        {
+            String currentTable = currentTable();
+            if (currentTable != null)
+                Keyspace.open(KEYSPACE).getColumnFamilyStore(currentTable).forceMajorCompaction();
+        }
+        catch (InterruptedException | ExecutionException e)
+        {
+            throw new RuntimeException(e);
+        }
+    }
+
+    public void cleanupCache()
     {
-        return USE_PREPARED_VALUES;
+        String currentTable = currentTable();
+        if (currentTable != null)
+            Keyspace.open(KEYSPACE).getColumnFamilyStore(currentTable).cleanupCache();
     }
 
     private static void removeAllSSTables(String ks, List<String> tables)
@@ -196,14 +229,20 @@ public abstract class CQLTester
 
     protected String createTable(String query)
     {
-        String currentTable = "table_" + seqNumber.getAndIncrement();
-        tables.add(currentTable);
+        String currentTable = createTableName();
         String fullQuery = String.format(query, KEYSPACE + "." + currentTable);
         logger.info(fullQuery);
         schemaChange(fullQuery);
         return currentTable;
     }
 
+    protected String createTableName()
+    {
+        String currentTable = "table_" + seqNumber.getAndIncrement();
+        tables.add(currentTable);
+        return currentTable;
+    }
+
     protected void createTableMayThrow(String query) throws Throwable
     {
         String currentTable = "table_" + seqNumber.getAndIncrement();
@@ -255,6 +294,37 @@ public abstract class CQLTester
         schemaChange(fullQuery);
     }
 
+    /**
+     * Index creation is asynchronous, this method searches in the system table IndexInfo
+     * for the specified index and returns true if it finds it, which indicates the
+     * index was built. If we haven't found it after 5 seconds we give-up.
+     */
+    protected boolean waitForIndex(String keyspace, String table, String index) throws Throwable
+    {
+        long start = System.currentTimeMillis();
+        boolean indexCreated = false;
+        String indedName = String.format("%s.%s", table, index);
+        while (!indexCreated)
+        {
+            Object[][] results = getRows(execute("select index_name from system.\"IndexInfo\" where table_name = ?", keyspace));
+            for(int i = 0; i < results.length; i++)
+            {
+                if (indedName.equals(results[i][0]))
+                {
+                    indexCreated = true;
+                    break;
+                }
+            }
+
+            if (System.currentTimeMillis() - start > 5000)
+                break;
+
+            Thread.sleep(10);
+        }
+
+        return indexCreated;
+    }
+
     protected void createIndexMayThrow(String query) throws Throwable
     {
         String fullQuery = String.format(query, KEYSPACE + "." + currentTable());
@@ -276,7 +346,7 @@ public abstract class CQLTester
         schemaChange(fullQuery);
     }
 
-    private static void schemaChange(String query)
+    protected static void schemaChange(String query)
     {
         try
         {
@@ -341,7 +411,7 @@ public abstract class CQLTester
             Object[] expected = rows[i];
             UntypedResultSet.Row actual = iter.next();
 
-            Assert.assertEquals(String.format("Invalid number of (expected) values provided for row %d", i), meta.size(), expected.length);
+            Assert.assertEquals(String.format("Invalid number of (expected) values provided for row %d", i), expected.length, meta.size());
 
             for (int j = 0; j < meta.size(); j++)
             {
@@ -351,8 +421,17 @@ public abstract class CQLTester
                 ByteBuffer actualValue = actual.getBytes(column.name.toString());
 
                 if (!Objects.equal(expectedByteValue, actualValue))
-                    Assert.fail(String.format("Invalid value for row %d column %d (%s of type %s), expected <%s> but got <%s>",
-                                              i, j, column.name, column.type.asCQL3Type(), formatValue(expectedByteValue, column.type), formatValue(actualValue, column.type)));
+                {
+                    Object actualValueDecoded = column.type.getSerializer().deserialize(actualValue);
+                    if (!actualValueDecoded.equals(expected[j]))
+                        Assert.fail(String.format("Invalid value for row %d column %d (%s of type %s), expected <%s> but got <%s>",
+                                                  i,
+                                                  j,
+                                                  column.name,
+                                                  column.type.asCQL3Type(),
+                                                  formatValue(expectedByteValue, column.type),
+                                                  formatValue(actualValue, column.type)));
+                }
             }
             i++;
         }
@@ -370,6 +449,65 @@ public abstract class CQLTester
         Assert.assertTrue(String.format("Got more rows than expected. Expected %d but got %d", rows.length, i), i == rows.length);
     }
 
+    protected void assertRowCount(UntypedResultSet result, int numExpectedRows)
+    {
+        if (result == null)
+        {
+            if (numExpectedRows > 0)
+                Assert.fail(String.format("No rows returned by query but %d expected", numExpectedRows));
+            return;
+        }
+
+        List<ColumnSpecification> meta = result.metadata();
+        Iterator<UntypedResultSet.Row> iter = result.iterator();
+        int i = 0;
+        while (iter.hasNext() && i < numExpectedRows)
+        {
+            UntypedResultSet.Row actual = iter.next();
+            assertNotNull(actual);
+            i++;
+        }
+
+        if (iter.hasNext())
+        {
+            while (iter.hasNext())
+            {
+                iter.next();
+                i++;
+            }
+            Assert.fail(String.format("Got less rows than expected. Expected %d but got %d.", numExpectedRows, i));
+        }
+
+        Assert.assertTrue(String.format("Got %s rows than expected. Expected %d but got %d", numExpectedRows>i ? "less" : "more", numExpectedRows, i), i == numExpectedRows);
+    }
+
+    protected Object[][] getRows(UntypedResultSet result)
+    {
+        if (result == null)
+            return new Object[0][];
+
+        List<Object[]> ret = new ArrayList<>();
+        List<ColumnSpecification> meta = result.metadata();
+
+        Iterator<UntypedResultSet.Row> iter = result.iterator();
+        while (iter.hasNext())
+        {
+            UntypedResultSet.Row rowVal = iter.next();
+            Object[] row = new Object[meta.size()];
+            for (int j = 0; j < meta.size(); j++)
+            {
+                ColumnSpecification column = meta.get(j);
+                ByteBuffer val = rowVal.getBytes(column.name.toString());
+                row[j] = val == null ? null : column.type.getSerializer().deserialize(val);
+            }
+
+            ret.add(row);
+        }
+
+        Object[][] a = new Object[ret.size()][];
+        return ret.toArray(a);
+    }
+
     protected void assertAllRows(Object[]... rows) throws Throwable
     {
         assertRows(execute("SELECT * FROM %s"), rows);
@@ -386,6 +524,8 @@ public abstract class CQLTester
             throw new InvalidRequestException(String.format("Expected empty result but got %d rows", result.size()));
     }
 
+
+
     protected void assertInvalid(String query, Object... values) throws Throwable
     {
         assertInvalidMessage(null, query, values);
@@ -393,16 +533,32 @@ public abstract class CQLTester
 
     protected void assertInvalidMessage(String errorMessage, String query, Object... values) throws Throwable
     {
+        assertInvalidThrowMessage(errorMessage, null, query, values);
+    }
+
+    protected void assertInvalidThrow(Class<? extends Throwable> exception, String query, Object... values) throws Throwable
+    {
+        assertInvalidThrowMessage(null, exception, query, values);
+    }
+
+    protected void assertInvalidThrowMessage(String errorMessage, Class<? extends Throwable> exception, String query, Object... values) throws Throwable
+    {
         try
         {
             execute(query, values);
             String q = USE_PREPARED_VALUES
-                     ? query + " (values: " + formatAllValues(values) + ")"
-                     : replaceValues(query, values);
+                       ? query + " (values: " + formatAllValues(values) + ")"
+                       : replaceValues(query, values);
             Assert.fail("Query should be invalid but no error was thrown. Query is: " + q);
         }
-        catch (InvalidRequestException e)
+        catch (CassandraException e)
         {
+            if (exception != null && !exception.isAssignableFrom(e.getClass()))
+            {
+                Assert.fail("Query should be invalid but wrong error was thrown. " +
+                            "Expected: " + exception.getName() + ", got: " + e.getClass().getName() + ". " +
+                            "Query is: " + queryInfo(query, values));
+            }
             if (errorMessage != null)
             {
                 assertMessageContains(errorMessage, e);
@@ -410,6 +566,13 @@ public abstract class CQLTester
         }
     }
 
+    private static String queryInfo(String query, Object[] values)
+    {
+        return USE_PREPARED_VALUES
+               ? query + " (values: " + formatAllValues(values) + ")"
+               : replaceValues(query, values);
+    }
+
     protected void assertInvalidSyntax(String query, Object... values) throws Throwable
     {
         assertInvalidSyntaxMessage(null, query, values);
@@ -721,12 +884,27 @@ public abstract class CQLTester
         if (value instanceof Double)
             return DoubleType.instance;
 
+        if (value instanceof BigInteger)
+            return IntegerType.instance;
+
+        if (value instanceof BigDecimal)
+            return DecimalType.instance;
+
         if (value instanceof String)
             return UTF8Type.instance;
 
         if (value instanceof Boolean)
             return BooleanType.instance;
 
+        if (value instanceof InetAddress)
+            return InetAddressType.instance;
+
+        if (value instanceof Date)
+            return TimestampType.instance;
+
+        if (value instanceof UUID)
+            return UUIDType.instance;
+
         if (value instanceof List)
         {
             List l = (List)value;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f797bfa4/test/unit/org/apache/cassandra/cql3/CollectionsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/CollectionsTest.java b/test/unit/org/apache/cassandra/cql3/CollectionsTest.java
deleted file mode 100644
index 53895d0..0000000
--- a/test/unit/org/apache/cassandra/cql3/CollectionsTest.java
+++ /dev/null
@@ -1,240 +0,0 @@
-/*
- * 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;
-
-import org.junit.Test;
-
-public class CollectionsTest extends CQLTester
-{
-    @Test
-    public void testMapBulkRemoval() throws Throwable
-    {
-        createTable("CREATE TABLE %s (k int PRIMARY KEY, m map<text, text>)");
-
-        execute("INSERT INTO %s(k, m) VALUES (?, ?)", 0, map("k1", "v1", "k2", "v2", "k3", "v3"));
-
-        assertRows(execute("SELECT * FROM %s"),
-            row(0, map("k1", "v1", "k2", "v2", "k3", "v3"))
-        );
-
-        execute("UPDATE %s SET m = m - ? WHERE k = ?", set("k2"), 0);
-
-        assertRows(execute("SELECT * FROM %s"),
-            row(0, map("k1", "v1", "k3", "v3"))
-        );
-
-        execute("UPDATE %s SET m = m + ?, m = m - ? WHERE k = ?", map("k4", "v4"), set("k3"), 0);
-
-        assertRows(execute("SELECT * FROM %s"),
-            row(0, map("k1", "v1", "k4", "v4"))
-        );
-    }
-
-    @Test
-    public void testInvalidCollectionsMix() throws Throwable
-    {
-        createTable("CREATE TABLE %s (k int PRIMARY KEY, l list<text>, s set<text>, m map<text, text>)");
-
-        // Note: we force the non-prepared form for some of those tests because a list and a set
-        // have the same serialized format in practice and CQLTester don't validate that the type
-        // of what's passed as a value in the prepared case, so the queries would work (which is ok,
-        // CQLTester is just a "dumb" client).
-
-        assertInvalid("UPDATE %s SET l = l + { 'a', 'b' } WHERE k = 0");
-        assertInvalid("UPDATE %s SET l = l - { 'a', 'b' } WHERE k = 0");
-        assertInvalid("UPDATE %s SET l = l + ? WHERE k = 0", map("a", "b", "c", "d"));
-        assertInvalid("UPDATE %s SET l = l - ? WHERE k = 0", map("a", "b", "c", "d"));
-
-        assertInvalid("UPDATE %s SET s = s + [ 'a', 'b' ] WHERE k = 0");
-        assertInvalid("UPDATE %s SET s = s - [ 'a', 'b' ] WHERE k = 0");
-        assertInvalid("UPDATE %s SET s = s + ? WHERE k = 0", map("a", "b", "c", "d"));
-        assertInvalid("UPDATE %s SET s = s - ? WHERE k = 0", map("a", "b", "c", "d"));
-
-        assertInvalid("UPDATE %s SET m = m + ? WHERE k = 0", list("a", "b"));
-        assertInvalid("UPDATE %s SET m = m - [ 'a', 'b' ] WHERE k = 0");
-        assertInvalid("UPDATE %s SET m = m + ? WHERE k = 0", set("a", "b"));
-        assertInvalid("UPDATE %s SET m = m - ? WHERE k = 0", map("a", "b", "c", "d"));
-    }
-
-    @Test
-    public void testSets() throws Throwable
-    {
-        createTable("CREATE TABLE %s (k int PRIMARY KEY, s set<text>)");
-
-        execute("INSERT INTO %s(k, s) VALUES (0, ?)", set("v1", "v2", "v3", "v4"));
-
-        assertRows(execute("SELECT s FROM %s WHERE k = 0"),
-            row(set("v1", "v2", "v3", "v4"))
-        );
-
-        execute("DELETE s[?] FROM %s WHERE k = 0", "v1");
-
-        assertRows(execute("SELECT s FROM %s WHERE k = 0"),
-            row(set("v2", "v3", "v4"))
-        );
-
-        // Full overwrite
-        execute("UPDATE %s SET s = ? WHERE k = 0", set("v6", "v5"));
-
-        assertRows(execute("SELECT s FROM %s WHERE k = 0"),
-            row(set("v5", "v6"))
-        );
-
-        execute("UPDATE %s SET s = s + ? WHERE k = 0", set("v7"));
-
-        assertRows(execute("SELECT s FROM %s WHERE k = 0"),
-            row(set("v5", "v6", "v7"))
-        );
-
-        execute("UPDATE %s SET s = s - ? WHERE k = 0", set("v6", "v5"));
-
-        assertRows(execute("SELECT s FROM %s WHERE k = 0"),
-            row(set("v7"))
-        );
-
-        execute("DELETE s[?] FROM %s WHERE k = 0", set("v7"));
-
-        // Deleting an element that does not exist will succeed
-        execute("DELETE s[?] FROM %s WHERE k = 0", set("v7"));
-
-        execute("DELETE s FROM %s WHERE k = 0");
-
-        assertRows(execute("SELECT s FROM %s WHERE k = 0"),
-            row((Object)null)
-        );
-    }
-
-    @Test
-    public void testMaps() throws Throwable
-    {
-        createTable("CREATE TABLE %s (k int PRIMARY KEY, m map<text, int>)");
-
-        execute("INSERT INTO %s(k, m) VALUES (0, ?)", map("v1", 1, "v2", 2));
-
-        assertRows(execute("SELECT m FROM %s WHERE k = 0"),
-            row(map("v1", 1, "v2", 2))
-        );
-
-        execute("UPDATE %s SET m[?] = ?, m[?] = ? WHERE k = 0", "v3", 3, "v4", 4);
-
-        assertRows(execute("SELECT m FROM %s WHERE k = 0"),
-            row(map("v1", 1, "v2", 2, "v3", 3, "v4", 4))
-        );
-
-        execute("DELETE m[?] FROM %s WHERE k = 0", "v1");
-
-        assertRows(execute("SELECT m FROM %s WHERE k = 0"),
-            row(map("v2", 2, "v3", 3, "v4", 4))
-        );
-
-        // Full overwrite
-        execute("UPDATE %s SET m = ? WHERE k = 0", map("v6", 6, "v5", 5));
-
-        assertRows(execute("SELECT m FROM %s WHERE k = 0"),
-            row(map("v5", 5, "v6", 6))
-        );
-
-        execute("UPDATE %s SET m = m + ? WHERE k = 0", map("v7", 7));
-
-        assertRows(execute("SELECT m FROM %s WHERE k = 0"),
-            row(map("v5", 5, "v6", 6, "v7", 7))
-        );
-
-        execute("DELETE m[?] FROM %s WHERE k = 0", "v7");
-
-        assertRows(execute("SELECT m FROM %s WHERE k = 0"),
-            row(map("v5", 5, "v6", 6))
-        );
-
-        execute("DELETE m[?] FROM %s WHERE k = 0", "v6");
-
-        assertRows(execute("SELECT m FROM %s WHERE k = 0"),
-            row(map("v5", 5))
-        );
-
-        execute("DELETE m[?] FROM %s WHERE k = 0", "v5");
-
-        assertRows(execute("SELECT m FROM %s WHERE k = 0"),
-            row((Object)null)
-        );
-
-        // Deleting a non-existing key should succeed
-        execute("DELETE m[?] FROM %s WHERE k = 0", "v5");
-
-        assertRows(execute("SELECT m FROM %s WHERE k = 0"),
-            row((Object) null)
-        );
-
-        // The empty map is parsed as an empty set (because we don't have enough info at parsing
-        // time when we see a {}) and special cased later. This test checks this work properly
-        execute("UPDATE %s SET m = {} WHERE k = 0");
-
-        assertRows(execute("SELECT m FROM %s WHERE k = 0"),
-            row((Object)null)
-        );
-    }
-
-    @Test
-    public void testLists() throws Throwable
-    {
-        createTable("CREATE TABLE %s (k int PRIMARY KEY, l list<text>)");
-
-        execute("INSERT INTO %s(k, l) VALUES (0, ?)", list("v1", "v2", "v3"));
-
-        assertRows(execute("SELECT l FROM %s WHERE k = 0"), row(list("v1", "v2", "v3")));
-
-        execute("DELETE l[?] FROM %s WHERE k = 0", 1);
-
-        assertRows(execute("SELECT l FROM %s WHERE k = 0"), row(list("v1", "v3")));
-
-        execute("UPDATE %s SET l[?] = ? WHERE k = 0", 1, "v4");
-
-        assertRows(execute("SELECT l FROM %s WHERE k = 0"), row(list("v1", "v4")));
-
-        // Full overwrite
-        execute("UPDATE %s SET l = ? WHERE k = 0", list("v6", "v5"));
-
-        assertRows(execute("SELECT l FROM %s WHERE k = 0"), row(list("v6", "v5")));
-
-        execute("UPDATE %s SET l = l + ? WHERE k = 0", list("v7", "v8"));
-
-        assertRows(execute("SELECT l FROM %s WHERE k = 0"), row(list("v6", "v5", "v7", "v8")));
-
-        execute("UPDATE %s SET l = ? + l WHERE k = 0", list("v9"));
-
-        assertRows(execute("SELECT l FROM %s WHERE k = 0"), row(list("v9", "v6", "v5", "v7", "v8")));
-
-        execute("UPDATE %s SET l = l - ? WHERE k = 0", list("v5", "v8"));
-
-        assertRows(execute("SELECT l FROM %s WHERE k = 0"), row(list("v9", "v6", "v7")));
-
-        execute("DELETE l FROM %s WHERE k = 0");
-
-        assertRows(execute("SELECT l FROM %s WHERE k = 0"), row((Object) null));
-
-        assertInvalidMessage("Attempted to delete an element from a list which is null",
-                             "DELETE l[0] FROM %s WHERE k=0 ");
-
-        assertInvalidMessage("Attempted to set an element on a list which is null",
-                             "UPDATE %s SET l[0] = ? WHERE k=0", list("v10"));
-
-        execute("UPDATE %s SET l = l - ? WHERE k=0 ", list("v11"));
-
-        assertRows(execute("SELECT l FROM %s WHERE k = 0"), row((Object) null));
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f797bfa4/test/unit/org/apache/cassandra/cql3/ContainsRelationTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/ContainsRelationTest.java b/test/unit/org/apache/cassandra/cql3/ContainsRelationTest.java
deleted file mode 100644
index b51a639..0000000
--- a/test/unit/org/apache/cassandra/cql3/ContainsRelationTest.java
+++ /dev/null
@@ -1,269 +0,0 @@
-/*
- * 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;
-
-import org.junit.Test;
-
-public class ContainsRelationTest extends CQLTester
-{
-    @Test
-    public void testSetContains() throws Throwable
-    {
-        createTable("CREATE TABLE %s (account text, id int, categories set<text>, PRIMARY KEY (account, id))");
-        createIndex("CREATE INDEX ON %s(categories)");
-
-        execute("INSERT INTO %s (account, id , categories) VALUES (?, ?, ?)", "test", 5, set("lmn"));
-
-        assertEmpty(execute("SELECT * FROM %s WHERE account = ? AND categories CONTAINS ?", "xyz", "lmn"));
-
-        assertRows(execute("SELECT * FROM %s WHERE categories CONTAINS ?", "lmn"),
-            row("test", 5, set("lmn"))
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE account = ? AND categories CONTAINS ?", "test", "lmn"),
-            row("test", 5, set("lmn"))
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE account = ? AND id = ? AND categories CONTAINS ?", "test", 5, "lmn"),
-                   row("test", 5, set("lmn"))
-        );
-
-        assertInvalid("SELECT * FROM %s WHERE account = ? AND categories CONTAINS ? AND categories CONTAINS ?", "xyz", "lmn", "notPresent");
-        assertEmpty(execute("SELECT * FROM %s WHERE account = ? AND categories CONTAINS ? AND categories CONTAINS ? ALLOW FILTERING", "xyz", "lmn", "notPresent"));
-    }
-
-    @Test
-    public void testListContains() throws Throwable
-    {
-        createTable("CREATE TABLE %s (account text, id int, categories list<text>, PRIMARY KEY (account, id))");
-        createIndex("CREATE INDEX ON %s(categories)");
-
-        execute("INSERT INTO %s (account, id , categories) VALUES (?, ?, ?)", "test", 5, list("lmn"));
-
-        assertEmpty(execute("SELECT * FROM %s WHERE account = ? AND categories CONTAINS ?", "xyz", "lmn"));
-
-        assertRows(execute("SELECT * FROM %s WHERE account = ? AND categories CONTAINS ?;", "test", "lmn"),
-            row("test", 5, list("lmn"))
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE categories CONTAINS ?", "lmn"),
-            row("test", 5, list("lmn"))
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE account = ? AND id = ? AND categories CONTAINS ?;", "test", 5, "lmn"),
-                   row("test", 5, list("lmn"))
-        );
-
-        assertInvalid("SELECT * FROM %s WHERE account = ? AND id = ? AND categories CONTAINS ? AND categories CONTAINS ?",
-                      "test", 5, "lmn", "notPresent");
-        assertEmpty(execute("SELECT * FROM %s WHERE account = ? AND id = ? AND categories CONTAINS ? AND categories CONTAINS ? ALLOW FILTERING",
-                            "test", 5, "lmn", "notPresent"));
-    }
-
-    @Test
-    public void testListContainsWithFiltering() throws Throwable
-    {
-        createTable("CREATE TABLE %s (e int PRIMARY KEY, f list<text>, s int)");
-        createIndex("CREATE INDEX ON %s(f)");
-        for(int i = 0; i < 3; i++)
-        {
-            execute("INSERT INTO %s (e, f, s) VALUES (?, ?, ?)", i, list("Dubai"), 4);
-        }
-        for(int i = 3; i < 5; i++)
-        {
-            execute("INSERT INTO %s (e, f, s) VALUES (?, ?, ?)", i, list("Dubai"), 3);
-        }
-        assertRows(execute("SELECT * FROM %s WHERE f CONTAINS ? AND s=? allow filtering", "Dubai", 3),
-                   row(3, list("Dubai"), 3),
-                   row(4, list("Dubai"), 3));
-    }
-
-    @Test
-    public void testMapKeyContains() throws Throwable
-    {
-        createTable("CREATE TABLE %s (account text, id int, categories map<text,text>, PRIMARY KEY (account, id))");
-        createIndex("CREATE INDEX ON %s(keys(categories))");
-
-        execute("INSERT INTO %s (account, id , categories) VALUES (?, ?, ?)", "test", 5, map("lmn", "foo"));
-
-        assertEmpty(execute("SELECT * FROM %s WHERE account = ? AND categories CONTAINS KEY ?", "xyz", "lmn"));
-
-        assertRows(execute("SELECT * FROM %s WHERE account = ? AND categories CONTAINS KEY ?", "test", "lmn"),
-            row("test", 5, map("lmn", "foo"))
-        );
-        assertRows(execute("SELECT * FROM %s WHERE categories CONTAINS KEY ?", "lmn"),
-            row("test", 5, map("lmn", "foo"))
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE account = ? AND id = ? AND categories CONTAINS KEY ?", "test", 5, "lmn"),
-                   row("test", 5, map("lmn", "foo"))
-        );
-
-        assertInvalid("SELECT * FROM %s WHERE account = ? AND id = ? AND categories CONTAINS KEY ? AND categories CONTAINS KEY ?",
-                      "test", 5, "lmn", "notPresent");
-        assertEmpty(execute("SELECT * FROM %s WHERE account = ? AND id = ? AND categories CONTAINS KEY ? AND categories CONTAINS KEY ? ALLOW FILTERING",
-                            "test", 5, "lmn", "notPresent"));
-
-        assertInvalid("SELECT * FROM %s WHERE account = ? AND id = ? AND categories CONTAINS KEY ? AND categories CONTAINS ?",
-                      "test", 5, "lmn", "foo");
-    }
-
-    @Test
-    public void testMapValueContains() throws Throwable
-    {
-        createTable("CREATE TABLE %s (account text, id int, categories map<text,text>, PRIMARY KEY (account, id))");
-        createIndex("CREATE INDEX ON %s(categories)");
-
-        execute("INSERT INTO %s (account, id , categories) VALUES (?, ?, ?)", "test", 5, map("lmn", "foo"));
-
-        assertEmpty(execute("SELECT * FROM %s WHERE account = ? AND categories CONTAINS ?", "xyz", "foo"));
-
-        assertRows(execute("SELECT * FROM %s WHERE account = ? AND categories CONTAINS ?", "test", "foo"),
-            row("test", 5, map("lmn", "foo"))
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE categories CONTAINS ?", "foo"),
-            row("test", 5, map("lmn", "foo"))
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE account = ? AND id = ? AND categories CONTAINS ?", "test", 5, "foo"),
-                   row("test", 5, map("lmn", "foo"))
-        );
-
-        assertInvalid("SELECT * FROM %s WHERE account = ? AND id = ? AND categories CONTAINS ? AND categories CONTAINS ?"
-                           , "test", 5, "foo", "notPresent");
-
-        assertEmpty(execute("SELECT * FROM %s WHERE account = ? AND id = ? AND categories CONTAINS ? AND categories CONTAINS ? ALLOW FILTERING"
-                           , "test", 5, "foo", "notPresent"));
-    }
-
-    // See CASSANDRA-7525
-    @Test
-    public void testQueryMultipleIndexTypes() throws Throwable
-    {
-        createTable("CREATE TABLE %s (account text, id int, categories map<text,text>, PRIMARY KEY (account, id))");
-
-        // create an index on
-        createIndex("CREATE INDEX id_index ON %s(id)");
-        createIndex("CREATE INDEX categories_values_index ON %s(categories)");
-
-        execute("INSERT INTO %s (account, id , categories) VALUES (?, ?, ?)", "test", 5, map("lmn", "foo"));
-
-        assertRows(execute("SELECT * FROM %s WHERE categories CONTAINS ? AND id = ? ALLOW FILTERING", "foo", 5),
-                row("test", 5, map("lmn", "foo"))
-        );
-
-        assertRows(
-            execute("SELECT * FROM %s WHERE account = ? AND categories CONTAINS ? AND id = ? ALLOW FILTERING", "test", "foo", 5),
-            row("test", 5, map("lmn", "foo"))
-        );
-    }
-
-    // See CASSANDRA-8033
-    @Test
-    public void testFilterForContains() throws Throwable
-    {
-        createTable("CREATE TABLE %s (k1 int, k2 int, v set<int>, PRIMARY KEY ((k1, k2)))");
-        createIndex("CREATE INDEX ON %s(k2)");
-
-        execute("INSERT INTO %s (k1, k2, v) VALUES (?, ?, ?)", 0, 0, set(1, 2, 3));
-        execute("INSERT INTO %s (k1, k2, v) VALUES (?, ?, ?)", 0, 1, set(2, 3, 4));
-        execute("INSERT INTO %s (k1, k2, v) VALUES (?, ?, ?)", 1, 0, set(3, 4, 5));
-        execute("INSERT INTO %s (k1, k2, v) VALUES (?, ?, ?)", 1, 1, set(4, 5, 6));
-
-        assertRows(execute("SELECT * FROM %s WHERE k2 = ?", 1),
-            row(0, 1, set(2, 3, 4)),
-            row(1, 1, set(4, 5, 6))
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE k2 = ? AND v CONTAINS ? ALLOW FILTERING", 1, 6),
-            row(1, 1, set(4, 5, 6))
-        );
-
-        assertEmpty(execute("SELECT * FROM %s WHERE k2 = ? AND v CONTAINS ? ALLOW FILTERING", 1, 7));
-    }
-
-    // See CASSANDRA-8073
-    @Test
-    public void testIndexLookupWithClusteringPrefix() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a int, b int, c int, d set<int>, PRIMARY KEY (a, b, c))");
-        createIndex("CREATE INDEX ON %s(d)");
-        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 0, set(1, 2, 3));
-        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 1, set(3, 4, 5));
-        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 0, set(1, 2, 3));
-        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 1, set(3, 4, 5));
-
-        assertRows(execute("SELECT * FROM %s WHERE a=? AND b=? AND d CONTAINS ?", 0, 1, 3),
-            row(0, 1, 0, set(1, 2, 3)),
-            row(0, 1, 1, set(3, 4, 5))
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE a=? AND b=? AND d CONTAINS ?", 0, 1, 2),
-            row(0, 1, 0, set(1, 2, 3))
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE a=? AND b=? AND d CONTAINS ?", 0, 1, 5),
-            row(0, 1, 1, set(3, 4, 5))
-        );
-    }
-
-    @Test
-    public void testContainsKeyAndContainsWithIndexOnMapKey() throws Throwable
-    {
-        createTable("CREATE TABLE %s (account text, id int, categories map<text,text>, PRIMARY KEY (account, id))");
-        createIndex("CREATE INDEX ON %s(keys(categories))");
-
-        execute("INSERT INTO %s (account, id , categories) VALUES (?, ?, ?)", "test", 5, map("lmn", "foo"));
-        execute("INSERT INTO %s (account, id , categories) VALUES (?, ?, ?)", "test", 6, map("lmn", "foo2"));
-
-        assertInvalid("SELECT * FROM %s WHERE account = ? AND categories CONTAINS ?", "test", "foo");
-
-        assertRows(execute("SELECT * FROM %s WHERE account = ? AND categories CONTAINS KEY ?", "test", "lmn"),
-                   row("test", 5, map("lmn", "foo")),
-                   row("test", 6, map("lmn", "foo2")));
-        assertRows(execute("SELECT * FROM %s WHERE account = ? AND categories CONTAINS KEY ? AND categories CONTAINS ? ALLOW FILTERING",
-                           "test", "lmn", "foo"),
-                   row("test", 5, map("lmn", "foo")));
-        assertRows(execute("SELECT * FROM %s WHERE account = ? AND categories CONTAINS ? AND categories CONTAINS KEY ? ALLOW FILTERING",
-                           "test", "foo", "lmn"),
-                   row("test", 5, map("lmn", "foo")));
-    }
-
-    @Test
-    public void testContainsKeyAndContainsWithIndexOnMapValue() throws Throwable
-    {
-        createTable("CREATE TABLE %s (account text, id int, categories map<text,text>, PRIMARY KEY (account, id))");
-        createIndex("CREATE INDEX ON %s(categories)");
-
-        execute("INSERT INTO %s (account, id , categories) VALUES (?, ?, ?)", "test", 5, map("lmn", "foo"));
-        execute("INSERT INTO %s (account, id , categories) VALUES (?, ?, ?)", "test", 6, map("lmn2", "foo"));
-
-        assertInvalid("SELECT * FROM %s WHERE account = ? AND categories CONTAINS KEY ?", "test", "lmn");
-
-        assertRows(execute("SELECT * FROM %s WHERE account = ? AND categories CONTAINS ?", "test", "foo"),
-                   row("test", 5, map("lmn", "foo")),
-                   row("test", 6, map("lmn2", "foo")));
-        assertRows(execute("SELECT * FROM %s WHERE account = ? AND categories CONTAINS KEY ? AND categories CONTAINS ? ALLOW FILTERING",
-                           "test", "lmn", "foo"),
-                   row("test", 5, map("lmn", "foo")));
-        assertRows(execute("SELECT * FROM %s WHERE account = ? AND categories CONTAINS ? AND categories CONTAINS KEY ? ALLOW FILTERING",
-                           "test", "foo", "lmn"),
-                   row("test", 5, map("lmn", "foo")));
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f797bfa4/test/unit/org/apache/cassandra/cql3/CrcCheckChanceTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/CrcCheckChanceTest.java b/test/unit/org/apache/cassandra/cql3/CrcCheckChanceTest.java
deleted file mode 100644
index bed3cdd..0000000
--- a/test/unit/org/apache/cassandra/cql3/CrcCheckChanceTest.java
+++ /dev/null
@@ -1,159 +0,0 @@
-/*
- * 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;
-
-import java.util.List;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.Future;
-
-import junit.framework.Assert;
-import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.db.ColumnFamilyStore;
-import org.apache.cassandra.db.Keyspace;
-import org.apache.cassandra.db.compaction.CompactionInterruptedException;
-import org.apache.cassandra.db.compaction.CompactionManager;
-import org.apache.cassandra.utils.FBUtilities;
-
-import org.junit.Test;
-
-
-public class CrcCheckChanceTest extends CQLTester
-{
-    @Test
-    public void testChangingCrcCheckChance() throws Throwable
-    {
-        //Start with crc_check_chance of 99%
-        createTable("CREATE TABLE %s (p text, c text, v text, s text static, PRIMARY KEY (p, c)) WITH compression = {'sstable_compression': 'LZ4Compressor', 'crc_check_chance' : 0.99}");
-
-        execute("CREATE INDEX foo ON %s(v)");
-
-        execute("INSERT INTO %s(p, c, v, s) values (?, ?, ?, ?)", "p1", "k1", "v1", "sv1");
-        execute("INSERT INTO %s(p, c, v) values (?, ?, ?)", "p1", "k2", "v2");
-        execute("INSERT INTO %s(p, s) values (?, ?)", "p2", "sv2");
-
-
-        ColumnFamilyStore cfs = Keyspace.open(CQLTester.KEYSPACE).getColumnFamilyStore(currentTable());
-        ColumnFamilyStore indexCfs = cfs.indexManager.getIndexesBackedByCfs().iterator().next();
-        cfs.forceBlockingFlush();
-
-        Assert.assertEquals(0.99, cfs.metadata.compressionParameters.getCrcCheckChance());
-        Assert.assertEquals(0.99, cfs.getSSTables().iterator().next().getCompressionMetadata().parameters.getCrcCheckChance());
-        Assert.assertEquals(0.99, indexCfs.metadata.compressionParameters.getCrcCheckChance());
-        Assert.assertEquals(0.99, indexCfs.getSSTables().iterator().next().getCompressionMetadata().parameters.getCrcCheckChance());
-
-        //Test for stack overflow
-        cfs.setCrcCheckChance(0.99);
-
-        assertRows(execute("SELECT * FROM %s WHERE p=?", "p1"),
-                row("p1", "k1", "sv1", "v1"),
-                row("p1", "k2", "sv1", "v2")
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE v=?", "v1"),
-                row("p1", "k1", "sv1", "v1")
-        );
-
-
-
-        //Write a few SSTables then Compact
-
-        execute("INSERT INTO %s(p, c, v, s) values (?, ?, ?, ?)", "p1", "k1", "v1", "sv1");
-        execute("INSERT INTO %s(p, c, v) values (?, ?, ?)", "p1", "k2", "v2");
-        execute("INSERT INTO %s(p, s) values (?, ?)", "p2", "sv2");
-
-        cfs.forceBlockingFlush();
-
-
-        execute("INSERT INTO %s(p, c, v, s) values (?, ?, ?, ?)", "p1", "k1", "v1", "sv1");
-        execute("INSERT INTO %s(p, c, v) values (?, ?, ?)", "p1", "k2", "v2");
-        execute("INSERT INTO %s(p, s) values (?, ?)", "p2", "sv2");
-
-        cfs.forceBlockingFlush();
-
-        execute("INSERT INTO %s(p, c, v, s) values (?, ?, ?, ?)", "p1", "k1", "v1", "sv1");
-        execute("INSERT INTO %s(p, c, v) values (?, ?, ?)", "p1", "k2", "v2");
-        execute("INSERT INTO %s(p, s) values (?, ?)", "p2", "sv2");
-
-        cfs.forceBlockingFlush();
-
-        cfs.forceMajorCompaction();
-
-        //Verify when we alter the value the live sstable readers hold the new one
-        alterTable("ALTER TABLE %s WITH compression = {'sstable_compression': 'LZ4Compressor', 'crc_check_chance': 0.01}");
-
-        Assert.assertEquals( 0.01, cfs.metadata.compressionParameters.getCrcCheckChance());
-        Assert.assertEquals( 0.01, cfs.getSSTables().iterator().next().getCompressionMetadata().parameters.getCrcCheckChance());
-        Assert.assertEquals( 0.01, indexCfs.metadata.compressionParameters.getCrcCheckChance());
-        Assert.assertEquals( 0.01, indexCfs.getSSTables().iterator().next().getCompressionMetadata().parameters.getCrcCheckChance());
-
-        assertRows(execute("SELECT * FROM %s WHERE p=?", "p1"),
-                row("p1", "k1", "sv1", "v1"),
-                row("p1", "k2", "sv1", "v2")
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE v=?", "v1"),
-                row("p1", "k1", "sv1", "v1")
-        );
-
-
-        //Verify the call used by JMX still works
-        cfs.setCrcCheckChance(0.03);
-        Assert.assertEquals( 0.03, cfs.metadata.compressionParameters.getCrcCheckChance());
-        Assert.assertEquals( 0.03, cfs.getSSTables().iterator().next().getCompressionMetadata().parameters.getCrcCheckChance());
-        Assert.assertEquals( 0.03, indexCfs.metadata.compressionParameters.getCrcCheckChance());
-        Assert.assertEquals( 0.03, indexCfs.getSSTables().iterator().next().getCompressionMetadata().parameters.getCrcCheckChance());
-
-    }
-
-
-    @Test
-    public void testDropDuringCompaction() throws Throwable
-    {
-        CompactionManager.instance.disableAutoCompaction();
-
-        //Start with crc_check_chance of 99%
-        createTable("CREATE TABLE %s (p text, c text, v text, s text static, PRIMARY KEY (p, c)) WITH compression = {'sstable_compression': 'LZ4Compressor', 'crc_check_chance' : 0.99}");
-
-        ColumnFamilyStore cfs = Keyspace.open(CQLTester.KEYSPACE).getColumnFamilyStore(currentTable());
-
-        //Write a few SSTables then Compact, and drop
-        for (int i = 0; i < 100; i++)
-        {
-            execute("INSERT INTO %s(p, c, v, s) values (?, ?, ?, ?)", "p1", "k1", "v1", "sv1");
-            execute("INSERT INTO %s(p, c, v) values (?, ?, ?)", "p1", "k2", "v2");
-            execute("INSERT INTO %s(p, s) values (?, ?)", "p2", "sv2");
-
-            cfs.forceBlockingFlush();
-        }
-
-        DatabaseDescriptor.setCompactionThroughputMbPerSec(1);
-        List<Future<?>> futures = CompactionManager.instance.submitMaximal(cfs, CompactionManager.GC_ALL); 
-        execute("DROP TABLE %s");
-
-        try
-        {
-            FBUtilities.waitOnFutures(futures);
-        }
-        catch (Throwable t)
-        {
-            if (!(t.getCause() instanceof ExecutionException) || !(t.getCause().getCause() instanceof CompactionInterruptedException))
-                throw t;
-        }
-    }
-}
-

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f797bfa4/test/unit/org/apache/cassandra/cql3/CreateAndAlterKeyspaceTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/CreateAndAlterKeyspaceTest.java b/test/unit/org/apache/cassandra/cql3/CreateAndAlterKeyspaceTest.java
deleted file mode 100644
index 9e0ca21..0000000
--- a/test/unit/org/apache/cassandra/cql3/CreateAndAlterKeyspaceTest.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * 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;
-
-import org.junit.Test;
-
-public class CreateAndAlterKeyspaceTest extends CQLTester
-{
-    @Test
-    // tests CASSANDRA-9565
-    public void testCreateAndAlterWithDoubleWith() throws Throwable
-    {
-        String[] stmts = new String[] {"ALTER KEYSPACE WITH WITH DURABLE_WRITES = true",
-                                       "ALTER KEYSPACE ks WITH WITH DURABLE_WRITES = true",
-                                       "CREATE KEYSPACE WITH WITH DURABLE_WRITES = true",
-                                       "CREATE KEYSPACE ks WITH WITH DURABLE_WRITES = true"};
-
-        for (String stmt : stmts) {
-            assertInvalidSyntaxMessage("no viable alternative at input 'WITH'", stmt);
-        }
-    }
-}


[15/32] cassandra git commit: 2.2 commit for CASSANDRA-9160

Posted by jm...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/01115f72/test/unit/org/apache/cassandra/cql3/validation/operations/AggregationTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/AggregationTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/AggregationTest.java
new file mode 100644
index 0000000..3f6fdda
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/AggregationTest.java
@@ -0,0 +1,1481 @@
+/*
+ * 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.validation.operations;
+
+import java.math.BigDecimal;
+import java.text.SimpleDateFormat;
+import java.util.Calendar;
+import java.util.Date;
+import java.util.TimeZone;
+
+import org.apache.commons.lang3.time.DateUtils;
+import org.junit.Assert;
+import org.junit.Test;
+
+import org.apache.cassandra.cql3.QueryProcessor;
+import org.apache.cassandra.cql3.functions.Functions;
+import org.apache.cassandra.cql3.functions.UDAggregate;
+import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.exceptions.FunctionExecutionException;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.serializers.Int32Serializer;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.transport.Event;
+import org.apache.cassandra.transport.messages.ResultMessage;
+
+public class AggregationTest extends CQLTester
+{
+    @Test
+    public void testFunctions() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int, b int, c double, d decimal, primary key (a, b))");
+
+        // Test with empty table
+        assertColumnNames(execute("SELECT COUNT(*) FROM %s"), "count");
+        assertRows(execute("SELECT COUNT(*) FROM %s"), row(0L));
+        assertColumnNames(execute("SELECT max(b), min(b), sum(b), avg(b) , max(c), sum(c), avg(c), sum(d), avg(d) FROM %s"),
+                          "system.max(b)", "system.min(b)", "system.sum(b)", "system.avg(b)", "system.max(c)", "system.sum(c)", "system.avg(c)", "system.sum(d)", "system.avg(d)");
+        assertRows(execute("SELECT max(b), min(b), sum(b), avg(b) , max(c), sum(c), avg(c), sum(d), avg(d) FROM %s"),
+                   row(null, null, 0, 0, null, 0.0, 0.0, new BigDecimal("0"), new BigDecimal("0")));
+
+        execute("INSERT INTO %s (a, b, c, d) VALUES (1, 1, 11.5, 11.5)");
+        execute("INSERT INTO %s (a, b, c, d) VALUES (1, 2, 9.5, 1.5)");
+        execute("INSERT INTO %s (a, b, c, d) VALUES (1, 3, 9.0, 2.0)");
+
+        assertRows(execute("SELECT max(b), min(b), sum(b), avg(b) , max(c), sum(c), avg(c), sum(d), avg(d) FROM %s"),
+                   row(3, 1, 6, 2, 11.5, 30.0, 10.0, new BigDecimal("15.0"), new BigDecimal("5.0")));
+
+        execute("INSERT INTO %s (a, b, d) VALUES (1, 5, 1.0)");
+        assertRows(execute("SELECT COUNT(*) FROM %s"), row(4L));
+        assertRows(execute("SELECT COUNT(1) FROM %s"), row(4L));
+        assertRows(execute("SELECT COUNT(b), count(c) FROM %s"), row(4L, 3L));
+    }
+
+    @Test
+    public void testFunctionsWithCompactStorage() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int , b int, c double, primary key(a, b) ) WITH COMPACT STORAGE");
+
+        execute("INSERT INTO %s (a, b, c) VALUES (1, 1, 11.5)");
+        execute("INSERT INTO %s (a, b, c) VALUES (1, 2, 9.5)");
+        execute("INSERT INTO %s (a, b, c) VALUES (1, 3, 9.0)");
+
+        assertRows(execute("SELECT max(b), min(b), sum(b), avg(b) , max(c), sum(c), avg(c) FROM %s"),
+                   row(3, 1, 6, 2, 11.5, 30.0, 10.0));
+
+        assertRows(execute("SELECT COUNT(*) FROM %s"), row(3L));
+        assertRows(execute("SELECT COUNT(1) FROM %s"), row(3L));
+        assertRows(execute("SELECT COUNT(*) FROM %s WHERE a = 1 AND b > 1"), row(2L));
+        assertRows(execute("SELECT COUNT(1) FROM %s WHERE a = 1 AND b > 1"), row(2L));
+        assertRows(execute("SELECT max(b), min(b), sum(b), avg(b) , max(c), sum(c), avg(c) FROM %s WHERE a = 1 AND b > 1"),
+                   row(3, 2, 5, 2, 9.5, 18.5, 9.25));
+    }
+
+    @Test
+    public void testInvalidCalls() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int, b int, c int, primary key (a, b))");
+        execute("INSERT INTO %s (a, b, c) VALUES (1, 1, 10)");
+        execute("INSERT INTO %s (a, b, c) VALUES (1, 2, 9)");
+        execute("INSERT INTO %s (a, b, c) VALUES (1, 3, 8)");
+
+        assertInvalidSyntax("SELECT max(b), max(c) FROM %s WHERE max(a) = 1");
+        assertInvalidMessage("only aggregates or no aggregate", "SELECT max(b), c FROM %s");
+        assertInvalidMessage("only aggregates or no aggregate", "SELECT b, max(c) FROM %s");
+        assertInvalidMessage("aggregate functions cannot be used as arguments of aggregate functions", "SELECT max(sum(c)) FROM %s");
+        assertInvalidSyntax("SELECT COUNT(2) FROM %s");
+    }
+
+    @Test
+    public void testNestedFunctions() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int primary key, b timeuuid, c double, d double)");
+
+        String copySign = createFunction(KEYSPACE,
+                                         "double, double",
+                                         "CREATE OR REPLACE FUNCTION %s(magnitude double, sign double) " +
+                                         "RETURNS NULL ON NULL INPUT " +
+                                         "RETURNS double " +
+                                         "LANGUAGE JAVA " +
+                                         "AS 'return Double.valueOf(Math.copySign(magnitude, sign));';");
+
+        assertColumnNames(execute("SELECT max(a), max(toUnixTimestamp(b)) FROM %s"), "system.max(a)", "system.max(system.tounixtimestamp(b))");
+        assertRows(execute("SELECT max(a), max(toUnixTimestamp(b)) FROM %s"), row(null, null));
+        assertColumnNames(execute("SELECT max(a), toUnixTimestamp(max(b)) FROM %s"), "system.max(a)", "system.tounixtimestamp(system.max(b))");
+        assertRows(execute("SELECT max(a), toUnixTimestamp(max(b)) FROM %s"), row(null, null));
+
+        assertColumnNames(execute("SELECT max(" + copySign + "(c, d)) FROM %s"), "system.max(" + copySign + "(c, d))");
+        assertRows(execute("SELECT max(" + copySign + "(c, d)) FROM %s"), row((Object) null));
+
+        execute("INSERT INTO %s (a, b, c, d) VALUES (1, maxTimeuuid('2011-02-03 04:05:00+0000'), -1.2, 2.1)");
+        execute("INSERT INTO %s (a, b, c, d) VALUES (2, maxTimeuuid('2011-02-03 04:06:00+0000'), 1.3, -3.4)");
+        execute("INSERT INTO %s (a, b, c, d) VALUES (3, maxTimeuuid('2011-02-03 04:10:00+0000'), 1.4, 1.2)");
+
+        SimpleDateFormat format = new SimpleDateFormat("yyyy-MM-dd hh:mm:ss");
+        format.setTimeZone(TimeZone.getTimeZone("GMT"));
+        Date date = format.parse("2011-02-03 04:10:00");
+        date = DateUtils.truncate(date, Calendar.MILLISECOND);
+
+        assertRows(execute("SELECT max(a), max(toUnixTimestamp(b)) FROM %s"), row(3, date.getTime()));
+        assertRows(execute("SELECT max(a), toUnixTimestamp(max(b)) FROM %s"), row(3, date.getTime()));
+
+        assertRows(execute("SELECT " + copySign + "(max(c), min(c)) FROM %s"), row(-1.4));
+        assertRows(execute("SELECT " + copySign + "(c, d) FROM %s"), row(1.2), row(-1.3), row(1.4));
+        assertRows(execute("SELECT max(" + copySign + "(c, d)) FROM %s"), row(1.4));
+        assertInvalidMessage("must be either all aggregates or no aggregates", "SELECT " + copySign + "(c, max(c)) FROM %s");
+        assertInvalidMessage("must be either all aggregates or no aggregates", "SELECT " + copySign + "(max(c), c) FROM %s");
+    }
+
+    @Test
+    public void testSchemaChange() throws Throwable
+    {
+        String f = createFunction(KEYSPACE,
+                                  "double, double",
+                                  "CREATE OR REPLACE FUNCTION %s(state double, val double) " +
+                                  "RETURNS NULL ON NULL INPUT " +
+                                  "RETURNS double " +
+                                  "LANGUAGE javascript " +
+                                  "AS '\"string\";';");
+
+        createFunctionOverload(f,
+                               "double, double",
+                               "CREATE OR REPLACE FUNCTION %s(state int, val int) " +
+                               "RETURNS NULL ON NULL INPUT " +
+                               "RETURNS int " +
+                               "LANGUAGE javascript " +
+                               "AS '\"string\";';");
+
+        String a = createAggregate(KEYSPACE,
+                                   "double",
+                                   "CREATE OR REPLACE AGGREGATE %s(double) " +
+                                   "SFUNC " + shortFunctionName(f) + " " +
+                                   "STYPE double " +
+                                   "INITCOND 0");
+
+        assertLastSchemaChange(Event.SchemaChange.Change.CREATED, Event.SchemaChange.Target.AGGREGATE,
+                               KEYSPACE, parseFunctionName(a).name,
+                               "double");
+
+        schemaChange("CREATE OR REPLACE AGGREGATE " + a + "(double) " +
+                     "SFUNC " + shortFunctionName(f) + " " +
+                     "STYPE double " +
+                     "INITCOND 0");
+
+        assertLastSchemaChange(Event.SchemaChange.Change.UPDATED, Event.SchemaChange.Target.AGGREGATE,
+                               KEYSPACE, parseFunctionName(a).name,
+                               "double");
+
+        createAggregateOverload(a,
+                                "int",
+                                "CREATE OR REPLACE AGGREGATE %s(int) " +
+                                "SFUNC " + shortFunctionName(f) + " " +
+                                "STYPE int " +
+                                "INITCOND 0");
+
+        assertLastSchemaChange(Event.SchemaChange.Change.CREATED, Event.SchemaChange.Target.AGGREGATE,
+                               KEYSPACE, parseFunctionName(a).name,
+                               "int");
+
+        schemaChange("DROP AGGREGATE " + a + "(double)");
+
+        assertLastSchemaChange(Event.SchemaChange.Change.DROPPED, Event.SchemaChange.Target.AGGREGATE,
+                               KEYSPACE, parseFunctionName(a).name,
+                               "double");
+    }
+
+    @Test
+    public void testDropStatements() throws Throwable
+    {
+        String f = createFunction(KEYSPACE,
+                                  "double, double",
+                                  "CREATE OR REPLACE FUNCTION %s(state double, val double) " +
+                                  "RETURNS NULL ON NULL INPUT " +
+                                  "RETURNS double " +
+                                  "LANGUAGE javascript " +
+                                  "AS '\"string\";';");
+
+        createFunctionOverload(f,
+                               "double, double",
+                               "CREATE OR REPLACE FUNCTION %s(state int, val int) " +
+                               "RETURNS NULL ON NULL INPUT " +
+                               "RETURNS int " +
+                               "LANGUAGE javascript " +
+                               "AS '\"string\";';");
+
+        // DROP AGGREGATE must not succeed against a scalar
+        assertInvalidMessage("matches multiple function definitions", "DROP AGGREGATE " + f);
+        assertInvalidMessage("non existing", "DROP AGGREGATE " + f + "(double, double)");
+
+        String a = createAggregate(KEYSPACE,
+                                   "double",
+                                   "CREATE OR REPLACE AGGREGATE %s(double) " +
+                                   "SFUNC " + shortFunctionName(f) + " " +
+                                   "STYPE double " +
+                                   "INITCOND 0");
+        createAggregateOverload(a,
+                                "int",
+                                "CREATE OR REPLACE AGGREGATE %s(int) " +
+                                "SFUNC " + shortFunctionName(f) + " " +
+                                "STYPE int " +
+                                "INITCOND 0");
+
+        // DROP FUNCTION must not succeed against an aggregate
+        assertInvalidMessage("matches multiple function definitions", "DROP FUNCTION " + a);
+        assertInvalidMessage("non existing function", "DROP FUNCTION " + a + "(double)");
+
+        // ambigious
+        assertInvalidMessage("matches multiple function definitions", "DROP AGGREGATE " + a);
+        assertInvalidMessage("matches multiple function definitions", "DROP AGGREGATE IF EXISTS " + a);
+
+        execute("DROP AGGREGATE IF EXISTS " + KEYSPACE + ".non_existing");
+        execute("DROP AGGREGATE IF EXISTS " + a + "(int, text)");
+
+        execute("DROP AGGREGATE " + a + "(double)");
+
+        execute("DROP AGGREGATE IF EXISTS " + a + "(double)");
+    }
+
+    @Test
+    public void testDropReferenced() throws Throwable
+    {
+        String f = createFunction(KEYSPACE,
+                                  "double, double",
+                                  "CREATE OR REPLACE FUNCTION %s(state double, val double) " +
+                                  "RETURNS NULL ON NULL INPUT " +
+                                  "RETURNS double " +
+                                  "LANGUAGE javascript " +
+                                  "AS '\"string\";';");
+
+        String a = createAggregate(KEYSPACE,
+                                   "double",
+                                   "CREATE OR REPLACE AGGREGATE %s(double) " +
+                                   "SFUNC " + shortFunctionName(f) + " " +
+                                   "STYPE double " +
+                                   "INITCOND 0");
+
+        // DROP FUNCTION must not succeed because the function is still referenced by the aggregate
+        assertInvalidMessage("still referenced by", "DROP FUNCTION " + f);
+
+        execute("DROP AGGREGATE " + a + "(double)");
+    }
+
+    @Test
+    public void testJavaAggregateNoInit() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int primary key, b int)");
+        execute("INSERT INTO %s (a, b) VALUES (1, 1)");
+        execute("INSERT INTO %s (a, b) VALUES (2, 2)");
+        execute("INSERT INTO %s (a, b) VALUES (3, 3)");
+
+        String fState = createFunction(KEYSPACE,
+                                       "int, int",
+                                       "CREATE FUNCTION %s(a int, b int) " +
+                                       "CALLED ON NULL INPUT " +
+                                       "RETURNS int " +
+                                       "LANGUAGE java " +
+                                       "AS 'return Integer.valueOf((a!=null?a.intValue():0) + b.intValue());'");
+
+        String fFinal = createFunction(KEYSPACE,
+                                       "int",
+                                       "CREATE FUNCTION %s(a int) " +
+                                       "CALLED ON NULL INPUT " +
+                                       "RETURNS text " +
+                                       "LANGUAGE java " +
+                                       "AS 'return a.toString();'");
+
+        String a = createAggregate(KEYSPACE,
+                                   "int",
+                                   "CREATE AGGREGATE %s(int) " +
+                                   "SFUNC " + shortFunctionName(fState) + " " +
+                                   "STYPE int " +
+                                   "FINALFUNC " + shortFunctionName(fFinal));
+
+        // 1 + 2 + 3 = 6
+        assertRows(execute("SELECT " + a + "(b) FROM %s"), row("6"));
+
+        execute("DROP AGGREGATE " + a + "(int)");
+
+        assertInvalidMessage("Unknown function", "SELECT " + a + "(b) FROM %s");
+    }
+
+    @Test
+    public void testJavaAggregateNullInitcond() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int primary key, b int)");
+        execute("INSERT INTO %s (a, b) VALUES (1, 1)");
+        execute("INSERT INTO %s (a, b) VALUES (2, 2)");
+        execute("INSERT INTO %s (a, b) VALUES (3, 3)");
+
+        String fState = createFunction(KEYSPACE,
+                                       "int, int",
+                                       "CREATE FUNCTION %s(a int, b int) " +
+                                       "CALLED ON NULL INPUT " +
+                                       "RETURNS int " +
+                                       "LANGUAGE java " +
+                                       "AS 'return Integer.valueOf((a!=null?a.intValue():0) + b.intValue());'");
+
+        String fFinal = createFunction(KEYSPACE,
+                                       "int",
+                                       "CREATE FUNCTION %s(a int) " +
+                                       "CALLED ON NULL INPUT " +
+                                       "RETURNS text " +
+                                       "LANGUAGE java " +
+                                       "AS 'return a.toString();'");
+
+        String a = createAggregate(KEYSPACE,
+                                   "int",
+                                   "CREATE AGGREGATE %s(int) " +
+                                   "SFUNC " + shortFunctionName(fState) + " " +
+                                   "STYPE int " +
+                                   "FINALFUNC " + shortFunctionName(fFinal) + " " +
+                                   "INITCOND null");
+
+        // 1 + 2 + 3 = 6
+        assertRows(execute("SELECT " + a + "(b) FROM %s"), row("6"));
+
+        execute("DROP AGGREGATE " + a + "(int)");
+
+        assertInvalidMessage("Unknown function", "SELECT " + a + "(b) FROM %s");
+    }
+
+    @Test
+    public void testJavaAggregateInvalidInitcond() throws Throwable
+    {
+        String fState = createFunction(KEYSPACE,
+                                       "int, int",
+                                       "CREATE FUNCTION %s(a int, b int) " +
+                                       "CALLED ON NULL INPUT " +
+                                       "RETURNS int " +
+                                       "LANGUAGE java " +
+                                       "AS 'return Integer.valueOf((a!=null?a.intValue():0) + b.intValue());'");
+
+        String fFinal = createFunction(KEYSPACE,
+                                       "int",
+                                       "CREATE FUNCTION %s(a int) " +
+                                       "CALLED ON NULL INPUT " +
+                                       "RETURNS text " +
+                                       "LANGUAGE java " +
+                                       "AS 'return a.toString();'");
+
+        assertInvalidMessage("Invalid STRING constant (foobar)",
+                             "CREATE AGGREGATE " + KEYSPACE + ".aggrInvalid(int)" +
+                             "SFUNC " + shortFunctionName(fState) + " " +
+                             "STYPE int " +
+                             "FINALFUNC " + shortFunctionName(fFinal) + " " +
+                             "INITCOND 'foobar'");
+    }
+
+    @Test
+    public void testJavaAggregateIncompatibleTypes() throws Throwable
+    {
+        String fState = createFunction(KEYSPACE,
+                                       "int, int",
+                                       "CREATE FUNCTION %s(a int, b int) " +
+                                       "CALLED ON NULL INPUT " +
+                                       "RETURNS int " +
+                                       "LANGUAGE java " +
+                                       "AS 'return Integer.valueOf((a!=null?a.intValue():0) + b.intValue());'");
+
+        String fFinal = createFunction(KEYSPACE,
+                                       "int",
+                                       "CREATE FUNCTION %s(a int) " +
+                                       "CALLED ON NULL INPUT " +
+                                       "RETURNS text " +
+                                       "LANGUAGE java " +
+                                       "AS 'return a.toString();'");
+
+        String fState2 = createFunction(KEYSPACE,
+                                        "int, int",
+                                        "CREATE FUNCTION %s(a double, b double) " +
+                                        "CALLED ON NULL INPUT " +
+                                        "RETURNS double " +
+                                        "LANGUAGE java " +
+                                        "AS 'return Double.valueOf((a!=null?a.doubleValue():0d) + b.doubleValue());'");
+
+        String fFinal2 = createFunction(KEYSPACE,
+                                        "int",
+                                        "CREATE FUNCTION %s(a double) " +
+                                        "CALLED ON NULL INPUT " +
+                                        "RETURNS text " +
+                                        "LANGUAGE java " +
+                                        "AS 'return a.toString();'");
+
+        assertInvalidMessage("does not exist or is not a scalar function",
+                             "CREATE AGGREGATE " + KEYSPACE + ".aggrInvalid(double)" +
+                             "SFUNC " + shortFunctionName(fState) + " " +
+                             "STYPE double " +
+                             "FINALFUNC " + shortFunctionName(fFinal));
+        assertInvalidMessage("does not exist or is not a scalar function",
+                             "CREATE AGGREGATE " + KEYSPACE + ".aggrInvalid(int)" +
+                             "SFUNC " + shortFunctionName(fState) + " " +
+                             "STYPE double " +
+                             "FINALFUNC " + shortFunctionName(fFinal));
+        assertInvalidMessage("does not exist or is not a scalar function",
+                             "CREATE AGGREGATE " + KEYSPACE + ".aggrInvalid(double)" +
+                             "SFUNC " + shortFunctionName(fState) + " " +
+                             "STYPE int " +
+                             "FINALFUNC " + shortFunctionName(fFinal));
+        assertInvalidMessage("does not exist or is not a scalar function",
+                             "CREATE AGGREGATE " + KEYSPACE + ".aggrInvalid(double)" +
+                             "SFUNC " + shortFunctionName(fState) + " " +
+                             "STYPE int");
+        assertInvalidMessage("does not exist or is not a scalar function",
+                             "CREATE AGGREGATE " + KEYSPACE + ".aggrInvalid(int)" +
+                             "SFUNC " + shortFunctionName(fState) + " " +
+                             "STYPE double");
+
+        assertInvalidMessage("does not exist or is not a scalar function",
+                             "CREATE AGGREGATE " + KEYSPACE + ".aggrInvalid(double)" +
+                             "SFUNC " + shortFunctionName(fState2) + " " +
+                             "STYPE double " +
+                             "FINALFUNC " + shortFunctionName(fFinal));
+
+        assertInvalidMessage("does not exist or is not a scalar function",
+                             "CREATE AGGREGATE " + KEYSPACE + ".aggrInvalid(double)" +
+                             "SFUNC " + shortFunctionName(fState) + " " +
+                             "STYPE double " +
+                             "FINALFUNC " + shortFunctionName(fFinal2));
+    }
+
+    @Test
+    public void testJavaAggregateNonExistingFuncs() throws Throwable
+    {
+        String fState = createFunction(KEYSPACE,
+                                       "int, int",
+                                       "CREATE FUNCTION %s(a int, b int) " +
+                                       "CALLED ON NULL INPUT " +
+                                       "RETURNS int " +
+                                       "LANGUAGE java " +
+                                       "AS 'return Integer.valueOf((a!=null?a.intValue():0) + b.intValue());'");
+
+        String fFinal = createFunction(KEYSPACE,
+                                       "int",
+                                       "CREATE FUNCTION %s(a int) " +
+                                       "CALLED ON NULL INPUT " +
+                                       "RETURNS text " +
+                                       "LANGUAGE java " +
+                                       "AS 'return a.toString();'");
+
+        assertInvalidMessage("does not exist or is not a scalar function",
+                             "CREATE AGGREGATE " + KEYSPACE + ".aggrInvalid(int)" +
+                             "SFUNC " + shortFunctionName(fState) + "_not_there " +
+                             "STYPE int " +
+                             "FINALFUNC " + shortFunctionName(fFinal));
+
+        assertInvalidMessage("does not exist or is not a scalar function",
+                             "CREATE AGGREGATE " + KEYSPACE + ".aggrInvalid(int)" +
+                             "SFUNC " + shortFunctionName(fState) + " " +
+                             "STYPE int " +
+                             "FINALFUNC " + shortFunctionName(fFinal) + "_not_there");
+
+        execute("CREATE AGGREGATE " + KEYSPACE + ".aggrInvalid(int)" +
+                "SFUNC " + shortFunctionName(fState) + " " +
+                "STYPE int " +
+                "FINALFUNC " + shortFunctionName(fFinal));
+        execute("DROP AGGREGATE " + KEYSPACE + ".aggrInvalid(int)");
+    }
+
+    @Test
+    public void testJavaAggregateFailingFuncs() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int primary key, b int)");
+        execute("INSERT INTO %s (a, b) VALUES (1, 1)");
+        execute("INSERT INTO %s (a, b) VALUES (2, 2)");
+        execute("INSERT INTO %s (a, b) VALUES (3, 3)");
+
+        String fState = createFunction(KEYSPACE,
+                                       "int, int",
+                                       "CREATE FUNCTION %s(a int, b int) " +
+                                       "CALLED ON NULL INPUT " +
+                                       "RETURNS int " +
+                                       "LANGUAGE java " +
+                                       "AS 'throw new RuntimeException(\"thrown to unit test - not a bug\");'");
+
+        String fStateOK = createFunction(KEYSPACE,
+                                       "int, int",
+                                       "CREATE FUNCTION %s(a int, b int) " +
+                                       "CALLED ON NULL INPUT " +
+                                       "RETURNS int " +
+                                       "LANGUAGE java " +
+                                       "AS 'return Integer.valueOf(42);'");
+
+        String fFinal = createFunction(KEYSPACE,
+                                       "int",
+                                       "CREATE FUNCTION %s(a int) " +
+                                       "CALLED ON NULL INPUT " +
+                                       "RETURNS text " +
+                                       "LANGUAGE java " +
+                                       "AS 'throw new RuntimeException(\"thrown to unit test - not a bug\");'");
+
+        String fFinalOK = createFunction(KEYSPACE,
+                                       "int",
+                                       "CREATE FUNCTION %s(a int) " +
+                                       "CALLED ON NULL INPUT " +
+                                       "RETURNS text " +
+                                       "LANGUAGE java " +
+                                       "AS 'return \"foobar\";'");
+
+        String a0 = createAggregate(KEYSPACE,
+                                    "int",
+                                    "CREATE AGGREGATE %s(int) " +
+                                    "SFUNC " + shortFunctionName(fState) + " " +
+                                    "STYPE int " +
+                                    "FINALFUNC " + shortFunctionName(fFinal) + " " +
+                                    "INITCOND null");
+        String a1 = createAggregate(KEYSPACE,
+                                    "int",
+                                    "CREATE AGGREGATE %s(int) " +
+                                    "SFUNC " + shortFunctionName(fStateOK) + " " +
+                                    "STYPE int " +
+                                    "FINALFUNC " + shortFunctionName(fFinal) + " " +
+                                    "INITCOND null");
+        String a2 = createAggregate(KEYSPACE,
+                                    "int",
+                                    "CREATE AGGREGATE %s(int) " +
+                                    "SFUNC " + shortFunctionName(fStateOK) + " " +
+                                    "STYPE int " +
+                                    "FINALFUNC " + shortFunctionName(fFinalOK) + " " +
+                                    "INITCOND null");
+
+        assertInvalidThrowMessage("java.lang.RuntimeException", FunctionExecutionException.class, "SELECT " + a0 + "(b) FROM %s");
+        assertInvalidThrowMessage("java.lang.RuntimeException", FunctionExecutionException.class, "SELECT " + a1 + "(b) FROM %s");
+        assertRows(execute("SELECT " + a2 + "(b) FROM %s"), row("foobar"));
+    }
+
+    @Test
+    public void testJavaAggregateWithoutStateOrFinal() throws Throwable
+    {
+        assertInvalidMessage("does not exist or is not a scalar function",
+                             "CREATE AGGREGATE " + KEYSPACE + ".jSumFooNE1(int) " +
+                             "SFUNC jSumFooNEstate " +
+                             "STYPE int");
+
+        String f = createFunction(KEYSPACE,
+                                  "int, int",
+                                  "CREATE FUNCTION %s(a int, b int) " +
+                                  "RETURNS NULL ON NULL INPUT " +
+                                  "RETURNS int " +
+                                  "LANGUAGE java " +
+                                  "AS 'return Integer.valueOf(a + b);'");
+
+        assertInvalidMessage("does not exist or is not a scalar function",
+                             "CREATE AGGREGATE " + KEYSPACE + ".jSumFooNE2(int) " +
+                             "SFUNC " + shortFunctionName(f) + " " +
+                             "STYPE int " +
+                             "FINALFUNC jSumFooNEfinal");
+
+        execute("DROP FUNCTION " + f + "(int, int)");
+    }
+
+    @Test
+    public void testJavaAggregate() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int primary key, b int)");
+        execute("INSERT INTO %s (a, b) VALUES (1, 1)");
+        execute("INSERT INTO %s (a, b) VALUES (2, 2)");
+        execute("INSERT INTO %s (a, b) VALUES (3, 3)");
+
+        String fState = createFunction(KEYSPACE,
+                                       "int, int",
+                                       "CREATE FUNCTION %s(a int, b int) " +
+                                       "CALLED ON NULL INPUT " +
+                                       "RETURNS int " +
+                                       "LANGUAGE java " +
+                                       "AS 'return Integer.valueOf((a!=null?a.intValue():0) + b.intValue());'");
+
+        String fFinal = createFunction(KEYSPACE,
+                                       "int",
+                                       "CREATE FUNCTION %s(a int) " +
+                                       "CALLED ON NULL INPUT " +
+                                       "RETURNS text " +
+                                       "LANGUAGE java " +
+                                       "AS 'return a.toString();'");
+
+        String a = createAggregate(KEYSPACE,
+                                   "int",
+                                   "CREATE AGGREGATE %s(int) " +
+                                   "SFUNC " + shortFunctionName(fState) + " " +
+                                   "STYPE int " +
+                                   "FINALFUNC " + shortFunctionName(fFinal) + " " +
+                                   "INITCOND 42");
+
+        // 42 + 1 + 2 + 3 = 48
+        assertRows(execute("SELECT " + a + "(b) FROM %s"), row("48"));
+
+        execute("DROP AGGREGATE " + a + "(int)");
+
+        execute("DROP FUNCTION " + fFinal + "(int)");
+        execute("DROP FUNCTION " + fState + "(int, int)");
+
+        assertInvalidMessage("Unknown function", "SELECT " + a + "(b) FROM %s");
+    }
+
+    @Test
+    public void testJavaAggregateSimple() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int primary key, b int)");
+        execute("INSERT INTO %s (a, b) VALUES (1, 1)");
+        execute("INSERT INTO %s (a, b) VALUES (2, 2)");
+        execute("INSERT INTO %s (a, b) VALUES (3, 3)");
+
+        String fState = createFunction(KEYSPACE,
+                                       "int, int",
+                                       "CREATE FUNCTION %s(a int, b int) " +
+                                       "CALLED ON NULL INPUT " +
+                                       "RETURNS int " +
+                                       "LANGUAGE java " +
+                                       "AS 'return Integer.valueOf((a!=null?a.intValue():0) + b.intValue());'");
+
+        String a = createAggregate(KEYSPACE,
+                                   "int, int",
+                                   "CREATE AGGREGATE %s(int) " +
+                                   "SFUNC " + shortFunctionName(fState) + " " +
+                                   "STYPE int");
+
+        // 1 + 2 + 3 = 6
+        assertRows(execute("SELECT " + a + "(b) FROM %s"), row(6));
+
+        execute("DROP AGGREGATE " + a + "(int)");
+
+        execute("DROP FUNCTION " + fState + "(int, int)");
+
+        assertInvalidMessage("Unknown function", "SELECT " + a + "(b) FROM %s");
+    }
+
+    @Test
+    public void testJavaAggregateComplex() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int primary key, b int)");
+        execute("INSERT INTO %s (a, b) VALUES (1, 1)");
+        execute("INSERT INTO %s (a, b) VALUES (2, 2)");
+        execute("INSERT INTO %s (a, b) VALUES (3, 3)");
+
+        // build an average aggregation function using
+        // tuple<bigint,int> as state
+        // double as finaltype
+
+        String fState = createFunction(KEYSPACE,
+                                       "tuple<bigint, int>, int",
+                                       "CREATE FUNCTION %s(a tuple<bigint, int>, b int) " +
+                                       "CALLED ON NULL INPUT " +
+                                       "RETURNS tuple<bigint, int> " +
+                                       "LANGUAGE java " +
+                                       "AS '" +
+                                       "a.setLong(0, a.getLong(0) + b.intValue());" +
+                                       "a.setInt(1, a.getInt(1) + 1);" +
+                                       "return a;" +
+                                       "'");
+
+        String fFinal = createFunction(KEYSPACE,
+                                       "tuple<bigint, int>",
+                                       "CREATE FUNCTION %s(a tuple<bigint, int>) " +
+                                       "RETURNS NULL ON NULL INPUT " +
+                                       "RETURNS double " +
+                                       "LANGUAGE java " +
+                                       "AS '" +
+                                       "double r = a.getLong(0);" +
+                                       "r /= a.getInt(1);" +
+                                       "return Double.valueOf(r);" +
+                                       "'");
+
+        String a = createAggregate(KEYSPACE,
+                                   "int",
+                                   "CREATE AGGREGATE %s(int) " +
+                                   "SFUNC " + shortFunctionName(fState) + " " +
+                                   "STYPE tuple<bigint, int> "+
+                                   "FINALFUNC " + shortFunctionName(fFinal) + " " +
+                                   "INITCOND (0, 0)");
+
+        // 1 + 2 + 3 = 6 / 3 = 2
+        assertRows(execute("SELECT " + a + "(b) FROM %s"), row(2d));
+
+    }
+
+    @Test
+    public void testJavascriptAggregate() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int primary key, b int)");
+        execute("INSERT INTO %s (a, b) VALUES (1, 1)");
+        execute("INSERT INTO %s (a, b) VALUES (2, 2)");
+        execute("INSERT INTO %s (a, b) VALUES (3, 3)");
+
+        String fState = createFunction(KEYSPACE,
+                                       "int, int",
+                                       "CREATE FUNCTION %s(a int, b int) " +
+                                       "RETURNS NULL ON NULL INPUT " +
+                                       "RETURNS int " +
+                                       "LANGUAGE javascript " +
+                                       "AS 'a + b;'");
+
+        String fFinal = createFunction(KEYSPACE,
+                                       "int",
+                                       "CREATE FUNCTION %s(a int) " +
+                                       "RETURNS NULL ON NULL INPUT " +
+                                       "RETURNS text " +
+                                       "LANGUAGE javascript " +
+                                       "AS '\"\"+a'");
+
+        String a = createFunction(KEYSPACE,
+                                  "int",
+                                  "CREATE AGGREGATE %s(int) " +
+                                  "SFUNC " + shortFunctionName(fState) + " " +
+                                  "STYPE int " +
+                                  "FINALFUNC " + shortFunctionName(fFinal) + " " +
+                                  "INITCOND 42");
+
+        // 42 + 1 + 2 + 3 = 48
+        assertRows(execute("SELECT " + a + "(b) FROM %s"), row("48"));
+
+        execute("DROP AGGREGATE " + a + "(int)");
+
+        execute("DROP FUNCTION " + fFinal + "(int)");
+        execute("DROP FUNCTION " + fState + "(int, int)");
+
+        assertInvalidMessage("Unknown function", "SELECT " + a + "(b) FROM %s");
+    }
+
+    @Test
+    public void testJavascriptAggregateSimple() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int primary key, b int)");
+        execute("INSERT INTO %s (a, b) VALUES (1, 1)");
+        execute("INSERT INTO %s (a, b) VALUES (2, 2)");
+        execute("INSERT INTO %s (a, b) VALUES (3, 3)");
+
+        String fState = createFunction(KEYSPACE,
+                                       "int, int",
+                                       "CREATE FUNCTION %s(a int, b int) " +
+                                       "CALLED ON NULL INPUT " +
+                                       "RETURNS int " +
+                                       "LANGUAGE javascript " +
+                                       "AS 'a + b;'");
+
+        String a = createAggregate(KEYSPACE,
+                                   "int, int",
+                                   "CREATE AGGREGATE %s(int) " +
+                                   "SFUNC " + shortFunctionName(fState) + " " +
+                                   "STYPE int ");
+
+        // 1 + 2 + 3 = 6
+        assertRows(execute("SELECT " + a + "(b) FROM %s"), row(6));
+
+        execute("DROP AGGREGATE " + a + "(int)");
+
+        execute("DROP FUNCTION " + fState + "(int, int)");
+
+        assertInvalidMessage("Unknown function", "SELECT " + a + "(b) FROM %s");
+    }
+
+    @Test
+    public void testFunctionDropPreparedStatement() throws Throwable
+    {
+        String otherKS = "cqltest_foo";
+
+        execute("CREATE KEYSPACE IF NOT EXISTS " + otherKS + " WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 3};");
+        try
+        {
+            execute("CREATE TABLE " + otherKS + ".jsdp (a int primary key, b int)");
+
+            String fState = createFunction(otherKS,
+                                           "int, int",
+                                           "CREATE FUNCTION %s(a int, b int) " +
+                                           "CALLED ON NULL INPUT " +
+                                           "RETURNS int " +
+                                           "LANGUAGE javascript " +
+                                           "AS 'a + b;'");
+
+            String a = createAggregate(otherKS,
+                                       "int",
+                                       "CREATE AGGREGATE %s(int) " +
+                                       "SFUNC " + shortFunctionName(fState) + " " +
+                                       "STYPE int");
+
+            ResultMessage.Prepared prepared = QueryProcessor.prepare("SELECT " + a + "(b) FROM " + otherKS + ".jsdp", ClientState.forInternalCalls(), false);
+            Assert.assertNotNull(QueryProcessor.instance.getPrepared(prepared.statementId));
+
+            execute("DROP AGGREGATE " + a + "(int)");
+            Assert.assertNull(QueryProcessor.instance.getPrepared(prepared.statementId));
+
+            //
+
+            execute("CREATE AGGREGATE " + a + "(int) " +
+                    "SFUNC " + shortFunctionName(fState) + " " +
+                    "STYPE int");
+
+            prepared = QueryProcessor.prepare("SELECT " + a + "(b) FROM " + otherKS + ".jsdp", ClientState.forInternalCalls(), false);
+            Assert.assertNotNull(QueryProcessor.instance.getPrepared(prepared.statementId));
+
+            execute("DROP KEYSPACE " + otherKS + ";");
+
+            Assert.assertNull(QueryProcessor.instance.getPrepared(prepared.statementId));
+        }
+        finally
+        {
+            execute("DROP KEYSPACE IF EXISTS " + otherKS + ";");
+        }
+    }
+
+    @Test
+    public void testAggregatesReferencedInAggregates() throws Throwable
+    {
+
+        String fState = createFunction(KEYSPACE,
+                                       "int, int",
+                                       "CREATE FUNCTION %s(a int, b int) " +
+                                       "CALLED ON NULL INPUT " +
+                                       "RETURNS int " +
+                                       "LANGUAGE javascript " +
+                                       "AS 'a + b;'");
+
+        String a = createAggregate(KEYSPACE,
+                                   "int",
+                                   "CREATE AGGREGATE %s(int) " +
+                                   "SFUNC " + shortFunctionName(fState) + " " +
+                                   "STYPE int ");
+
+        assertInvalidMessage("does not exist or is not a scalar function",
+                             "CREATE AGGREGATE " + KEYSPACE + ".aggInv(int) " +
+                             "SFUNC " + shortFunctionName(a) + " " +
+                             "STYPE int ");
+
+        assertInvalidMessage("does not exist or is not a scalar function",
+                             "CREATE AGGREGATE " + KEYSPACE + ".aggInv(int) " +
+                             "SFUNC " + shortFunctionName(fState) + " " +
+                             "STYPE int " +
+                             "FINALFUNC " + shortFunctionName(a));
+    }
+
+    @Test
+    public void testCalledOnNullInput() throws Throwable
+    {
+        String fStateNonNull = createFunction(KEYSPACE,
+                                              "int, int",
+                                              "CREATE OR REPLACE FUNCTION %s(state int, val int) " +
+                                              "RETURNS NULL ON NULL INPUT " +
+                                              "RETURNS int " +
+                                              "LANGUAGE java\n" +
+                                              "AS 'return Integer.valueOf(state + val);';");
+        String fStateNull = createFunction(KEYSPACE,
+                                           "int, int",
+                                           "CREATE OR REPLACE FUNCTION %s(state int, val int) " +
+                                           "CALLED ON NULL INPUT " +
+                                           "RETURNS int " +
+                                           "LANGUAGE java\n" +
+                                           "AS 'return Integer.valueOf(" +
+                                           "   (state != null ? state.intValue() : 0) " +
+                                           "   + (val != null ? val.intValue() : 0));';");
+        String fStateAlwaysNull = createFunction(KEYSPACE,
+                                           "int, int",
+                                           "CREATE OR REPLACE FUNCTION %s(state int, val int) " +
+                                           "CALLED ON NULL INPUT " +
+                                           "RETURNS int " +
+                                           "LANGUAGE java\n" +
+                                           "AS 'return null;';");
+        String fFinalNonNull = createFunction(KEYSPACE,
+                                              "int",
+                                              "CREATE OR REPLACE FUNCTION %s(state int) " +
+                                              "RETURNS NULL ON NULL INPUT " +
+                                              "RETURNS int " +
+                                              "LANGUAGE java\n" +
+                                              "AS 'return Integer.valueOf(state);';");
+        String fFinalNull = createFunction(KEYSPACE,
+                                           "int",
+                                           "CREATE OR REPLACE FUNCTION %s(state int) " +
+                                           "CALLED ON NULL INPUT " +
+                                           "RETURNS int " +
+                                           "LANGUAGE java\n" +
+                                           "AS 'return state;';");
+
+        assertInvalid("CREATE AGGREGATE " + KEYSPACE + ".invAggr(int) " +
+                      "SFUNC " + shortFunctionName(fStateNonNull) + " " +
+                      "STYPE int");
+        assertInvalid("CREATE AGGREGATE " + KEYSPACE + ".invAggr(int) " +
+                      "SFUNC " + shortFunctionName(fStateNonNull) + " " +
+                      "STYPE int " +
+                      "FINALFUNC " + shortFunctionName(fFinalNonNull));
+
+        String aStateNull = createAggregate(KEYSPACE,
+                                               "int",
+                                               "CREATE AGGREGATE %s(int) " +
+                                               "SFUNC " + shortFunctionName(fStateNull) + " " +
+                                               "STYPE int");
+        String aStateNullFinalNull = createAggregate(KEYSPACE,
+                                                        "int",
+                                                        "CREATE AGGREGATE %s(int) " +
+                                                        "SFUNC " + shortFunctionName(fStateNull) + " " +
+                                                        "STYPE int " +
+                                                        "FINALFUNC " + shortFunctionName(fFinalNull));
+        String aStateNullFinalNonNull = createAggregate(KEYSPACE,
+                                                        "int",
+                                                        "CREATE AGGREGATE %s(int) " +
+                                                        "SFUNC " + shortFunctionName(fStateNull) + " " +
+                                                        "STYPE int " +
+                                                        "FINALFUNC " + shortFunctionName(fFinalNonNull));
+        String aStateNonNull = createAggregate(KEYSPACE,
+                                               "int",
+                                               "CREATE AGGREGATE %s(int) " +
+                                               "SFUNC " + shortFunctionName(fStateNonNull) + " " +
+                                               "STYPE int " +
+                                               "INITCOND 0");
+        String aStateNonNullFinalNull = createAggregate(KEYSPACE,
+                                                        "int",
+                                                        "CREATE AGGREGATE %s(int) " +
+                                                        "SFUNC " + shortFunctionName(fStateNonNull) + " " +
+                                                        "STYPE int " +
+                                                        "FINALFUNC " + shortFunctionName(fFinalNull) + " " +
+                                                        "INITCOND 0");
+        String aStateNonNullFinalNonNull = createAggregate(KEYSPACE,
+                                                           "int",
+                                                           "CREATE AGGREGATE %s(int) " +
+                                                           "SFUNC " + shortFunctionName(fStateNonNull) + " " +
+                                                           "STYPE int " +
+                                                           "FINALFUNC " + shortFunctionName(fFinalNonNull) + " " +
+                                                           "INITCOND 0");
+        String aStateAlwaysNullFinalNull = createAggregate(KEYSPACE,
+                                                           "int",
+                                                           "CREATE AGGREGATE %s(int) " +
+                                                           "SFUNC " + shortFunctionName(fStateAlwaysNull) + " " +
+                                                           "STYPE int " +
+                                                           "FINALFUNC " + shortFunctionName(fFinalNull));
+        String aStateAlwaysNullFinalNonNull = createAggregate(KEYSPACE,
+                                                           "int",
+                                                           "CREATE AGGREGATE %s(int) " +
+                                                           "SFUNC " + shortFunctionName(fStateAlwaysNull) + " " +
+                                                           "STYPE int " +
+                                                           "FINALFUNC " + shortFunctionName(fFinalNonNull));
+
+        createTable("CREATE TABLE %s (key int PRIMARY KEY, i int)");
+
+        execute("INSERT INTO %s (key, i) VALUES (0, null)");
+        execute("INSERT INTO %s (key, i) VALUES (1, 1)");
+        execute("INSERT INTO %s (key, i) VALUES (2, 2)");
+        execute("INSERT INTO %s (key, i) VALUES (3, 3)");
+
+        assertRows(execute("SELECT " + aStateNull + "(i) FROM %s WHERE key = 0"), row(0));
+        assertRows(execute("SELECT " + aStateNullFinalNull + "(i) FROM %s WHERE key = 0"), row(0));
+        assertRows(execute("SELECT " + aStateNullFinalNonNull + "(i) FROM %s WHERE key = 0"), row(0));
+        assertRows(execute("SELECT " + aStateNonNull + "(i) FROM %s WHERE key = 0"), row(0));
+        assertRows(execute("SELECT " + aStateNonNullFinalNull + "(i) FROM %s WHERE key = 0"), row(0));
+        assertRows(execute("SELECT " + aStateNonNullFinalNonNull + "(i) FROM %s WHERE key = 0"), row(0));
+        assertRows(execute("SELECT " + aStateAlwaysNullFinalNull + "(i) FROM %s WHERE key = 0"), row(new Object[]{null}));
+        assertRows(execute("SELECT " + aStateAlwaysNullFinalNonNull + "(i) FROM %s WHERE key = 0"), row(new Object[]{null}));
+
+        assertRows(execute("SELECT " + aStateNull + "(i) FROM %s WHERE key = 1"), row(1));
+        assertRows(execute("SELECT " + aStateNullFinalNull + "(i) FROM %s WHERE key = 1"), row(1));
+        assertRows(execute("SELECT " + aStateNullFinalNonNull + "(i) FROM %s WHERE key = 1"), row(1));
+        assertRows(execute("SELECT " + aStateNonNull + "(i) FROM %s WHERE key = 1"), row(1));
+        assertRows(execute("SELECT " + aStateNonNullFinalNull + "(i) FROM %s WHERE key = 1"), row(1));
+        assertRows(execute("SELECT " + aStateNonNullFinalNonNull + "(i) FROM %s WHERE key = 1"), row(1));
+        assertRows(execute("SELECT " + aStateAlwaysNullFinalNull + "(i) FROM %s WHERE key = 1"), row(new Object[]{null}));
+        assertRows(execute("SELECT " + aStateAlwaysNullFinalNonNull + "(i) FROM %s WHERE key = 1"), row(new Object[]{null}));
+
+        assertRows(execute("SELECT " + aStateNull + "(i) FROM %s WHERE key IN (1, 2, 3)"), row(6));
+        assertRows(execute("SELECT " + aStateNullFinalNull + "(i) FROM %s WHERE key IN (1, 2, 3)"), row(6));
+        assertRows(execute("SELECT " + aStateNullFinalNonNull + "(i) FROM %s WHERE key IN (1, 2, 3)"), row(6));
+        assertRows(execute("SELECT " + aStateNonNull + "(i) FROM %s WHERE key IN (1, 2, 3)"), row(6));
+        assertRows(execute("SELECT " + aStateNonNullFinalNull + "(i) FROM %s WHERE key IN (1, 2, 3)"), row(6));
+        assertRows(execute("SELECT " + aStateNonNullFinalNonNull + "(i) FROM %s WHERE key IN (1, 2, 3)"), row(6));
+        assertRows(execute("SELECT " + aStateAlwaysNullFinalNull + "(i) FROM %s WHERE key IN (1, 2, 3)"), row(new Object[]{null}));
+        assertRows(execute("SELECT " + aStateAlwaysNullFinalNonNull + "(i) FROM %s WHERE key IN (1, 2, 3)"), row(new Object[]{null}));
+    }
+
+    @Test
+    public void testBrokenAggregate() throws Throwable
+    {
+        createTable("CREATE TABLE %s (key int primary key, val int)");
+        execute("INSERT INTO %s (key, val) VALUES (?, ?)", 1, 1);
+
+        String fState = createFunction(KEYSPACE,
+                                       "int, int",
+                                       "CREATE FUNCTION %s(a int, b int) " +
+                                       "CALLED ON NULL INPUT " +
+                                       "RETURNS int " +
+                                       "LANGUAGE javascript " +
+                                       "AS 'a + b;'");
+
+        String a = createAggregate(KEYSPACE,
+                                   "int",
+                                   "CREATE AGGREGATE %s(int) " +
+                                   "SFUNC " + shortFunctionName(fState) + " " +
+                                   "STYPE int ");
+
+        UDAggregate f = (UDAggregate) Functions.find(parseFunctionName(a)).get(0);
+
+        Functions.addOrReplaceFunction(UDAggregate.createBroken(f.name(), f.argTypes(), f.returnType(),
+                                                                null, new InvalidRequestException("foo bar is broken")));
+
+        assertInvalidThrowMessage("foo bar is broken", InvalidRequestException.class,
+                                  "SELECT " + a + "(val) FROM %s");
+    }
+
+    @Test
+    public void testWrongStateType() throws Throwable
+    {
+        createTable("CREATE TABLE %s (key int primary key, val int)");
+        execute("INSERT INTO %s (key, val) VALUES (?, ?)", 1, 1);
+
+        String fState = createFunction(KEYSPACE,
+                                       "int, int",
+                                       "CREATE FUNCTION %s(a int, b int) " +
+                                       "CALLED ON NULL INPUT " +
+                                       "RETURNS double " +
+                                       "LANGUAGE java " +
+                                       "AS 'return Double.valueOf(1.0);'");
+
+        String fFinal = createFunction(KEYSPACE,
+                                       "int",
+                                       "CREATE FUNCTION %s(a int) " +
+                                       "CALLED ON NULL INPUT " +
+                                       "RETURNS int " +
+                                       "LANGUAGE java " +
+                                       "AS 'return Integer.valueOf(1);';");
+
+        assertInvalidMessage("return type must be the same as the first argument type - check STYPE, argument and return types",
+                             "CREATE AGGREGATE %s(int) " +
+                             "SFUNC " + shortFunctionName(fState) + ' ' +
+                             "STYPE int " +
+                             "FINALFUNC " + shortFunctionName(fFinal) + ' ' +
+                             "INITCOND 1");
+    }
+
+    @Test
+    public void testWrongKeyspace() throws Throwable
+    {
+        String typeName = createType("CREATE TYPE %s (txt text, i int)");
+        String type = KEYSPACE + '.' + typeName;
+
+        String fState = createFunction(KEYSPACE_PER_TEST,
+                                       "int, int",
+                                       "CREATE FUNCTION %s(a int, b int) " +
+                                       "CALLED ON NULL INPUT " +
+                                       "RETURNS double " +
+                                       "LANGUAGE java " +
+                                       "AS 'return Double.valueOf(1.0);'");
+
+        String fFinal = createFunction(KEYSPACE_PER_TEST,
+                                       "int",
+                                       "CREATE FUNCTION %s(a int) " +
+                                       "CALLED ON NULL INPUT " +
+                                       "RETURNS int " +
+                                       "LANGUAGE java " +
+                                       "AS 'return Integer.valueOf(1);';");
+
+        String fStateWrong = createFunction(KEYSPACE,
+                                       "int, int",
+                                       "CREATE FUNCTION %s(a int, b int) " +
+                                       "CALLED ON NULL INPUT " +
+                                       "RETURNS double " +
+                                       "LANGUAGE java " +
+                                       "AS 'return Double.valueOf(1.0);'");
+
+        String fFinalWrong = createFunction(KEYSPACE,
+                                       "int",
+                                       "CREATE FUNCTION %s(a int) " +
+                                       "CALLED ON NULL INPUT " +
+                                       "RETURNS int " +
+                                       "LANGUAGE java " +
+                                       "AS 'return Integer.valueOf(1);';");
+
+        assertInvalidMessage(String.format("Statement on keyspace %s cannot refer to a user type in keyspace %s; user types can only be used in the keyspace they are defined in",
+                                           KEYSPACE_PER_TEST, KEYSPACE),
+                             "CREATE AGGREGATE " + KEYSPACE_PER_TEST + ".test_wrong_ks(int) " +
+                             "SFUNC " + shortFunctionName(fState) + ' ' +
+                             "STYPE " + type + " " +
+                             "FINALFUNC " + shortFunctionName(fFinal) + ' ' +
+                             "INITCOND 1");
+
+        assertInvalidMessage(String.format("Statement on keyspace %s cannot refer to a user type in keyspace %s; user types can only be used in the keyspace they are defined in",
+                                           KEYSPACE_PER_TEST, KEYSPACE),
+                             "CREATE AGGREGATE " + KEYSPACE_PER_TEST + ".test_wrong_ks(int) " +
+                             "SFUNC " + fStateWrong + ' ' +
+                             "STYPE " + type + " " +
+                             "FINALFUNC " + shortFunctionName(fFinal) + ' ' +
+                             "INITCOND 1");
+
+        assertInvalidMessage(String.format("Statement on keyspace %s cannot refer to a user type in keyspace %s; user types can only be used in the keyspace they are defined in",
+                                           KEYSPACE_PER_TEST, KEYSPACE),
+                             "CREATE AGGREGATE " + KEYSPACE_PER_TEST + ".test_wrong_ks(int) " +
+                             "SFUNC " + shortFunctionName(fState) + ' ' +
+                             "STYPE " + type + " " +
+                             "FINALFUNC " + fFinalWrong + ' ' +
+                             "INITCOND 1");
+    }
+
+    @Test
+    public void testSystemKeyspace() throws Throwable
+    {
+        String fState = createFunction(KEYSPACE,
+                                       "text, text",
+                                       "CREATE FUNCTION %s(a text, b text) " +
+                                       "CALLED ON NULL INPUT " +
+                                       "RETURNS text " +
+                                       "LANGUAGE java " +
+                                       "AS 'return \"foobar\";'");
+
+        createAggregate(KEYSPACE,
+                        "text",
+                        "CREATE AGGREGATE %s(text) " +
+                        "SFUNC " + shortFunctionName(fState) + ' ' +
+                        "STYPE text " +
+                        "FINALFUNC system.varcharasblob " +
+                        "INITCOND 'foobar'");
+    }
+
+    @Test
+    public void testFunctionWithFrozenSetType() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int PRIMARY KEY, b frozen<set<int>>)");
+        createIndex("CREATE INDEX ON %s (FULL(b))");
+
+        execute("INSERT INTO %s (a, b) VALUES (?, ?)", 0, set());
+        execute("INSERT INTO %s (a, b) VALUES (?, ?)", 1, set(1, 2, 3));
+        execute("INSERT INTO %s (a, b) VALUES (?, ?)", 2, set(4, 5, 6));
+        execute("INSERT INTO %s (a, b) VALUES (?, ?)", 3, set(7, 8, 9));
+
+        String fState = createFunction(KEYSPACE,
+                                       "set<int>",
+                                       "CREATE FUNCTION %s (state set<int>, values set<int>) " +
+                                       "CALLED ON NULL INPUT " +
+                                       "RETURNS set<int> " +
+                                       "LANGUAGE java\n" +
+                                       "AS 'return values;';");
+
+        String fFinal = createFunction(KEYSPACE,
+                                       "set<int>",
+                                       "CREATE FUNCTION %s(state set<int>) " +
+                                       "CALLED ON NULL INPUT " +
+                                       "RETURNS set<int> " +
+                                       "LANGUAGE java " +
+                                       "AS 'return state;'");
+
+        assertInvalidMessage("The function state type should not be frozen",
+                             "CREATE AGGREGATE %s(set<int>) " +
+                             "SFUNC " + fState + " " +
+                             "STYPE frozen<set<int>> " +
+                             "FINALFUNC " + fFinal + " " +
+                             "INITCOND null");
+
+        String aggregation = createAggregate(KEYSPACE,
+                                             "set<int>",
+                                             "CREATE AGGREGATE %s(set<int>) " +
+                                             "SFUNC " + fState + " " +
+                                             "STYPE set<int> " +
+                                             "FINALFUNC " + fFinal + " " +
+                                             "INITCOND null");
+
+        assertRows(execute("SELECT " + aggregation + "(b) FROM %s"),
+                   row(set(7, 8, 9)));
+
+        assertInvalidMessage("The function arguments should not be frozen",
+                             "DROP AGGREGATE %s (frozen<set<int>>);");
+    }
+
+    @Test
+    public void testFunctionWithFrozenListType() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int PRIMARY KEY, b frozen<list<int>>)");
+        createIndex("CREATE INDEX ON %s (FULL(b))");
+
+        execute("INSERT INTO %s (a, b) VALUES (?, ?)", 0, list());
+        execute("INSERT INTO %s (a, b) VALUES (?, ?)", 1, list(1, 2, 3));
+        execute("INSERT INTO %s (a, b) VALUES (?, ?)", 2, list(4, 5, 6));
+        execute("INSERT INTO %s (a, b) VALUES (?, ?)", 3, list(7, 8, 9));
+
+        String fState = createFunction(KEYSPACE,
+                                       "list<int>",
+                                       "CREATE FUNCTION %s (state list<int>, values list<int>) " +
+                                       "CALLED ON NULL INPUT " +
+                                       "RETURNS list<int> " +
+                                       "LANGUAGE java\n" +
+                                       "AS 'return values;';");
+
+        String fFinal = createFunction(KEYSPACE,
+                                       "list<int>",
+                                       "CREATE FUNCTION %s(state list<int>) " +
+                                       "CALLED ON NULL INPUT " +
+                                       "RETURNS list<int> " +
+                                       "LANGUAGE java " +
+                                       "AS 'return state;'");
+
+        assertInvalidMessage("The function state type should not be frozen",
+                             "CREATE AGGREGATE %s(list<int>) " +
+                             "SFUNC " + fState + " " +
+                             "STYPE frozen<list<int>> " +
+                             "FINALFUNC " + fFinal + " " +
+                             "INITCOND null");
+
+        String aggregation = createAggregate(KEYSPACE,
+                                             "list<int>",
+                                             "CREATE AGGREGATE %s(list<int>) " +
+                                             "SFUNC " + fState + " " +
+                                             "STYPE list<int> " +
+                                             "FINALFUNC " + fFinal + " " +
+                                             "INITCOND null");
+
+        assertRows(execute("SELECT " + aggregation + "(b) FROM %s"),
+                   row(list(7, 8, 9)));
+
+        assertInvalidMessage("The function arguments should not be frozen",
+                             "DROP AGGREGATE %s (frozen<list<int>>);");
+    }
+
+    @Test
+    public void testFunctionWithFrozenMapType() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int PRIMARY KEY, b frozen<map<int, int>>)");
+        createIndex("CREATE INDEX ON %s (FULL(b))");
+
+        execute("INSERT INTO %s (a, b) VALUES (?, ?)", 0, map());
+        execute("INSERT INTO %s (a, b) VALUES (?, ?)", 1, map(1, 2, 3, 4));
+        execute("INSERT INTO %s (a, b) VALUES (?, ?)", 2, map(4, 5, 6, 7));
+        execute("INSERT INTO %s (a, b) VALUES (?, ?)", 3, map(7, 8, 9, 10));
+
+        String fState = createFunction(KEYSPACE,
+                                       "map<int, int>",
+                                       "CREATE FUNCTION %s (state map<int, int>, values map<int, int>) " +
+                                       "CALLED ON NULL INPUT " +
+                                       "RETURNS map<int, int> " +
+                                       "LANGUAGE java\n" +
+                                       "AS 'return values;';");
+
+        String fFinal = createFunction(KEYSPACE,
+                                       "map<int, int>",
+                                       "CREATE FUNCTION %s(state map<int, int>) " +
+                                       "CALLED ON NULL INPUT " +
+                                       "RETURNS map<int, int> " +
+                                       "LANGUAGE java " +
+                                       "AS 'return state;'");
+
+        assertInvalidMessage("The function state type should not be frozen",
+                             "CREATE AGGREGATE %s(map<int, int>) " +
+                             "SFUNC " + fState + " " +
+                             "STYPE frozen<map<int, int>> " +
+                             "FINALFUNC " + fFinal + " " +
+                             "INITCOND null");
+
+        String aggregation = createAggregate(KEYSPACE,
+                                             "map<int, int>",
+                                             "CREATE AGGREGATE %s(map<int, int>) " +
+                                             "SFUNC " + fState + " " +
+                                             "STYPE map<int, int> " +
+                                             "FINALFUNC " + fFinal + " " +
+                                             "INITCOND null");
+
+        assertRows(execute("SELECT " + aggregation + "(b) FROM %s"),
+                   row(map(7, 8, 9, 10)));
+
+        assertInvalidMessage("The function arguments should not be frozen",
+                             "DROP AGGREGATE %s (frozen<map<int, int>>);");
+    }
+
+    @Test
+    public void testFunctionWithFrozenTupleType() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int PRIMARY KEY, b frozen<tuple<int, int>>)");
+        createIndex("CREATE INDEX ON %s (b)");
+
+        execute("INSERT INTO %s (a, b) VALUES (?, ?)", 0, tuple());
+        execute("INSERT INTO %s (a, b) VALUES (?, ?)", 1, tuple(1, 2));
+        execute("INSERT INTO %s (a, b) VALUES (?, ?)", 2, tuple(4, 5));
+        execute("INSERT INTO %s (a, b) VALUES (?, ?)", 3, tuple(7, 8));
+
+        String fState = createFunction(KEYSPACE,
+                                       "tuple<int, int>",
+                                       "CREATE FUNCTION %s (state tuple<int, int>, values tuple<int, int>) " +
+                                       "CALLED ON NULL INPUT " +
+                                       "RETURNS tuple<int, int> " +
+                                       "LANGUAGE java\n" +
+                                       "AS 'return values;';");
+
+        String fFinal = createFunction(KEYSPACE,
+                                       "tuple<int, int>",
+                                       "CREATE FUNCTION %s(state tuple<int, int>) " +
+                                       "CALLED ON NULL INPUT " +
+                                       "RETURNS tuple<int, int> " +
+                                       "LANGUAGE java " +
+                                       "AS 'return state;'");
+
+        assertInvalidMessage("The function state type should not be frozen",
+                             "CREATE AGGREGATE %s(tuple<int, int>) " +
+                             "SFUNC " + fState + " " +
+                             "STYPE frozen<tuple<int, int>> " +
+                             "FINALFUNC " + fFinal + " " +
+                             "INITCOND null");
+
+        String aggregation = createAggregate(KEYSPACE,
+                                             "tuple<int, int>",
+                                             "CREATE AGGREGATE %s(tuple<int, int>) " +
+                                             "SFUNC " + fState + " " +
+                                             "STYPE tuple<int, int> " +
+                                             "FINALFUNC " + fFinal + " " +
+                                             "INITCOND null");
+
+        assertRows(execute("SELECT " + aggregation + "(b) FROM %s"),
+                   row(tuple(7, 8)));
+
+        assertInvalidMessage("The function arguments should not be frozen",
+                             "DROP AGGREGATE %s (frozen<tuple<int, int>>);");
+    }
+
+    @Test
+    public void testFunctionWithFrozenUDFType() throws Throwable
+    {
+        String myType = createType("CREATE TYPE %s (f int)");
+        createTable("CREATE TABLE %s (a int PRIMARY KEY, b frozen<" + myType + ">)");
+        createIndex("CREATE INDEX ON %s (b)");
+
+        execute("INSERT INTO %s (a, b) VALUES (?, {f : ?})", 0, 1);
+        execute("INSERT INTO %s (a, b) VALUES (?, {f : ?})", 1, 2);
+        execute("INSERT INTO %s (a, b) VALUES (?, {f : ?})", 2, 4);
+        execute("INSERT INTO %s (a, b) VALUES (?, {f : ?})", 3, 7);
+
+        String fState = createFunction(KEYSPACE,
+                                       "tuple<int, int>",
+                                       "CREATE FUNCTION %s (state " + myType + ", values " + myType + ") " +
+                                       "CALLED ON NULL INPUT " +
+                                       "RETURNS " + myType + " " +
+                                       "LANGUAGE java\n" +
+                                       "AS 'return values;';");
+
+        String fFinal = createFunction(KEYSPACE,
+                                       myType,
+                                       "CREATE FUNCTION %s(state " + myType + ") " +
+                                       "CALLED ON NULL INPUT " +
+                                       "RETURNS " + myType + " " +
+                                       "LANGUAGE java " +
+                                       "AS 'return state;'");
+
+        assertInvalidMessage("The function state type should not be frozen",
+                             "CREATE AGGREGATE %s(" + myType + ") " +
+                             "SFUNC " + fState + " " +
+                             "STYPE frozen<" + myType + "> " +
+                             "FINALFUNC " + fFinal + " " +
+                             "INITCOND null");
+
+        String aggregation = createAggregate(KEYSPACE,
+                                             myType,
+                                             "CREATE AGGREGATE %s(" + myType + ") " +
+                                             "SFUNC " + fState + " " +
+                                             "STYPE " + myType + " " +
+                                             "FINALFUNC " + fFinal + " " +
+                                             "INITCOND null");
+
+        assertRows(execute("SELECT " + aggregation + "(b).f FROM %s"),
+                   row(7));
+
+        assertInvalidMessage("The function arguments should not be frozen",
+                             "DROP AGGREGATE %s (frozen<" + myType + ">);");
+    }
+
+    @Test
+    public void testEmptyValues() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int primary key, b text)");
+        execute("INSERT INTO %s (a, b) VALUES (1, '')");
+        execute("INSERT INTO %s (a, b) VALUES (2, '')");
+        execute("INSERT INTO %s (a, b) VALUES (3, '')");
+
+        String fCON = createFunction(KEYSPACE,
+                                     "text, text",
+                                     "CREATE FUNCTION %s(a text, b text) " +
+                                     "CALLED ON NULL INPUT " +
+                                     "RETURNS text " +
+                                     "LANGUAGE java " +
+                                     "AS 'return a + \"x\" + b + \"y\";'");
+
+        String fCONf = createFunction(KEYSPACE,
+                                     "text",
+                                     "CREATE FUNCTION %s(a text) " +
+                                     "CALLED ON NULL INPUT " +
+                                     "RETURNS text " +
+                                     "LANGUAGE java " +
+                                     "AS 'return \"fin\" + a;'");
+
+        String aCON = createAggregate(KEYSPACE,
+                                      "text",
+                                      "CREATE AGGREGATE %s(text) " +
+                                      "SFUNC " + shortFunctionName(fCON) + ' ' +
+                                      "STYPE text " +
+                                      "FINALFUNC " + shortFunctionName(fCONf) + ' ' +
+                                      "INITCOND ''");
+
+        String fRNON = createFunction(KEYSPACE,
+                                      "text",
+                                      "CREATE FUNCTION %s(a text, b text) " +
+                                      "RETURNS NULL ON NULL INPUT " +
+                                      "RETURNS text " +
+                                      "LANGUAGE java " +
+                                      "AS 'return a + \"x\" + b + \"y\";'");
+
+        String fRNONf = createFunction(KEYSPACE,
+                                      "text",
+                                      "CREATE FUNCTION %s(a text) " +
+                                      "RETURNS NULL ON NULL INPUT " +
+                                      "RETURNS text " +
+                                      "LANGUAGE java " +
+                                      "AS 'return \"fin\" + a;'");
+
+        String aRNON = createAggregate(KEYSPACE,
+                                      "int",
+                                      "CREATE AGGREGATE %s(text) " +
+                                      "SFUNC " + shortFunctionName(fRNON) + ' ' +
+                                      "STYPE text " +
+                                      "FINALFUNC " + shortFunctionName(fRNONf) + ' ' +
+                                      "INITCOND ''");
+
+        assertRows(execute("SELECT " + aCON + "(b) FROM %s"), row("finxyxyxy"));
+        assertRows(execute("SELECT " + aRNON + "(b) FROM %s"), row("finxyxyxy"));
+
+        createTable("CREATE TABLE %s (a int primary key, b text)");
+        execute("INSERT INTO %s (a, b) VALUES (1, null)");
+        execute("INSERT INTO %s (a, b) VALUES (2, null)");
+        execute("INSERT INTO %s (a, b) VALUES (3, null)");
+
+        assertRows(execute("SELECT " + aCON + "(b) FROM %s"), row("finxnullyxnullyxnully"));
+        assertRows(execute("SELECT " + aRNON + "(b) FROM %s"), row("fin"));
+
+    }
+
+    @Test
+    public void testSystemKsFuncs() throws Throwable
+    {
+
+        String fAdder = createFunction(KEYSPACE,
+                                      "int, int",
+                                      "CREATE FUNCTION %s(a int, b int) " +
+                                      "CALLED ON NULL INPUT " +
+                                      "RETURNS int " +
+                                      "LANGUAGE java " +
+                                      "AS 'return (a != null ? a : 0) + (b != null ? b : 0);'");
+
+        String aAggr = createAggregate(KEYSPACE,
+                                      "int",
+                                      "CREATE AGGREGATE %s(int) " +
+                                      "SFUNC " + shortFunctionName(fAdder) + ' ' +
+                                      "STYPE int " +
+                                      "FINALFUNC intasblob");
+
+        createTable("CREATE TABLE %s (a int primary key, b int)");
+        execute("INSERT INTO %s (a, b) VALUES (1, 1)");
+        execute("INSERT INTO %s (a, b) VALUES (2, 2)");
+        execute("INSERT INTO %s (a, b) VALUES (3, 3)");
+
+        assertRows(execute("SELECT " + aAggr + "(b) FROM %s"), row(Int32Serializer.instance.serialize(6)));
+
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/01115f72/test/unit/org/apache/cassandra/cql3/validation/operations/AlterTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/AlterTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/AlterTest.java
new file mode 100644
index 0000000..95380f4
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/AlterTest.java
@@ -0,0 +1,203 @@
+/*
+ * 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.validation.operations;
+
+import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.exceptions.SyntaxException;
+
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+
+public class AlterTest extends CQLTester
+{
+    @Test
+    public void testAddList() throws Throwable
+    {
+        createTable("CREATE TABLE %s (id text PRIMARY KEY, content text);");
+        execute("ALTER TABLE %s ADD myCollection list<text>;");
+        execute("INSERT INTO %s (id, content , myCollection) VALUES ('test', 'first test', ['first element']);");
+
+        assertRows(execute("SELECT * FROM %s;"), row("test", "first test", list("first element")));
+    }
+
+    @Test
+    public void testDropList() throws Throwable
+    {
+        createTable("CREATE TABLE %s (id text PRIMARY KEY, content text, myCollection list<text>);");
+        execute("INSERT INTO %s (id, content , myCollection) VALUES ('test', 'first test', ['first element']);");
+        execute("ALTER TABLE %s DROP myCollection;");
+
+        assertRows(execute("SELECT * FROM %s;"), row("test", "first test"));
+    }
+    @Test
+    public void testAddMap() throws Throwable
+    {
+        createTable("CREATE TABLE %s (id text PRIMARY KEY, content text);");
+        execute("ALTER TABLE %s ADD myCollection map<text, text>;");
+        execute("INSERT INTO %s (id, content , myCollection) VALUES ('test', 'first test', { '1' : 'first element'});");
+
+        assertRows(execute("SELECT * FROM %s;"), row("test", "first test", map("1", "first element")));
+    }
+
+    @Test
+    public void testDropMap() throws Throwable
+    {
+        createTable("CREATE TABLE %s (id text PRIMARY KEY, content text, myCollection map<text, text>);");
+        execute("INSERT INTO %s (id, content , myCollection) VALUES ('test', 'first test', { '1' : 'first element'});");
+        execute("ALTER TABLE %s DROP myCollection;");
+
+        assertRows(execute("SELECT * FROM %s;"), row("test", "first test"));
+    }
+
+    @Test
+    public void testDropListAndAddListWithSameName() throws Throwable
+    {
+        createTable("CREATE TABLE %s (id text PRIMARY KEY, content text, myCollection list<text>);");
+        execute("INSERT INTO %s (id, content , myCollection) VALUES ('test', 'first test', ['first element']);");
+        execute("ALTER TABLE %s DROP myCollection;");
+        execute("ALTER TABLE %s ADD myCollection list<text>;");
+
+        assertRows(execute("SELECT * FROM %s;"), row("test", "first test", null));
+        execute("UPDATE %s set myCollection = ['second element'] WHERE id = 'test';");
+        assertRows(execute("SELECT * FROM %s;"), row("test", "first test", list("second element")));
+    }
+    @Test
+    public void testDropListAndAddMapWithSameName() throws Throwable
+    {
+        createTable("CREATE TABLE %s (id text PRIMARY KEY, content text, myCollection list<text>);");
+        execute("INSERT INTO %s (id, content , myCollection) VALUES ('test', 'first test', ['first element']);");
+        execute("ALTER TABLE %s DROP myCollection;");
+
+        assertInvalid("ALTER TABLE %s ADD myCollection map<int, int>;");
+    }
+
+    @Test
+    public void testChangeStrategyWithUnquotedAgrument() throws Throwable
+    {
+        createTable("CREATE TABLE %s (id text PRIMARY KEY);");
+
+        assertInvalidSyntaxMessage("no viable alternative at input '}'",
+                                   "ALTER TABLE %s WITH caching = {'keys' : 'all', 'rows_per_partition' : ALL};");
+    }
+
+    @Test
+    // tests CASSANDRA-7976
+    public void testAlterIndexInterval() throws Throwable
+    {
+        String tableName = createTable("CREATE TABLE IF NOT EXISTS %s (id uuid, album text, artist text, data blob, PRIMARY KEY (id))");
+        ColumnFamilyStore cfs = Keyspace.open(KEYSPACE).getColumnFamilyStore(tableName);
+
+        alterTable("ALTER TABLE %s WITH min_index_interval=256 AND max_index_interval=512");
+        assertEquals(256, cfs.metadata.getMinIndexInterval());
+        assertEquals(512, cfs.metadata.getMaxIndexInterval());
+
+        alterTable("ALTER TABLE %s WITH caching = 'none'");
+        assertEquals(256, cfs.metadata.getMinIndexInterval());
+        assertEquals(512, cfs.metadata.getMaxIndexInterval());
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.create_alter_options_test()
+     */
+    @Test
+    public void testCreateAlterKeyspaces() throws Throwable
+    {
+        assertInvalidThrow(SyntaxException.class, "CREATE KEYSPACE ks1");
+        assertInvalidThrow(ConfigurationException.class, "CREATE KEYSPACE ks1 WITH replication= { 'replication_factor' : 1 }");
+
+        execute("CREATE KEYSPACE ks1 WITH replication={ 'class' : 'SimpleStrategy', 'replication_factor' : 1 }");
+        execute("CREATE KEYSPACE ks2 WITH replication={ 'class' : 'SimpleStrategy', 'replication_factor' : 1 } AND durable_writes=false");
+
+        assertRows(execute("SELECT keyspace_name, durable_writes FROM system.schema_keyspaces"),
+                   row("ks1", true),
+                   row(KEYSPACE, true),
+                   row(KEYSPACE_PER_TEST, true),
+                   row("ks2", false));
+
+        execute("ALTER KEYSPACE ks1 WITH replication = { 'class' : 'NetworkTopologyStrategy', 'dc1' : 1 } AND durable_writes=False");
+        execute("ALTER KEYSPACE ks2 WITH durable_writes=true");
+
+        assertRows(execute("SELECT keyspace_name, durable_writes, strategy_class FROM system.schema_keyspaces"),
+                   row("ks1", false, "org.apache.cassandra.locator.NetworkTopologyStrategy"),
+                   row(KEYSPACE, true, "org.apache.cassandra.locator.SimpleStrategy"),
+                   row(KEYSPACE_PER_TEST, true, "org.apache.cassandra.locator.SimpleStrategy"),
+                   row("ks2", true, "org.apache.cassandra.locator.SimpleStrategy"));
+
+        execute("USE ks1");
+
+        assertInvalidThrow(ConfigurationException.class, "CREATE TABLE cf1 (a int PRIMARY KEY, b int) WITH compaction = { 'min_threshold' : 4 }");
+
+        execute("CREATE TABLE cf1 (a int PRIMARY KEY, b int) WITH compaction = { 'class' : 'SizeTieredCompactionStrategy', 'min_threshold' : 7 }");
+        assertRows(execute("SELECT columnfamily_name, min_compaction_threshold FROM system.schema_columnfamilies WHERE keyspace_name='ks1'"),
+                   row("cf1", 7));
+
+        // clean-up
+        execute("DROP KEYSPACE ks1");
+        execute("DROP KEYSPACE ks2");
+    }
+
+    /**
+     * Test for bug of 5232,
+     * migrated from cql_tests.py:TestCQL.alter_bug_test()
+     */
+    @Test
+    public void testAlterStatementWithAdd() throws Throwable
+    {
+        createTable("CREATE TABLE %s (id int PRIMARY KEY, t text)");
+
+        execute("UPDATE %s SET t = '111' WHERE id = 1");
+
+        execute("ALTER TABLE %s ADD l list<text>");
+        assertRows(execute("SELECT * FROM %s"),
+                   row(1, null, "111"));
+
+        execute("ALTER TABLE %s ADD m map<int, text>");
+        assertRows(execute("SELECT * FROM %s"),
+                   row(1, null, null, "111"));
+    }
+
+    /**
+     * Test for 7744,
+     * migrated from cql_tests.py:TestCQL.downgrade_to_compact_bug_test()
+     */
+    @Test
+    public void testDowngradeToCompact() throws Throwable
+    {
+        createTable("create table %s (k int primary key, v set<text>)");
+        execute("insert into %s (k, v) VALUES (0, {'f'})");
+        flush();
+        execute("alter table %s drop v");
+        execute("alter table %s add v int");
+    }
+
+    @Test
+    // tests CASSANDRA-9565
+    public void testDoubleWith() throws Throwable
+    {
+        String[] stmts = new String[] { "ALTER KEYSPACE WITH WITH DURABLE_WRITES = true",
+                                        "ALTER KEYSPACE ks WITH WITH DURABLE_WRITES = true" };
+
+        for (String stmt : stmts) {
+            assertInvalidSyntaxMessage("no viable alternative at input 'WITH'", stmt);
+        }
+    }
+}


[24/32] cassandra git commit: 2.2 commit for CASSANDRA-9160

Posted by jm...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/01115f72/test/unit/org/apache/cassandra/cql3/UFAuthTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/UFAuthTest.java b/test/unit/org/apache/cassandra/cql3/UFAuthTest.java
deleted file mode 100644
index 2c36bd1..0000000
--- a/test/unit/org/apache/cassandra/cql3/UFAuthTest.java
+++ /dev/null
@@ -1,724 +0,0 @@
-/*
- * 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;
-
-import java.lang.reflect.Field;
-import java.util.*;
-
-import com.google.common.base.Joiner;
-import com.google.common.collect.ImmutableSet;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.cassandra.auth.*;
-import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.cql3.functions.Function;
-import org.apache.cassandra.cql3.functions.FunctionName;
-import org.apache.cassandra.cql3.functions.Functions;
-import org.apache.cassandra.cql3.statements.BatchStatement;
-import org.apache.cassandra.cql3.statements.ModificationStatement;
-import org.apache.cassandra.exceptions.*;
-import org.apache.cassandra.service.ClientState;
-import org.apache.cassandra.utils.Pair;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-public class UFAuthTest extends CQLTester
-{
-    private static final Logger logger = LoggerFactory.getLogger(UFAuthTest.class);
-
-    String roleName = "test_role";
-    AuthenticatedUser user;
-    RoleResource role;
-    ClientState clientState;
-
-    @BeforeClass
-    public static void setupAuthorizer()
-    {
-        try
-        {
-            IAuthorizer authorizer = new StubAuthorizer();
-            Field authorizerField = DatabaseDescriptor.class.getDeclaredField("authorizer");
-            authorizerField.setAccessible(true);
-            authorizerField.set(null, authorizer);
-            DatabaseDescriptor.setPermissionsValidity(0);
-        }
-        catch (IllegalAccessException | NoSuchFieldException e)
-        {
-            throw new RuntimeException(e);
-        }
-    }
-
-    @Before
-    public void setup() throws Throwable
-    {
-        ((StubAuthorizer) DatabaseDescriptor.getAuthorizer()).clear();
-        setupClientState();
-        setupTable("CREATE TABLE %s (k int, v1 int, v2 int, PRIMARY KEY (k, v1))");
-    }
-
-    @Test
-    public void functionInSelection() throws Throwable
-    {
-        String functionName = createSimpleFunction();
-        String cql = String.format("SELECT k, %s FROM %s WHERE k = 1;",
-                                   functionCall(functionName),
-                                   KEYSPACE + "." + currentTable());
-        assertPermissionsOnFunction(cql, functionName);
-    }
-
-    @Test
-    public void functionInSelectPKRestriction() throws Throwable
-    {
-        String functionName = createSimpleFunction();
-        String cql = String.format("SELECT * FROM %s WHERE k = %s",
-                                   KEYSPACE + "." + currentTable(),
-                                   functionCall(functionName));
-        assertPermissionsOnFunction(cql, functionName);
-    }
-
-    @Test
-    public void functionInSelectClusteringRestriction() throws Throwable
-    {
-        String functionName = createSimpleFunction();
-        String cql = String.format("SELECT * FROM %s WHERE k = 0 AND v1 = %s",
-                                   KEYSPACE + "." + currentTable(),
-                                   functionCall(functionName));
-        assertPermissionsOnFunction(cql, functionName);
-    }
-
-    @Test
-    public void functionInSelectInRestriction() throws Throwable
-    {
-        String functionName = createSimpleFunction();
-        String cql = String.format("SELECT * FROM %s WHERE k IN (%s, %s)",
-                                   KEYSPACE + "." + currentTable(),
-                                   functionCall(functionName),
-                                   functionCall(functionName));
-        assertPermissionsOnFunction(cql, functionName);
-    }
-
-    @Test
-    public void functionInSelectMultiColumnInRestriction() throws Throwable
-    {
-        setupTable("CREATE TABLE %s (k int, v1 int, v2 int, v3 int, PRIMARY KEY (k, v1, v2))");
-        String functionName = createSimpleFunction();
-        String cql = String.format("SELECT * FROM %s WHERE k=0 AND (v1, v2) IN ((%s, %s))",
-                                   KEYSPACE + "." + currentTable(),
-                                   functionCall(functionName),
-                                   functionCall(functionName));
-        assertPermissionsOnFunction(cql, functionName);
-    }
-
-    @Test
-    public void functionInSelectMultiColumnEQRestriction() throws Throwable
-    {
-        setupTable("CREATE TABLE %s (k int, v1 int, v2 int, v3 int, PRIMARY KEY (k, v1, v2))");
-        String functionName = createSimpleFunction();
-        String cql = String.format("SELECT * FROM %s WHERE k=0 AND (v1, v2) = (%s, %s)",
-                                   KEYSPACE + "." + currentTable(),
-                                   functionCall(functionName),
-                                   functionCall(functionName));
-        assertPermissionsOnFunction(cql, functionName);
-    }
-
-    @Test
-    public void functionInSelectMultiColumnSliceRestriction() throws Throwable
-    {
-        setupTable("CREATE TABLE %s (k int, v1 int, v2 int, v3 int, PRIMARY KEY (k, v1, v2))");
-        String functionName = createSimpleFunction();
-        String cql = String.format("SELECT * FROM %s WHERE k=0 AND (v1, v2) < (%s, %s)",
-                                   KEYSPACE + "." + currentTable(),
-                                   functionCall(functionName),
-                                   functionCall(functionName));
-        assertPermissionsOnFunction(cql, functionName);
-    }
-
-    @Test
-    public void functionInSelectTokenEQRestriction() throws Throwable
-    {
-        String functionName = createSimpleFunction();
-        String cql = String.format("SELECT * FROM %s WHERE token(k) = token(%s)",
-                                   KEYSPACE + "." + currentTable(),
-                                   functionCall(functionName));
-        assertPermissionsOnFunction(cql, functionName);
-    }
-
-    @Test
-    public void functionInSelectTokenSliceRestriction() throws Throwable
-    {
-        String functionName = createSimpleFunction();
-        String cql = String.format("SELECT * FROM %s WHERE token(k) < token(%s)",
-                                   KEYSPACE + "." + currentTable(),
-                                   functionCall(functionName));
-        assertPermissionsOnFunction(cql, functionName);
-    }
-
-    @Test
-    public void functionInPKForInsert() throws Throwable
-    {
-        String functionName = createSimpleFunction();
-        String cql = String.format("INSERT INTO %s (k, v1, v2) VALUES (%s, 0, 0)",
-                                   KEYSPACE + "." + currentTable(),
-                                   functionCall(functionName));
-        assertPermissionsOnFunction(cql, functionName);
-    }
-
-    @Test
-    public void functionInClusteringValuesForInsert() throws Throwable
-    {
-        String functionName = createSimpleFunction();
-        String cql = String.format("INSERT INTO %s (k, v1, v2) VALUES (0, %s, 0)",
-                                   KEYSPACE + "." + currentTable(),
-                                   functionCall(functionName));
-        assertPermissionsOnFunction(cql, functionName);
-    }
-
-    @Test
-    public void functionInPKForDelete() throws Throwable
-    {
-        String functionName = createSimpleFunction();
-        String cql = String.format("DELETE FROM %s WHERE k = %s",
-                                   KEYSPACE + "." + currentTable(),
-                                   functionCall(functionName));
-        assertPermissionsOnFunction(cql, functionName);
-    }
-
-    @Test
-    public void functionInClusteringValuesForDelete() throws Throwable
-    {
-        String functionName = createSimpleFunction();
-        String cql = String.format("DELETE FROM %s WHERE k = 0 AND v1 = %s",
-                                   KEYSPACE + "." + currentTable(),
-                                   functionCall(functionName));
-        assertPermissionsOnFunction(cql, functionName);
-    }
-
-    @Test
-    public void testBatchStatement() throws Throwable
-    {
-        List<ModificationStatement> statements = new ArrayList<>();
-        List<String> functions = new ArrayList<>();
-        for (int i = 0; i < 3; i++)
-        {
-            String functionName = createSimpleFunction();
-            ModificationStatement stmt =
-            (ModificationStatement) getStatement(String.format("INSERT INTO %s (k, v1, v2) " +
-                                                               "VALUES (%s, %s, %s)",
-                                                               KEYSPACE + "." + currentTable(),
-                                                               i, i, functionCall(functionName)));
-            functions.add(functionName);
-            statements.add(stmt);
-        }
-        BatchStatement batch = new BatchStatement(-1, BatchStatement.Type.LOGGED, statements, Attributes.none());
-        assertUnauthorized(batch, functions);
-
-        grantExecuteOnFunction(functions.get(0));
-        assertUnauthorized(batch, functions.subList(1, functions.size()));
-
-        grantExecuteOnFunction(functions.get(1));
-        assertUnauthorized(batch, functions.subList(2, functions.size()));
-
-        grantExecuteOnFunction(functions.get(2));
-        batch.checkAccess(clientState);
-    }
-
-    @Test
-    public void testNestedFunctions() throws Throwable
-    {
-        String innerFunctionName = createSimpleFunction();
-        String outerFunctionName = createFunction("int",
-                                                  "CREATE FUNCTION %s(input int) " +
-                                                  " CALLED ON NULL INPUT" +
-                                                  " RETURNS int" +
-                                                  " LANGUAGE java" +
-                                                  " AS 'return Integer.valueOf(0);'");
-        assertPermissionsOnNestedFunctions(innerFunctionName, outerFunctionName);
-    }
-
-    @Test
-    public void functionInStaticColumnRestrictionInSelect() throws Throwable
-    {
-        setupTable("CREATE TABLE %s (k int, s int STATIC, v1 int, v2 int, PRIMARY KEY(k, v1))");
-        String functionName = createSimpleFunction();
-        String cql = String.format("SELECT k FROM %s WHERE k = 0 AND s = %s",
-                                   KEYSPACE + "." + currentTable(),
-                                   functionCall(functionName));
-        assertPermissionsOnFunction(cql, functionName);
-    }
-
-    @Test
-    public void functionInRegularCondition() throws Throwable
-    {
-        String functionName = createSimpleFunction();
-        String cql = String.format("UPDATE %s SET v2 = 0 WHERE k = 0 AND v1 = 0 IF v2 = %s",
-                                   KEYSPACE + "." + currentTable(),
-                                   functionCall(functionName));
-        assertPermissionsOnFunction(cql, functionName);
-    }
-    @Test
-    public void functionInStaticColumnCondition() throws Throwable
-    {
-        setupTable("CREATE TABLE %s (k int, s int STATIC, v1 int, v2 int, PRIMARY KEY(k, v1))");
-        String functionName = createSimpleFunction();
-        String cql = String.format("UPDATE %s SET v2 = 0 WHERE k = 0 AND v1 = 0 IF s = %s",
-                                   KEYSPACE + "." + currentTable(),
-                                   functionCall(functionName));
-        assertPermissionsOnFunction(cql, functionName);
-    }
-
-    @Test
-    public void functionInCollectionLiteralCondition() throws Throwable
-    {
-        setupTable("CREATE TABLE %s (k int, v1 int, m_val map<int, int>, PRIMARY KEY(k))");
-        String functionName = createSimpleFunction();
-        String cql = String.format("UPDATE %s SET v1 = 0 WHERE k = 0 IF m_val = {%s : %s}",
-                                   KEYSPACE + "." + currentTable(),
-                                   functionCall(functionName),
-                                   functionCall(functionName));
-        assertPermissionsOnFunction(cql, functionName);
-    }
-
-    @Test
-    public void functionInCollectionElementCondition() throws Throwable
-    {
-        setupTable("CREATE TABLE %s (k int, v1 int, m_val map<int, int>, PRIMARY KEY(k))");
-        String functionName = createSimpleFunction();
-        String cql = String.format("UPDATE %s SET v1 = 0 WHERE k = 0 IF m_val[%s] = %s",
-                                   KEYSPACE + "." + currentTable(),
-                                   functionCall(functionName),
-                                   functionCall(functionName));
-        assertPermissionsOnFunction(cql, functionName);
-    }
-
-    @Test
-    public void systemFunctionsRequireNoExplicitPrivileges() throws Throwable
-    {
-        // with terminal arguments, so evaluated at prepare time
-        String cql = String.format("UPDATE %s SET v2 = 0 WHERE k = blobasint(intasblob(0))",
-                                   KEYSPACE + "." + currentTable());
-        getStatement(cql).checkAccess(clientState);
-
-        // with non-terminal arguments, so evaluated at execution
-        String functionName = createSimpleFunction();
-        grantExecuteOnFunction(functionName);
-        cql = String.format("UPDATE %s SET v2 = 0 WHERE k = blobasint(intasblob(%s))",
-                            KEYSPACE + "." + currentTable(),
-                            functionCall(functionName));
-        getStatement(cql).checkAccess(clientState);
-    }
-
-    @Test
-    public void requireExecutePermissionOnComponentFunctionsWhenDefiningAggregate() throws Throwable
-    {
-        String sFunc = createSimpleStateFunction();
-        String fFunc = createSimpleFinalFunction();
-        // aside from the component functions, we need CREATE on the keyspace's functions
-        DatabaseDescriptor.getAuthorizer().grant(AuthenticatedUser.SYSTEM_USER,
-                                                 ImmutableSet.of(Permission.CREATE),
-                                                 FunctionResource.keyspace(KEYSPACE),
-                                                 role);
-        String aggDef = String.format(aggregateCql(sFunc, fFunc),
-                                      KEYSPACE + ".aggregate_for_permissions_test");
-
-        assertUnauthorized(aggDef, sFunc, "int, int");
-        grantExecuteOnFunction(sFunc);
-
-        assertUnauthorized(aggDef, fFunc, "int");
-        grantExecuteOnFunction(fFunc);
-
-        getStatement(aggDef).checkAccess(clientState);
-    }
-
-    @Test
-    public void revokeExecutePermissionsOnAggregateComponents() throws Throwable
-    {
-        String sFunc = createSimpleStateFunction();
-        String fFunc = createSimpleFinalFunction();
-        String aggDef = aggregateCql(sFunc, fFunc);
-        grantExecuteOnFunction(sFunc);
-        grantExecuteOnFunction(fFunc);
-
-        String aggregate = createAggregate(KEYSPACE, "int", aggDef);
-        grantExecuteOnFunction(aggregate);
-
-        String cql = String.format("SELECT %s(v1) FROM %s",
-                                   aggregate,
-                                   KEYSPACE + "." + currentTable());
-        getStatement(cql).checkAccess(clientState);
-
-        // check that revoking EXECUTE permission on any one of the
-        // component functions means we lose the ability to execute it
-        revokeExecuteOnFunction(aggregate);
-        assertUnauthorized(cql, aggregate, "int");
-        grantExecuteOnFunction(aggregate);
-        getStatement(cql).checkAccess(clientState);
-
-        revokeExecuteOnFunction(sFunc);
-        assertUnauthorized(cql, sFunc, "int, int");
-        grantExecuteOnFunction(sFunc);
-        getStatement(cql).checkAccess(clientState);
-
-        revokeExecuteOnFunction(fFunc);
-        assertUnauthorized(cql, fFunc, "int");
-        grantExecuteOnFunction(fFunc);
-        getStatement(cql).checkAccess(clientState);
-    }
-
-    @Test
-    public void functionWrappingAggregate() throws Throwable
-    {
-        String outerFunc = createFunction("int",
-                                          "CREATE FUNCTION %s(input int) " +
-                                          "CALLED ON NULL INPUT " +
-                                          "RETURNS int " +
-                                          "LANGUAGE java " +
-                                          "AS 'return input;'");
-
-        String sFunc = createSimpleStateFunction();
-        String fFunc = createSimpleFinalFunction();
-        String aggDef = aggregateCql(sFunc, fFunc);
-        grantExecuteOnFunction(sFunc);
-        grantExecuteOnFunction(fFunc);
-
-        String aggregate = createAggregate(KEYSPACE, "int", aggDef);
-
-        String cql = String.format("SELECT %s(%s(v1)) FROM %s",
-                                   outerFunc,
-                                   aggregate,
-                                   KEYSPACE + "." + currentTable());
-
-        assertUnauthorized(cql, outerFunc, "int");
-        grantExecuteOnFunction(outerFunc);
-
-        assertUnauthorized(cql, aggregate, "int");
-        grantExecuteOnFunction(aggregate);
-
-        getStatement(cql).checkAccess(clientState);
-    }
-
-    @Test
-    public void aggregateWrappingFunction() throws Throwable
-    {
-        String innerFunc = createFunction("int",
-                                          "CREATE FUNCTION %s(input int) " +
-                                          "CALLED ON NULL INPUT " +
-                                          "RETURNS int " +
-                                          "LANGUAGE java " +
-                                          "AS 'return input;'");
-
-        String sFunc = createSimpleStateFunction();
-        String fFunc = createSimpleFinalFunction();
-        String aggDef = aggregateCql(sFunc, fFunc);
-        grantExecuteOnFunction(sFunc);
-        grantExecuteOnFunction(fFunc);
-
-        String aggregate = createAggregate(KEYSPACE, "int", aggDef);
-
-        String cql = String.format("SELECT %s(%s(v1)) FROM %s",
-                                   aggregate,
-                                   innerFunc,
-                                   KEYSPACE + "." + currentTable());
-
-        assertUnauthorized(cql, aggregate, "int");
-        grantExecuteOnFunction(aggregate);
-
-        assertUnauthorized(cql, innerFunc, "int");
-        grantExecuteOnFunction(innerFunc);
-
-        getStatement(cql).checkAccess(clientState);
-    }
-
-    private void assertPermissionsOnNestedFunctions(String innerFunction, String outerFunction) throws Throwable
-    {
-        String cql = String.format("SELECT k, %s FROM %s WHERE k=0",
-                                   functionCall(outerFunction, functionCall(innerFunction)),
-                                   KEYSPACE + "." + currentTable());
-        // fail fast with an UAE on the first function
-        assertUnauthorized(cql, outerFunction, "int");
-        grantExecuteOnFunction(outerFunction);
-
-        // after granting execute on the first function, still fail due to the inner function
-        assertUnauthorized(cql, innerFunction, "");
-        grantExecuteOnFunction(innerFunction);
-
-        // now execution of both is permitted
-        getStatement(cql).checkAccess(clientState);
-    }
-
-    private void assertPermissionsOnFunction(String cql, String functionName) throws Throwable
-    {
-        assertPermissionsOnFunction(cql, functionName, "");
-    }
-
-    private void assertPermissionsOnFunction(String cql, String functionName, String argTypes) throws Throwable
-    {
-        assertUnauthorized(cql, functionName, argTypes);
-        grantExecuteOnFunction(functionName);
-        getStatement(cql).checkAccess(clientState);
-    }
-
-    private void assertUnauthorized(BatchStatement batch, Iterable<String> functionNames) throws Throwable
-    {
-        try
-        {
-            batch.checkAccess(clientState);
-            fail("Expected an UnauthorizedException, but none was thrown");
-        }
-        catch (UnauthorizedException e)
-        {
-            String functions = String.format("(%s)", Joiner.on("|").join(functionNames));
-            assertTrue(e.getLocalizedMessage()
-                        .matches(String.format("User %s has no EXECUTE permission on <function %s\\(\\)> or any of its parents",
-                                               roleName,
-                                               functions)));
-        }
-    }
-
-    private void assertUnauthorized(String cql, String functionName, String argTypes) throws Throwable
-    {
-        try
-        {
-            getStatement(cql).checkAccess(clientState);
-            fail("Expected an UnauthorizedException, but none was thrown");
-        }
-        catch (UnauthorizedException e)
-        {
-            assertEquals(String.format("User %s has no EXECUTE permission on <function %s(%s)> or any of its parents",
-                                       roleName,
-                                       functionName,
-                                       argTypes),
-                         e.getLocalizedMessage());
-        }
-    }
-
-    private void grantExecuteOnFunction(String functionName)
-    {
-            DatabaseDescriptor.getAuthorizer().grant(AuthenticatedUser.SYSTEM_USER,
-                                                     ImmutableSet.of(Permission.EXECUTE),
-                                                     functionResource(functionName),
-                                                     role);
-    }
-
-    private void revokeExecuteOnFunction(String functionName)
-    {
-        DatabaseDescriptor.getAuthorizer().revoke(AuthenticatedUser.SYSTEM_USER,
-                                                  ImmutableSet.of(Permission.EXECUTE),
-                                                  functionResource(functionName),
-                                                  role);
-    }
-
-    void setupClientState()
-    {
-
-        try
-        {
-            role = RoleResource.role(roleName);
-            // use reflection to set the logged in user so that we don't need to
-            // bother setting up an IRoleManager
-            user = new AuthenticatedUser(roleName);
-            clientState = ClientState.forInternalCalls();
-            Field userField = ClientState.class.getDeclaredField("user");
-            userField.setAccessible(true);
-            userField.set(clientState, user);
-        }
-        catch (IllegalAccessException | NoSuchFieldException e)
-        {
-            throw new RuntimeException(e);
-        }
-    }
-
-    private void setupTable(String tableDef) throws Throwable
-    {
-        createTable(tableDef);
-        // test user needs SELECT & MODIFY on the table regardless of permissions on any function
-        DatabaseDescriptor.getAuthorizer().grant(AuthenticatedUser.SYSTEM_USER,
-                                                 ImmutableSet.of(Permission.SELECT, Permission.MODIFY),
-                                                 DataResource.table(KEYSPACE, currentTable()),
-                                                 RoleResource.role(user.getName()));
-    }
-
-    private String aggregateCql(String sFunc, String fFunc)
-    {
-        return "CREATE AGGREGATE %s(int) " +
-               "SFUNC " + shortFunctionName(sFunc) + " " +
-               "STYPE int " +
-               "FINALFUNC " + shortFunctionName(fFunc) + " " +
-               "INITCOND 0";
-    }
-
-    private String createSimpleStateFunction() throws Throwable
-    {
-        return createFunction("int, int",
-                              "CREATE FUNCTION %s(a int, b int) " +
-                              "CALLED ON NULL INPUT " +
-                              "RETURNS int " +
-                              "LANGUAGE java " +
-                              "AS 'return Integer.valueOf( (a != null ? a.intValue() : 0 ) + b.intValue());'");
-    }
-
-    private String createSimpleFinalFunction() throws Throwable
-    {
-        return createFunction("int",
-                              "CREATE FUNCTION %s(a int) " +
-                              "CALLED ON NULL INPUT " +
-                              "RETURNS int " +
-                              "LANGUAGE java " +
-                              "AS 'return a;'");
-    }
-
-    private String createSimpleFunction() throws Throwable
-    {
-        return createFunction("",
-                              "CREATE FUNCTION %s() " +
-                              "  CALLED ON NULL INPUT " +
-                              "  RETURNS int " +
-                              "  LANGUAGE java " +
-                              "  AS 'return Integer.valueOf(0);'");
-    }
-
-    private String createFunction(String argTypes, String functionDef) throws Throwable
-    {
-        return createFunction(KEYSPACE, argTypes, functionDef);
-    }
-
-    private CQLStatement getStatement(String cql)
-    {
-        return QueryProcessor.getStatement(cql, clientState).statement;
-    }
-
-    private FunctionResource functionResource(String functionName)
-    {
-        // Note that this is somewhat brittle as it assumes that function names are
-        // truly unique. As such, it will break in the face of overloading.
-        // It is here to avoid having to duplicate the functionality of CqlParser
-        // for transforming cql types into AbstractTypes
-        FunctionName fn = parseFunctionName(functionName);
-        List<Function> functions = Functions.find(fn);
-        assertEquals(String.format("Expected a single function definition for %s, but found %s",
-                                   functionName,
-                                   functions.size()),
-                     1, functions.size());
-        return FunctionResource.function(fn.keyspace, fn.name, functions.get(0).argTypes());
-    }
-
-    private String functionCall(String functionName, String...args)
-    {
-        return String.format("%s(%s)", functionName, Joiner.on(",").join(args));
-    }
-
-    static class StubAuthorizer implements IAuthorizer
-    {
-        Map<Pair<String, IResource>, Set<Permission>> userPermissions = new HashMap<>();
-
-        private void clear()
-        {
-            userPermissions.clear();
-        }
-
-        public Set<Permission> authorize(AuthenticatedUser user, IResource resource)
-        {
-            Pair<String, IResource> key = Pair.create(user.getName(), resource);
-            Set<Permission> perms = userPermissions.get(key);
-            return perms != null ? perms : Collections.<Permission>emptySet();
-        }
-
-        public void grant(AuthenticatedUser performer,
-                          Set<Permission> permissions,
-                          IResource resource,
-                          RoleResource grantee) throws RequestValidationException, RequestExecutionException
-        {
-            Pair<String, IResource> key = Pair.create(grantee.getRoleName(), resource);
-            Set<Permission> perms = userPermissions.get(key);
-            if (null == perms)
-            {
-                perms = new HashSet<>();
-                userPermissions.put(key, perms);
-            }
-            perms.addAll(permissions);
-        }
-
-        public void revoke(AuthenticatedUser performer,
-                           Set<Permission> permissions,
-                           IResource resource,
-                           RoleResource revokee) throws RequestValidationException, RequestExecutionException
-        {
-            Pair<String, IResource> key = Pair.create(revokee.getRoleName(), resource);
-            Set<Permission> perms = userPermissions.get(key);
-            if (null != perms)
-                perms.removeAll(permissions);
-            if (perms.isEmpty())
-                userPermissions.remove(key);
-        }
-
-        public Set<PermissionDetails> list(AuthenticatedUser performer,
-                                           Set<Permission> permissions,
-                                           IResource resource,
-                                           RoleResource grantee) throws RequestValidationException, RequestExecutionException
-        {
-            Pair<String, IResource> key = Pair.create(grantee.getRoleName(), resource);
-            Set<Permission> perms = userPermissions.get(key);
-            if (perms == null)
-                return Collections.emptySet();
-
-
-            Set<PermissionDetails> details = new HashSet<>();
-            for (Permission permission : perms)
-            {
-                if (permissions.contains(permission))
-                    details.add(new PermissionDetails(grantee.getRoleName(), resource, permission));
-            }
-            return details;
-        }
-
-        public void revokeAllFrom(RoleResource revokee)
-        {
-            for (Pair<String, IResource> key : userPermissions.keySet())
-                if (key.left.equals(revokee.getRoleName()))
-                    userPermissions.remove(key);
-        }
-
-        public void revokeAllOn(IResource droppedResource)
-        {
-            for (Pair<String, IResource> key : userPermissions.keySet())
-                if (key.right.equals(droppedResource))
-                    userPermissions.remove(key);
-
-        }
-
-        public Set<? extends IResource> protectedResources()
-        {
-            return Collections.emptySet();
-        }
-
-        public void validateConfiguration() throws ConfigurationException
-        {
-
-        }
-
-        public void setup()
-        {
-
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/01115f72/test/unit/org/apache/cassandra/cql3/UFIdentificationTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/UFIdentificationTest.java b/test/unit/org/apache/cassandra/cql3/UFIdentificationTest.java
deleted file mode 100644
index 703bd9a..0000000
--- a/test/unit/org/apache/cassandra/cql3/UFIdentificationTest.java
+++ /dev/null
@@ -1,376 +0,0 @@
-/*
- * 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;
-
-import java.util.*;
-
-import com.google.common.base.Joiner;
-import com.google.common.collect.Iterables;
-import org.junit.Before;
-import org.junit.Ignore;
-import org.junit.Test;
-
-import org.apache.cassandra.cql3.functions.Function;
-import org.apache.cassandra.cql3.statements.BatchStatement;
-import org.apache.cassandra.cql3.statements.ModificationStatement;
-import org.apache.cassandra.service.ClientState;
-
-import static org.junit.Assert.assertTrue;
-
-/**
- * Checks the collection of Function objects returned by CQLStatement.getFunction
- * matches expectations. This is intended to verify the various subcomponents of
- * the statement (Operations, Terms, Restrictions, RestrictionSet, Selection,
- * Selector, SelectorFactories etc) properly report any constituent functions.
- * Some purely terminal functions are resolved at preparation, so those are not
- * included in the reported list. They still need to be surveyed, to verify the
- * calling client has the necessary permissions. UFAuthTest includes tests which
- * verify this more thoroughly than we can here.
- */
-public class UFIdentificationTest extends CQLTester
-{
-    private com.google.common.base.Function<Function, String> toFunctionNames = new com.google.common.base.Function<Function, String>()
-    {
-        public String apply(Function f)
-        {
-            return f.name().keyspace + "." + f.name().name;
-        }
-    };
-
-    String tFunc;
-    String iFunc;
-    String lFunc;
-    String sFunc;
-    String mFunc;
-    String uFunc;
-    String udtFunc;
-
-    String userType;
-
-    @Before
-    public void setup() throws Throwable
-    {
-        userType = KEYSPACE + "." + createType("CREATE TYPE %s (t text, i int)");
-
-        createTable("CREATE TABLE %s (" +
-                    "   key int, " +
-                    "   t_sc text STATIC," +
-                    "   i_cc int, " +
-                    "   t_cc text, " +
-                    "   i_val int," +
-                    "   l_val list<int>," +
-                    "   s_val set<int>," +
-                    "   m_val map<int, int>," +
-                    "   u_val timeuuid," +
-                    "   udt_val frozen<" + userType + ">," +
-                    "   PRIMARY KEY (key, i_cc, t_cc)" +
-                    ")");
-
-        tFunc = createEchoFunction("text");
-        iFunc = createEchoFunction("int");
-        lFunc = createEchoFunction("list<int>");
-        sFunc = createEchoFunction("set<int>");
-        mFunc = createEchoFunction("map<int, int>");
-        uFunc = createEchoFunction("timeuuid");
-        udtFunc = createEchoFunction(userType);
-    }
-
-    @Test
-    public void testSimpleModificationStatement() throws Throwable
-    {
-        assertFunctions(cql("INSERT INTO %s (key, t_sc) VALUES (0, %s)", functionCall(tFunc, "'foo'")), tFunc);
-        assertFunctions(cql("INSERT INTO %s (key, i_cc) VALUES (0, %s)", functionCall(iFunc, "1")), iFunc);
-        assertFunctions(cql("INSERT INTO %s (key, t_cc) VALUES (0, %s)", functionCall(tFunc, "'foo'")), tFunc);
-        assertFunctions(cql("INSERT INTO %s (key, i_val) VALUES (0, %s)", functionCall(iFunc, "1")), iFunc);
-        assertFunctions(cql("INSERT INTO %s (key, l_val) VALUES (0, %s)", functionCall(lFunc, "[1]")), lFunc);
-        assertFunctions(cql("INSERT INTO %s (key, s_val) VALUES (0, %s)", functionCall(sFunc, "{1}")), sFunc);
-        assertFunctions(cql("INSERT INTO %s (key, m_val) VALUES (0, %s)", functionCall(mFunc, "{1:1}")), mFunc);
-        assertFunctions(cql("INSERT INTO %s (key, udt_val) VALUES (0,%s)", functionCall(udtFunc, "{i : 1, t : 'foo'}")), udtFunc);
-        assertFunctions(cql("INSERT INTO %s (key, u_val) VALUES (0, %s)", functionCall(uFunc, "now()")), uFunc, "system.now");
-    }
-
-    @Test
-    public void testNonTerminalCollectionLiterals() throws Throwable
-    {
-        String iFunc2 = createEchoFunction("int");
-        String mapValue = String.format("{%s:%s}", functionCall(iFunc, "1"), functionCall(iFunc2, "1"));
-        assertFunctions(cql("INSERT INTO %s (key, m_val) VALUES (0, %s)", mapValue), iFunc, iFunc2);
-
-        String listValue = String.format("[%s]", functionCall(iFunc, "1"));
-        assertFunctions(cql("INSERT INTO %s (key, l_val) VALUES (0, %s)", listValue), iFunc);
-
-        String setValue = String.format("{%s}", functionCall(iFunc, "1"));
-        assertFunctions(cql("INSERT INTO %s (key, s_val) VALUES (0, %s)", setValue), iFunc);
-    }
-
-    @Test
-    public void testNonTerminalUDTLiterals() throws Throwable
-    {
-        String udtValue = String.format("{ i: %s, t : %s } ", functionCall(iFunc, "1"), functionCall(tFunc, "'foo'"));
-        assertFunctions(cql("INSERT INTO %s (key, udt_val) VALUES (0, %s)", udtValue), iFunc, tFunc);
-    }
-
-    @Test
-    public void testModificationStatementWithConditions() throws Throwable
-    {
-        assertFunctions(cql("UPDATE %s SET i_val=0 WHERE key=0 IF t_sc=%s", functionCall(tFunc, "'foo'")), tFunc);
-        assertFunctions(cql("UPDATE %s SET i_val=0 WHERE key=0 IF i_val=%s", functionCall(iFunc, "1")), iFunc);
-        assertFunctions(cql("UPDATE %s SET i_val=0 WHERE key=0 IF l_val=%s", functionCall(lFunc, "[1]")), lFunc);
-        assertFunctions(cql("UPDATE %s SET i_val=0 WHERE key=0 IF s_val=%s", functionCall(sFunc, "{1}")), sFunc);
-        assertFunctions(cql("UPDATE %s SET i_val=0 WHERE key=0 IF m_val=%s", functionCall(mFunc, "{1:1}")), mFunc);
-
-
-        String iFunc2 = createEchoFunction("int");
-        assertFunctions(cql("UPDATE %s SET i_val=0 WHERE key=0 IF i_val IN (%s, %S)",
-                            functionCall(iFunc, "1"),
-                            functionCall(iFunc2, "2")),
-                        iFunc, iFunc2);
-
-        assertFunctions(cql("UPDATE %s SET i_val=0 WHERE key=0 IF u_val=%s",
-                            functionCall(uFunc, "now()")),
-                        uFunc, "system.now");
-
-        // conditions on collection elements
-        assertFunctions(cql("UPDATE %s SET i_val=0 WHERE key=0 IF l_val[%s] = %s",
-                            functionCall(iFunc, "1"),
-                            functionCall(iFunc2, "1")),
-                        iFunc, iFunc2);
-        assertFunctions(cql("UPDATE %s SET i_val=0 WHERE key=0 IF m_val[%s] = %s",
-                            functionCall(iFunc, "1"),
-                            functionCall(iFunc2, "1")),
-                        iFunc, iFunc2);
-    }
-
-    @Test @Ignore
-    // Technically, attributes like timestamp and ttl are Terms so could potentially
-    // resolve to function calls (& so you can call getFunctions on them)
-    // However, this is currently disallowed by CQL syntax
-    public void testModificationStatementWithAttributesFromFunction() throws Throwable
-    {
-        String longFunc = createEchoFunction("bigint");
-        assertFunctions(cql("INSERT INTO %s (key, i_cc, t_cc, i_val) VALUES (0, 0, 'foo', 0) USING TIMESTAMP %s",
-                            functionCall(longFunc, "9999")),
-                        longFunc);
-
-        assertFunctions(cql("INSERT INTO %s (key, i_cc, t_cc, i_val) VALUES (0, 0, 'foo', 0) USING TTL %s",
-                            functionCall(iFunc, "8888")),
-                        iFunc);
-
-        assertFunctions(cql("INSERT INTO %s (key, i_cc, t_cc, i_val) VALUES (0, 0, 'foo', 0) USING TIMESTAMP %s AND TTL %s",
-                            functionCall(longFunc, "9999"), functionCall(iFunc, "8888")),
-                        longFunc, iFunc);
-    }
-
-    @Test
-    public void testModificationStatementWithNestedFunctions() throws Throwable
-    {
-        String iFunc2 = createEchoFunction("int");
-        String iFunc3 = createEchoFunction("int");
-        String iFunc4 = createEchoFunction("int");
-        String iFunc5 = createEchoFunction("int");
-        String iFunc6 = createEchoFunction("int");
-        String nestedFunctionCall = nestedFunctionCall(iFunc6, iFunc5,
-                                                       nestedFunctionCall(iFunc4, iFunc3,
-                                                                          nestedFunctionCall(iFunc2, iFunc, "1")));
-
-        assertFunctions(cql("DELETE FROM %s WHERE key=%s", nestedFunctionCall),
-                        iFunc, iFunc2, iFunc3, iFunc4, iFunc5, iFunc6);
-    }
-
-    @Test
-    public void testSelectStatementSimpleRestrictions() throws Throwable
-    {
-        assertFunctions(cql("SELECT i_val FROM %s WHERE key=%s", functionCall(iFunc, "1")), iFunc);
-        assertFunctions(cql("SELECT i_val FROM %s WHERE key=0 AND t_sc=%s", functionCall(tFunc, "'foo'")), tFunc);
-        assertFunctions(cql("SELECT i_val FROM %s WHERE key=0 AND i_cc=%s AND t_cc='foo'", functionCall(iFunc, "1")), iFunc);
-        assertFunctions(cql("SELECT i_val FROM %s WHERE key=0 AND i_cc=0 AND t_cc=%s", functionCall(tFunc, "'foo'")), tFunc);
-
-        String iFunc2 = createEchoFunction("int");
-        String tFunc2 = createEchoFunction("text");
-        assertFunctions(cql("SELECT i_val FROM %s WHERE key=%s AND t_sc=%s AND i_cc=%s AND t_cc=%s",
-                            functionCall(iFunc, "1"),
-                            functionCall(tFunc, "'foo'"),
-                            functionCall(iFunc2, "1"),
-                            functionCall(tFunc2, "'foo'")),
-                        iFunc, tFunc, iFunc2, tFunc2);
-    }
-
-    @Test
-    public void testSelectStatementRestrictionsWithNestedFunctions() throws Throwable
-    {
-        String iFunc2 = createEchoFunction("int");
-        String iFunc3 = createEchoFunction("int");
-        String iFunc4 = createEchoFunction("int");
-        String iFunc5 = createEchoFunction("int");
-        String iFunc6 = createEchoFunction("int");
-        String nestedFunctionCall = nestedFunctionCall(iFunc6, iFunc5,
-                                                       nestedFunctionCall(iFunc3, iFunc4,
-                                                                          nestedFunctionCall(iFunc, iFunc2, "1")));
-
-        assertFunctions(cql("SELECT i_val FROM %s WHERE key=%s", nestedFunctionCall),
-                        iFunc, iFunc2, iFunc3, iFunc4, iFunc5, iFunc6);
-    }
-
-    @Test
-    public void testNonTerminalTupleInSelectRestrictions() throws Throwable
-    {
-        assertFunctions(cql("SELECT i_val FROM %s WHERE key=0 AND (i_cc, t_cc) IN ((%s, %s))",
-                            functionCall(iFunc, "1"),
-                            functionCall(tFunc, "'foo'")),
-                        iFunc, tFunc);
-
-        assertFunctions(cql("SELECT i_val FROM %s WHERE key=0 AND (i_cc, t_cc) = (%s, %s)",
-                            functionCall(iFunc, "1"),
-                            functionCall(tFunc, "'foo'")),
-                        iFunc, tFunc);
-
-        assertFunctions(cql("SELECT i_val FROM %s WHERE key=0 AND (i_cc, t_cc) > (%s, %s)",
-                            functionCall(iFunc, "1"),
-                            functionCall(tFunc, "'foo'")),
-                        iFunc, tFunc);
-
-        assertFunctions(cql("SELECT i_val FROM %s WHERE key=0 AND (i_cc, t_cc) < (%s, %s)",
-                            functionCall(iFunc, "1"),
-                            functionCall(tFunc, "'foo'")),
-                        iFunc, tFunc);
-
-         assertFunctions(cql("SELECT i_val FROM %s WHERE key=0 AND (i_cc, t_cc) > (%s, %s) AND (i_cc, t_cc) < (%s, %s)",
-                            functionCall(iFunc, "1"),
-                            functionCall(tFunc, "'foo'"),
-                            functionCall(iFunc, "1"),
-                            functionCall(tFunc, "'foo'")),
-                         iFunc, tFunc);
-    }
-
-    @Test
-    public void testNestedFunctionInTokenRestriction() throws Throwable
-    {
-        String iFunc2 = createEchoFunction("int");
-        assertFunctions(cql("SELECT i_val FROM %s WHERE token(key) = token(%s)", functionCall(iFunc, "1")),
-                        "system.token", iFunc);
-        assertFunctions(cql("SELECT i_val FROM %s WHERE token(key) > token(%s)", functionCall(iFunc, "1")),
-                        "system.token", iFunc);
-        assertFunctions(cql("SELECT i_val FROM %s WHERE token(key) < token(%s)", functionCall(iFunc, "1")),
-                        "system.token", iFunc);
-        assertFunctions(cql("SELECT i_val FROM %s WHERE token(key) > token(%s) AND token(key) < token(%s)",
-                            functionCall(iFunc, "1"),
-                            functionCall(iFunc2, "1")),
-                        "system.token", iFunc, iFunc2);
-    }
-
-    @Test
-    public void testSelectStatementSimpleSelections() throws Throwable
-    {
-        String iFunc2 = createEchoFunction("int");
-        execute("INSERT INTO %s (key, i_cc, t_cc, i_val) VALUES (0, 0, 'foo', 0)");
-        assertFunctions(cql2("SELECT i_val, %s FROM %s WHERE key=0", functionCall(iFunc, "i_val")), iFunc);
-        assertFunctions(cql2("SELECT i_val, %s FROM %s WHERE key=0", nestedFunctionCall(iFunc, iFunc2, "i_val")), iFunc, iFunc2);
-    }
-
-    @Test
-    public void testSelectStatementNestedSelections() throws Throwable
-    {
-        String iFunc2 = createEchoFunction("int");
-        execute("INSERT INTO %s (key, i_cc, t_cc, i_val) VALUES (0, 0, 'foo', 0)");
-        assertFunctions(cql2("SELECT i_val, %s FROM %s WHERE key=0", functionCall(iFunc, "i_val")), iFunc);
-        assertFunctions(cql2("SELECT i_val, %s FROM %s WHERE key=0", nestedFunctionCall(iFunc, iFunc2, "i_val")), iFunc, iFunc2);
-    }
-
-    @Test
-    public void testBatchStatement() throws Throwable
-    {
-        String iFunc2 = createEchoFunction("int");
-        List<ModificationStatement> statements = new ArrayList<>();
-        statements.add(modificationStatement(cql("INSERT INTO %s (key, i_cc, t_cc) VALUES (%s, 0, 'foo')",
-                                                 functionCall(iFunc, "0"))));
-        statements.add(modificationStatement(cql("INSERT INTO %s (key, i_cc, t_cc) VALUES (1, %s, 'foo')",
-                                                 functionCall(iFunc2, "1"))));
-        statements.add(modificationStatement(cql("INSERT INTO %s (key, i_cc, t_cc) VALUES (2, 2, %s)",
-                                                 functionCall(tFunc, "'foo'"))));
-
-        BatchStatement batch = new BatchStatement(-1, BatchStatement.Type.LOGGED, statements, Attributes.none());
-        assertFunctions(batch, iFunc, iFunc2, tFunc);
-    }
-
-    @Test
-    public void testBatchStatementWithConditions() throws Throwable
-    {
-        List<ModificationStatement> statements = new ArrayList<>();
-        statements.add(modificationStatement(cql("UPDATE %s SET i_val = %s WHERE key=0 AND i_cc=0 and t_cc='foo' IF l_val = %s",
-                                                 functionCall(iFunc, "0"), functionCall(lFunc, "[1]"))));
-        statements.add(modificationStatement(cql("UPDATE %s SET i_val = %s WHERE key=0 AND i_cc=1 and t_cc='foo' IF s_val = %s",
-                                                 functionCall(iFunc, "0"), functionCall(sFunc, "{1}"))));
-
-        BatchStatement batch = new BatchStatement(-1, BatchStatement.Type.LOGGED, statements, Attributes.none());
-        assertFunctions(batch, iFunc, lFunc, sFunc);
-    }
-
-    private ModificationStatement modificationStatement(String cql)
-    {
-        return (ModificationStatement)QueryProcessor.getStatement(cql, ClientState.forInternalCalls()).statement;
-    }
-
-    private void assertFunctions(String cql, String... function)
-    {
-        CQLStatement stmt = QueryProcessor.getStatement(cql, ClientState.forInternalCalls()).statement;
-        assertFunctions(stmt, function);
-    }
-
-    private void assertFunctions(CQLStatement stmt, String... function)
-    {
-        Set<String> expected = com.google.common.collect.Sets.newHashSet(function);
-        Set<String> actual = com.google.common.collect.Sets.newHashSet(Iterables.transform(stmt.getFunctions(),
-                                                                                           toFunctionNames));
-        assertTrue(com.google.common.collect.Sets.symmetricDifference(expected, actual).isEmpty());
-    }
-
-    private String cql(String template, String... params)
-    {
-        String tableName = KEYSPACE + "." + currentTable();
-        return String.format(template, com.google.common.collect.Lists.asList(tableName, params).toArray());
-    }
-
-    // Alternative query builder - appends the table name to the supplied params,
-    // for stmts of the form "SELECT x, %s FROM %s WHERE y=0"
-    private String cql2(String template, String... params)
-    {
-        Object[] args = Arrays.copyOf(params, params.length + 1);
-        args[params.length] = KEYSPACE + "." + currentTable();
-        return String.format(template, args);
-    }
-
-    private String functionCall(String fName, String... args)
-    {
-        return String.format("%s(%s)", fName, Joiner.on(",").join(args));
-    }
-
-    private String nestedFunctionCall(String outer, String inner, String innerArgs)
-    {
-        return functionCall(outer, functionCall(inner, innerArgs));
-    }
-
-    private String createEchoFunction(String type) throws Throwable
-    {
-        return createFunction(KEYSPACE, type,
-           "CREATE FUNCTION %s(input " + type + ")" +
-           " CALLED ON NULL INPUT" +
-           " RETURNS " + type +
-           " LANGUAGE java" +
-           " AS ' return input;'");
-    }
-}