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

[14/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/operations/BatchTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/BatchTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/BatchTest.java
new file mode 100644
index 0000000..1447845
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/BatchTest.java
@@ -0,0 +1,106 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.cql3.validation.operations;
+
+import org.junit.Test;
+
+import org.apache.cassandra.cql3.CQLTester;
+
+public class BatchTest extends CQLTester
+{
+    /**
+     * Test batch statements
+     * migrated from cql_tests.py:TestCQL.batch_test()
+     */
+    @Test
+    public void testBatch() throws Throwable
+    {
+        createTable("CREATE TABLE %s (userid text PRIMARY KEY, name text, password text)");
+
+        String query = "BEGIN BATCH\n"
+                       + "INSERT INTO %1$s (userid, password, name) VALUES ('user2', 'ch@ngem3b', 'second user');\n"
+                       + "UPDATE %1$s SET password = 'ps22dhds' WHERE userid = 'user3';\n"
+                       + "INSERT INTO %1$s (userid, password) VALUES ('user4', 'ch@ngem3c');\n"
+                       + "DELETE name FROM %1$s WHERE userid = 'user1';\n"
+                       + "APPLY BATCH;";
+
+        execute(query);
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.batch_and_list_test()
+     */
+    @Test
+    public void testBatchAndList() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, l list<int>)");
+
+        execute("BEGIN BATCH " +
+                "UPDATE %1$s SET l = l +[ 1 ] WHERE k = 0; " +
+                "UPDATE %1$s SET l = l + [ 2 ] WHERE k = 0; " +
+                "UPDATE %1$s SET l = l + [ 3 ] WHERE k = 0; " +
+                "APPLY BATCH");
+
+        assertRows(execute("SELECT l FROM %s WHERE k = 0"),
+                   row(list(1, 2, 3)));
+
+        execute("BEGIN BATCH " +
+                "UPDATE %1$s SET l =[ 1 ] + l WHERE k = 1; " +
+                "UPDATE %1$s SET l = [ 2 ] + l WHERE k = 1; " +
+                "UPDATE %1$s SET l = [ 3 ] + l WHERE k = 1; " +
+                "APPLY BATCH ");
+
+        assertRows(execute("SELECT l FROM %s WHERE k = 1"),
+                   row(list(3, 2, 1)));
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.bug_6115_test()
+     */
+    @Test
+    public void testBatchDeleteInsert() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int, v int, PRIMARY KEY (k, v))");
+
+        execute("INSERT INTO %s (k, v) VALUES (0, 1)");
+        execute("BEGIN BATCH DELETE FROM %1$s WHERE k=0 AND v=1; INSERT INTO %1$s (k, v) VALUES (0, 2); APPLY BATCH");
+
+        assertRows(execute("SELECT * FROM %s"),
+                   row(0, 2));
+    }
+
+    @Test
+    public void testBatchWithUnset() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, s text, i int)");
+
+        // test batch and update
+        String qualifiedTable = keyspace() + "." + currentTable();
+        execute("BEGIN BATCH " +
+                "INSERT INTO %s (k, s, i) VALUES (100, 'batchtext', 7); " +
+                "INSERT INTO " + qualifiedTable + " (k, s, i) VALUES (111, 'batchtext', 7); " +
+                "UPDATE " + qualifiedTable + " SET s=?, i=? WHERE k = 100; " +
+                "UPDATE " + qualifiedTable + " SET s=?, i=? WHERE k=111; " +
+                "APPLY BATCH;", null, unset(), unset(), null);
+        assertRows(execute("SELECT k, s, i FROM %s where k in (100,111)"),
+                   row(100, null, 7),
+                   row(111, "batchtext", null)
+        );
+    }
+}

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

