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

[4/9] cassandra git commit: Migrate CQL tests from dtest to unit tests

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);
+        }
+    }
+}