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

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

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