http://git-wip-us.apache.org/repos/asf/cassandra/blob/01115f72/test/unit/org/apache/cassandra/cql3/validation/operations/DeleteTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/DeleteTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/DeleteTest.java
new file mode 100644
index 0000000..476ec83
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/DeleteTest.java
@@ -0,0 +1,329 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.cql3.validation.operations;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+
+import org.junit.Test;
+
+import org.apache.cassandra.cql3.CQLTester;
+
+import static org.junit.Assert.assertEquals;
+
+public class DeleteTest extends CQLTester
+{
+    /** Test for cassandra 8558 */
+    @Test
+    public void testRangeDeletion() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int, b int, c int, d int, PRIMARY KEY (a, b, c))");
+
+        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 1, 1, 1, 1);
+        flush();
+        execute("DELETE FROM %s WHERE a=? AND b=?", 1, 1);
+        flush();
+        assertEmpty(execute("SELECT * FROM %s WHERE a=? AND b=? AND c=?", 1, 1, 1));
+    }
+
+    /**
+     * Test simple deletion and in particular check for #4193 bug
+     * migrated from cql_tests.py:TestCQL.deletion_test()
+     */
+    @Test
+    public void testDeletion() throws Throwable
+    {
+        createTable("CREATE TABLE %s (username varchar, id int, name varchar, stuff varchar, PRIMARY KEY(username, id))");
+
+        execute("INSERT INTO %s (username, id, name, stuff) VALUES (?, ?, ?, ?)", "abc", 2, "rst", "some value");
+        execute("INSERT INTO %s (username, id, name, stuff) VALUES (?, ?, ?, ?)", "abc", 4, "xyz", "some other value");
+
+        assertRows(execute("SELECT * FROM %s"),
+                   row("abc", 2, "rst", "some value"),
+                   row("abc", 4, "xyz", "some other value"));
+
+        execute("DELETE FROM %s WHERE username='abc' AND id=2");
+
+        assertRows(execute("SELECT * FROM %s"),
+                   row("abc", 4, "xyz", "some other value"));
+
+        createTable("CREATE TABLE %s (username varchar, id int, name varchar, stuff varchar, PRIMARY KEY(username, id, name)) WITH COMPACT STORAGE");
+
+        execute("INSERT INTO %s (username, id, name, stuff) VALUES (?, ?, ?, ?)", "abc", 2, "rst", "some value");
+        execute("INSERT INTO %s (username, id, name, stuff) VALUES (?, ?, ?, ?)", "abc", 4, "xyz", "some other value");
+
+        assertRows(execute("SELECT * FROM %s"),
+                   row("abc", 2, "rst", "some value"),
+                   row("abc", 4, "xyz", "some other value"));
+
+        execute("DELETE FROM %s WHERE username='abc' AND id=2");
+
+        assertRows(execute("SELECT * FROM %s"),
+                   row("abc", 4, "xyz", "some other value"));
+    }
+
+    /**
+     * Test deletion by 'composite prefix' (range tombstones)
+     * migrated from cql_tests.py:TestCQL.range_tombstones_test()
+     */
+    @Test
+    public void testDeleteByCompositePrefix() throws Throwable
+    { // This test used 3 nodes just to make sure RowMutation are correctly serialized
+
+        createTable("CREATE TABLE %s ( k int, c1 int, c2 int, v1 int, v2 int, PRIMARY KEY (k, c1, c2))");
+
+        int numRows = 5;
+        int col1 = 2;
+        int col2 = 2;
+        int cpr = col1 * col2;
+
+        for (int i = 0; i < numRows; i++)
+            for (int j = 0; j < col1; j++)
+                for (int k = 0; k < col2; k++)
+                {
+                    int n = (i * cpr) + (j * col2) + k;
+                    execute("INSERT INTO %s (k, c1, c2, v1, v2) VALUES (?, ?, ?, ?, ?)", i, j, k, n, n);
+                }
+
+        for (int i = 0; i < numRows; i++)
+        {
+            Object[][] rows = getRows(execute("SELECT v1, v2 FROM %s where k = ?", i));
+            for (int x = i * cpr; x < (i + 1) * cpr; x++)
+            {
+                assertEquals(x, rows[x - i * cpr][0]);
+                assertEquals(x, rows[x - i * cpr][1]);
+            }
+        }
+
+        for (int i = 0; i < numRows; i++)
+            execute("DELETE FROM %s WHERE k = ? AND c1 = 0", i);
+
+        for (int i = 0; i < numRows; i++)
+        {
+            Object[][] rows = getRows(execute("SELECT v1, v2 FROM %s WHERE k = ?", i));
+            for (int x = i * cpr + col1; x < (i + 1) * cpr; x++)
+            {
+                assertEquals(x, rows[x - i * cpr - col1][0]);
+                assertEquals(x, rows[x - i * cpr - col1][1]);
+            }
+        }
+
+        for (int i = 0; i < numRows; i++)
+        {
+            Object[][] rows = getRows(execute("SELECT v1, v2 FROM %s WHERE k = ?", i));
+            for (int x = i * cpr + col1; x < (i + 1) * cpr; x++)
+            {
+                assertEquals(x, rows[x - i * cpr - col1][0]);
+                assertEquals(x, rows[x - i * cpr - col1][1]);
+            }
+        }
+    }
+
+    /**
+     * Test deletion by 'composite prefix' (range tombstones) with compaction
+     * migrated from cql_tests.py:TestCQL.range_tombstones_compaction_test()
+     */
+    @Test
+    public void testDeleteByCompositePrefixWithCompaction() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int, c1 int, c2 int, v1 text, PRIMARY KEY (k, c1, c2))");
+
+        for (int c1 = 0; c1 < 4; c1++)
+            for (int c2 = 0; c2 < 2; c2++)
+                execute("INSERT INTO %s (k, c1, c2, v1) VALUES (0, ?, ?, ?)", c1, c2, String.format("%d%d", c1, c2));
+
+        flush();
+
+        execute("DELETE FROM %s WHERE k = 0 AND c1 = 1");
+
+        flush();
+        compact();
+
+        Object[][] rows = getRows(execute("SELECT v1 FROM %s WHERE k = 0"));
+
+        int idx = 0;
+        for (int c1 = 0; c1 < 4; c1++)
+            for (int c2 = 0; c2 < 2; c2++)
+                if (c1 != 1)
+                    assertEquals(String.format("%d%d", c1, c2), rows[idx++][0]);
+    }
+
+    /**
+     * Test deletion of rows
+     * migrated from cql_tests.py:TestCQL.delete_row_test()
+     */
+    @Test
+    public void testRowDeletion() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int, c1 int, c2 int, v1 int, v2 int, PRIMARY KEY (k, c1, c2))");
+
+        execute("INSERT INTO %s (k, c1, c2, v1, v2) VALUES (?, ?, ?, ?, ?)", 0, 0, 0, 0, 0);
+        execute("INSERT INTO %s (k, c1, c2, v1, v2) VALUES (?, ?, ?, ?, ?)", 0, 0, 1, 1, 1);
+        execute("INSERT INTO %s (k, c1, c2, v1, v2) VALUES (?, ?, ?, ?, ?)", 0, 0, 2, 2, 2);
+        execute("INSERT INTO %s (k, c1, c2, v1, v2) VALUES (?, ?, ?, ?, ?)", 0, 1, 0, 3, 3);
+
+        execute("DELETE FROM %s WHERE k = 0 AND c1 = 0 AND c2 = 0");
+
+        assertRowCount(execute("SELECT * FROM %s"), 3);
+    }
+
+    /**
+     * Check the semantic of CQL row existence (part of #4361),
+     * migrated from cql_tests.py:TestCQL.row_existence_test()
+     */
+    @Test
+    public void testRowExistence() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int, c int, v1 int, v2 int, PRIMARY KEY (k, c))");
+
+        execute("INSERT INTO %s (k, c, v1, v2) VALUES (1, 1, 1, 1)");
+        assertRows(execute("SELECT * FROM %s"),
+                   row(1, 1, 1, 1));
+
+        assertInvalid("DELETE c FROM %s WHERE k = 1 AND c = 1");
+
+        execute("DELETE v2 FROM %s WHERE k = 1 AND c = 1");
+        assertRows(execute("SELECT * FROM %s"),
+                   row(1, 1, 1, null));
+
+        execute("DELETE v1 FROM %s WHERE k = 1 AND c = 1");
+        assertRows(execute("SELECT * FROM %s"),
+                   row(1, 1, null, null));
+
+        execute("DELETE FROM %s WHERE k = 1 AND c = 1");
+        assertEmpty(execute("SELECT * FROM %s"));
+
+        execute("INSERT INTO %s (k, c) VALUES (2, 2)");
+        assertRows(execute("SELECT * FROM %s"),
+                   row(2, 2, null, null));
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.remove_range_slice_test()
+     */
+    @Test
+    public void testRemoveRangeSlice() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, v int)");
+
+        for (int i = 0; i < 3; i++)
+            execute("INSERT INTO %s (k, v) VALUES (?, ?)", i, i);
+
+        execute("DELETE FROM %s WHERE k = 1");
+        assertRows(execute("SELECT * FROM %s"),
+                   row(0, 0),
+                   row(2, 2));
+    }
+
+    /**
+     * Test deletions
+     * migrated from cql_tests.py:TestCQL.no_range_ghost_test()
+     */
+    @Test
+    public void testNoRangeGhost() throws Throwable
+    {
+        createTable("CREATE TABLE %s ( k int PRIMARY KEY, v int ) ");
+
+        for (int k = 0; k < 5; k++)
+            execute("INSERT INTO %s (k, v) VALUES (?, 0)", k);
+
+        Object[][] rows = getRows(execute("SELECT k FROM %s"));
+
+        int[] ordered = sortIntRows(rows);
+        for (int k = 0; k < 5; k++)
+            assertEquals(k, ordered[k]);
+
+        execute("DELETE FROM %s WHERE k=2");
+
+        rows = getRows(execute("SELECT k FROM %s"));
+        ordered = sortIntRows(rows);
+
+        int idx = 0;
+        for (int k = 0; k < 5; k++)
+            if (k != 2)
+                assertEquals(k, ordered[idx++]);
+
+        // Example from #3505
+        createTable("CREATE TABLE %s ( KEY varchar PRIMARY KEY, password varchar, gender varchar, birth_year bigint)");
+        execute("INSERT INTO %s (KEY, password) VALUES ('user1', 'ch@ngem3a')");
+        execute("UPDATE %s SET gender = 'm', birth_year = 1980 WHERE KEY = 'user1'");
+
+        assertRows(execute("SELECT * FROM %s WHERE KEY='user1'"),
+                   row("user1", 1980L, "m", "ch@ngem3a"));
+
+        execute("TRUNCATE %s");
+        assertEmpty(execute("SELECT * FROM %s"));
+
+        assertEmpty(execute("SELECT * FROM %s WHERE KEY='user1'"));
+    }
+
+    private int[] sortIntRows(Object[][] rows)
+    {
+        int[] ret = new int[rows.length];
+        for (int i = 0; i < ret.length; i++)
+            ret[i] = rows[i][0] == null ? Integer.MIN_VALUE : (Integer) rows[i][0];
+        Arrays.sort(ret);
+        return ret;
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.range_with_deletes_test()
+     */
+    @Test
+    public void testRandomDeletions() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, v int,)");
+
+        int nb_keys = 30;
+        int nb_deletes = 5;
+
+        List<Integer> deletions = new ArrayList<>(nb_keys);
+        for (int i = 0; i < nb_keys; i++)
+        {
+            execute("INSERT INTO %s (k, v) VALUES (?, ?)", i, i);
+            deletions.add(i);
+        }
+
+        Collections.shuffle(deletions);
+
+        for (int i = 0; i < nb_deletes; i++)
+            execute("DELETE FROM %s WHERE k = ?", deletions.get(i));
+
+        assertRowCount(execute("SELECT * FROM %s LIMIT ?", (nb_keys / 2)), nb_keys / 2);
+    }
+
+    /**
+     * Test for CASSANDRA-8558, deleted row still can be selected out
+     * migrated from cql_tests.py:TestCQL.bug_8558_test()
+     */
+    @Test
+    public void testDeletedRowCannotBeSelected() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int, b int, c text,primary key(a,b))");
+        execute("INSERT INTO %s (a,b,c) VALUES(1,1,'1')");
+        flush();
+
+        execute("DELETE FROM %s  where a=1 and b=1");
+        flush();
+
+        assertEmpty(execute("select * from %s  where a=1 and b=1"));
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/01115f72/test/unit/org/apache/cassandra/cql3/validation/operations/InsertTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/InsertTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/InsertTest.java
new file mode 100644
index 0000000..6e9d212
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/InsertTest.java
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.cql3.validation.operations;
+
+import org.junit.Test;
+
+import org.apache.cassandra.cql3.CQLTester;
+
+public class InsertTest extends CQLTester
+{
+    @Test
+    public void testInsertWithUnset() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, s text, i int)");
+
+        // insert using nulls
+        execute("INSERT INTO %s (k, s, i) VALUES (10, ?, ?)", "text", 10);
+        execute("INSERT INTO %s (k, s, i) VALUES (10, ?, ?)", null, null);
+        assertRows(execute("SELECT s, i FROM %s WHERE k = 10"),
+                   row(null, null) // sending null deletes the data
+        );
+        // insert using UNSET
+        execute("INSERT INTO %s (k, s, i) VALUES (11, ?, ?)", "text", 10);
+        execute("INSERT INTO %s (k, s, i) VALUES (11, ?, ?)", unset(), unset());
+        assertRows(execute("SELECT s, i FROM %s WHERE k=11"),
+                   row("text", 10) // unset columns does not delete the existing data
+        );
+
+        assertInvalidMessage("Invalid unset value for column k", "UPDATE %s SET i = 0 WHERE k = ?", unset());
+        assertInvalidMessage("Invalid unset value for column k", "DELETE FROM %s WHERE k = ?", unset());
+        assertInvalidMessage("Invalid unset value for argument in call to function blobasint", "SELECT * FROM %s WHERE k = blobAsInt(?)", unset());
+    }
+
+    @Test
+    public void testInsertTtlWithUnset() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, i int)");
+        execute("INSERT INTO %s (k, i) VALUES (1, 1) USING TTL ?", unset()); // treat as 'unlimited'
+        assertRows(execute("SELECT ttl(i) FROM %s"),
+                   row(new Object[]{ null })
+        );
+    }
+}

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

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