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:15:08 UTC

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

http://git-wip-us.apache.org/repos/asf/cassandra/blob/01115f72/test/unit/org/apache/cassandra/cql3/CollectionsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/CollectionsTest.java b/test/unit/org/apache/cassandra/cql3/CollectionsTest.java
deleted file mode 100644
index c9c6d31..0000000
--- a/test/unit/org/apache/cassandra/cql3/CollectionsTest.java
+++ /dev/null
@@ -1,340 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.cassandra.cql3;
-
-import org.junit.Test;
-
-public class CollectionsTest extends CQLTester
-{
-    @Test
-    public void testMapBulkRemoval() throws Throwable
-    {
-        createTable("CREATE TABLE %s (k int PRIMARY KEY, m map<text, text>)");
-
-        execute("INSERT INTO %s(k, m) VALUES (?, ?)", 0, map("k1", "v1", "k2", "v2", "k3", "v3"));
-
-        assertRows(execute("SELECT * FROM %s"),
-            row(0, map("k1", "v1", "k2", "v2", "k3", "v3"))
-        );
-
-        execute("UPDATE %s SET m = m - ? WHERE k = ?", set("k2"), 0);
-
-        assertRows(execute("SELECT * FROM %s"),
-            row(0, map("k1", "v1", "k3", "v3"))
-        );
-
-        execute("UPDATE %s SET m = m + ?, m = m - ? WHERE k = ?", map("k4", "v4"), set("k3"), 0);
-
-        assertRows(execute("SELECT * FROM %s"),
-            row(0, map("k1", "v1", "k4", "v4"))
-        );
-    }
-
-    @Test
-    public void testInvalidCollectionsMix() throws Throwable
-    {
-        createTable("CREATE TABLE %s (k int PRIMARY KEY, l list<text>, s set<text>, m map<text, text>)");
-
-        // Note: we force the non-prepared form for some of those tests because a list and a set
-        // have the same serialized format in practice and CQLTester don't validate that the type
-        // of what's passed as a value in the prepared case, so the queries would work (which is ok,
-        // CQLTester is just a "dumb" client).
-
-        assertInvalid("UPDATE %s SET l = l + { 'a', 'b' } WHERE k = 0");
-        assertInvalid("UPDATE %s SET l = l - { 'a', 'b' } WHERE k = 0");
-        assertInvalid("UPDATE %s SET l = l + ? WHERE k = 0", map("a", "b", "c", "d"));
-        assertInvalid("UPDATE %s SET l = l - ? WHERE k = 0", map("a", "b", "c", "d"));
-
-        assertInvalid("UPDATE %s SET s = s + [ 'a', 'b' ] WHERE k = 0");
-        assertInvalid("UPDATE %s SET s = s - [ 'a', 'b' ] WHERE k = 0");
-        assertInvalid("UPDATE %s SET s = s + ? WHERE k = 0", map("a", "b", "c", "d"));
-        assertInvalid("UPDATE %s SET s = s - ? WHERE k = 0", map("a", "b", "c", "d"));
-
-        assertInvalid("UPDATE %s SET m = m + ? WHERE k = 0", list("a", "b"));
-        assertInvalid("UPDATE %s SET m = m - [ 'a', 'b' ] WHERE k = 0");
-        assertInvalid("UPDATE %s SET m = m + ? WHERE k = 0", set("a", "b"));
-        assertInvalid("UPDATE %s SET m = m - ? WHERE k = 0", map("a", "b", "c", "d"));
-    }
-
-    @Test
-    public void testSets() throws Throwable
-    {
-        createTable("CREATE TABLE %s (k int PRIMARY KEY, s set<text>)");
-
-        execute("INSERT INTO %s(k, s) VALUES (0, ?)", set("v1", "v2", "v3", "v4"));
-
-        assertRows(execute("SELECT s FROM %s WHERE k = 0"),
-            row(set("v1", "v2", "v3", "v4"))
-        );
-
-        execute("DELETE s[?] FROM %s WHERE k = 0", "v1");
-
-        assertRows(execute("SELECT s FROM %s WHERE k = 0"),
-            row(set("v2", "v3", "v4"))
-        );
-
-        // Full overwrite
-        execute("UPDATE %s SET s = ? WHERE k = 0", set("v6", "v5"));
-
-        assertRows(execute("SELECT s FROM %s WHERE k = 0"),
-            row(set("v5", "v6"))
-        );
-
-        execute("UPDATE %s SET s = s + ? WHERE k = 0", set("v7"));
-
-        assertRows(execute("SELECT s FROM %s WHERE k = 0"),
-            row(set("v5", "v6", "v7"))
-        );
-
-        execute("UPDATE %s SET s = s - ? WHERE k = 0", set("v6", "v5"));
-
-        assertRows(execute("SELECT s FROM %s WHERE k = 0"),
-            row(set("v7"))
-        );
-
-        execute("DELETE s[?] FROM %s WHERE k = 0", set("v7"));
-
-        // Deleting an element that does not exist will succeed
-        execute("DELETE s[?] FROM %s WHERE k = 0", set("v7"));
-
-        execute("DELETE s FROM %s WHERE k = 0");
-
-        assertRows(execute("SELECT s FROM %s WHERE k = 0"),
-            row((Object)null)
-        );
-    }
-
-    @Test
-    public void testMaps() throws Throwable
-    {
-        createTable("CREATE TABLE %s (k int PRIMARY KEY, m map<text, int>)");
-
-        execute("INSERT INTO %s(k, m) VALUES (0, ?)", map("v1", 1, "v2", 2));
-
-        assertRows(execute("SELECT m FROM %s WHERE k = 0"),
-            row(map("v1", 1, "v2", 2))
-        );
-
-        execute("UPDATE %s SET m[?] = ?, m[?] = ? WHERE k = 0", "v3", 3, "v4", 4);
-
-        assertRows(execute("SELECT m FROM %s WHERE k = 0"),
-            row(map("v1", 1, "v2", 2, "v3", 3, "v4", 4))
-        );
-
-        execute("DELETE m[?] FROM %s WHERE k = 0", "v1");
-
-        assertRows(execute("SELECT m FROM %s WHERE k = 0"),
-            row(map("v2", 2, "v3", 3, "v4", 4))
-        );
-
-        // Full overwrite
-        execute("UPDATE %s SET m = ? WHERE k = 0", map("v6", 6, "v5", 5));
-
-        assertRows(execute("SELECT m FROM %s WHERE k = 0"),
-            row(map("v5", 5, "v6", 6))
-        );
-
-        execute("UPDATE %s SET m = m + ? WHERE k = 0", map("v7", 7));
-
-        assertRows(execute("SELECT m FROM %s WHERE k = 0"),
-            row(map("v5", 5, "v6", 6, "v7", 7))
-        );
-
-        execute("DELETE m[?] FROM %s WHERE k = 0", "v7");
-
-        assertRows(execute("SELECT m FROM %s WHERE k = 0"),
-            row(map("v5", 5, "v6", 6))
-        );
-
-        execute("DELETE m[?] FROM %s WHERE k = 0", "v6");
-
-        assertRows(execute("SELECT m FROM %s WHERE k = 0"),
-            row(map("v5", 5))
-        );
-
-        execute("DELETE m[?] FROM %s WHERE k = 0", "v5");
-
-        assertRows(execute("SELECT m FROM %s WHERE k = 0"),
-            row((Object)null)
-        );
-
-        // Deleting a non-existing key should succeed
-        execute("DELETE m[?] FROM %s WHERE k = 0", "v5");
-
-        assertRows(execute("SELECT m FROM %s WHERE k = 0"),
-            row((Object) null)
-        );
-
-        // The empty map is parsed as an empty set (because we don't have enough info at parsing
-        // time when we see a {}) and special cased later. This test checks this work properly
-        execute("UPDATE %s SET m = {} WHERE k = 0");
-
-        assertRows(execute("SELECT m FROM %s WHERE k = 0"),
-            row((Object)null)
-        );
-    }
-
-    @Test
-    public void testLists() throws Throwable
-    {
-        createTable("CREATE TABLE %s (k int PRIMARY KEY, l list<text>)");
-
-        execute("INSERT INTO %s(k, l) VALUES (0, ?)", list("v1", "v2", "v3"));
-
-        assertRows(execute("SELECT l FROM %s WHERE k = 0"), row(list("v1", "v2", "v3")));
-
-        execute("DELETE l[?] FROM %s WHERE k = 0", 1);
-
-        assertRows(execute("SELECT l FROM %s WHERE k = 0"), row(list("v1", "v3")));
-
-        execute("UPDATE %s SET l[?] = ? WHERE k = 0", 1, "v4");
-
-        assertRows(execute("SELECT l FROM %s WHERE k = 0"), row(list("v1", "v4")));
-
-        // Full overwrite
-        execute("UPDATE %s SET l = ? WHERE k = 0", list("v6", "v5"));
-
-        assertRows(execute("SELECT l FROM %s WHERE k = 0"), row(list("v6", "v5")));
-
-        execute("UPDATE %s SET l = l + ? WHERE k = 0", list("v7", "v8"));
-
-        assertRows(execute("SELECT l FROM %s WHERE k = 0"), row(list("v6", "v5", "v7", "v8")));
-
-        execute("UPDATE %s SET l = ? + l WHERE k = 0", list("v9"));
-
-        assertRows(execute("SELECT l FROM %s WHERE k = 0"), row(list("v9", "v6", "v5", "v7", "v8")));
-
-        execute("UPDATE %s SET l = l - ? WHERE k = 0", list("v5", "v8"));
-
-        assertRows(execute("SELECT l FROM %s WHERE k = 0"), row(list("v9", "v6", "v7")));
-
-        execute("DELETE l FROM %s WHERE k = 0");
-
-        assertRows(execute("SELECT l FROM %s WHERE k = 0"), row((Object) null));
-
-        assertInvalidMessage("Attempted to delete an element from a list which is null",
-                             "DELETE l[0] FROM %s WHERE k=0 ");
-
-        assertInvalidMessage("Attempted to set an element on a list which is null",
-                             "UPDATE %s SET l[0] = ? WHERE k=0", list("v10"));
-
-        execute("UPDATE %s SET l = l - ? WHERE k=0 ", list("v11"));
-
-        assertRows(execute("SELECT l FROM %s WHERE k = 0"), row((Object) null));
-    }
-
-    @Test
-    public void testMapWithUnsetValues() throws Throwable
-    {
-        createTable("CREATE TABLE %s (k int PRIMARY KEY, m map<text,text>)");
-        // set up
-        Object m = map("k", "v");
-        execute("INSERT INTO %s (k, m) VALUES (10, ?)", m);
-        assertRows(execute("SELECT m FROM %s WHERE k = 10"),
-                row(m)
-        );
-
-        // test putting an unset map, should not delete the contents
-        execute("INSERT INTO %s (k, m) VALUES (10, ?)", unset());
-        assertRows(execute("SELECT m FROM %s WHERE k = 10"),
-                row(m)
-        );
-        // test unset variables in a map update operaiotn, should not delete the contents
-        execute("UPDATE %s SET m['k'] = ? WHERE k = 10", unset());
-        assertRows(execute("SELECT m FROM %s WHERE k = 10"),
-                row(m)
-        );
-        assertInvalidMessage("Invalid unset map key", "UPDATE %s SET m[?] = 'foo' WHERE k = 10", unset());
-
-        // test unset value for map key
-        assertInvalidMessage("Invalid unset map key", "DELETE m[?] FROM %s WHERE k = 10", unset());
-    }
-
-    @Test
-    public void testListWithUnsetValues() throws Throwable
-    {
-        createTable("CREATE TABLE %s (k int PRIMARY KEY, l list<text>)");
-        // set up
-        Object l = list("foo", "foo");
-        execute("INSERT INTO %s (k, l) VALUES (10, ?)", l);
-        assertRows(execute("SELECT l FROM %s WHERE k = 10"),
-                row(l)
-        );
-
-        // replace list with unset value
-        execute("INSERT INTO %s (k, l) VALUES (10, ?)", unset());
-        assertRows(execute("SELECT l FROM %s WHERE k = 10"),
-                row(l)
-        );
-
-        // add to position
-        execute("UPDATE %s SET l[1] = ? WHERE k = 10", unset());
-        assertRows(execute("SELECT l FROM %s WHERE k = 10"),
-                row(l)
-        );
-
-        // set in index
-        assertInvalidMessage("Invalid unset value for list index", "UPDATE %s SET l[?] = 'foo' WHERE k = 10", unset());
-
-        // remove element by index
-        execute("DELETE l[?] FROM %s WHERE k = 10", unset());
-        assertRows(execute("SELECT l FROM %s WHERE k = 10"),
-                row(l)
-        );
-
-        // remove all occurrences of element
-        execute("UPDATE %s SET l = l - ? WHERE k = 10", unset());
-        assertRows(execute("SELECT l FROM %s WHERE k = 10"),
-                row(l)
-        );
-
-        // select with in clause
-        assertInvalidMessage("Invalid unset value for column k", "SELECT * FROM %s WHERE k IN ?", unset());
-        assertInvalidMessage("Invalid unset value for column k", "SELECT * FROM %s WHERE k IN (?)", unset());
-    }
-
-    @Test
-    public void testSetWithUnsetValues() throws Throwable
-    {
-        createTable("CREATE TABLE %s (k int PRIMARY KEY, s set<text>)");
-
-        Object s = set("bar", "baz", "foo");
-        execute("INSERT INTO %s (k, s) VALUES (10, ?)", s);
-        assertRows(execute("SELECT s FROM %s WHERE k = 10"),
-                row(s)
-        );
-
-        // replace set with unset value
-        execute("INSERT INTO %s (k, s) VALUES (10, ?)", unset());
-        assertRows(execute("SELECT s FROM %s WHERE k = 10"),
-                row(s)
-        );
-
-        // add to set
-        execute("UPDATE %s SET s = s + ? WHERE k = 10", unset());
-        assertRows(execute("SELECT s FROM %s WHERE k = 10"),
-                row(s)
-        );
-
-        // remove all occurrences of element
-        execute("UPDATE %s SET s = s - ? WHERE k = 10", unset());
-        assertRows(execute("SELECT s FROM %s WHERE k = 10"),
-                row(s)
-        );
-    }
-}

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

http://git-wip-us.apache.org/repos/asf/cassandra/blob/01115f72/test/unit/org/apache/cassandra/cql3/CrcCheckChanceTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/CrcCheckChanceTest.java b/test/unit/org/apache/cassandra/cql3/CrcCheckChanceTest.java
deleted file mode 100644
index ac3ffbc..0000000
--- a/test/unit/org/apache/cassandra/cql3/CrcCheckChanceTest.java
+++ /dev/null
@@ -1,159 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.cassandra.cql3;
-
-import java.util.List;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.Future;
-
-import junit.framework.Assert;
-import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.db.ColumnFamilyStore;
-import org.apache.cassandra.db.Keyspace;
-import org.apache.cassandra.db.compaction.CompactionInterruptedException;
-import org.apache.cassandra.db.compaction.CompactionManager;
-import org.apache.cassandra.utils.FBUtilities;
-
-import org.junit.Test;
-
-
-public class CrcCheckChanceTest extends CQLTester
-{
-    @Test
-    public void testChangingCrcCheckChance() throws Throwable
-    {
-        //Start with crc_check_chance of 99%
-        createTable("CREATE TABLE %s (p text, c text, v text, s text static, PRIMARY KEY (p, c)) WITH compression = {'sstable_compression': 'LZ4Compressor', 'crc_check_chance' : 0.99}");
-
-        execute("CREATE INDEX foo ON %s(v)");
-
-        execute("INSERT INTO %s(p, c, v, s) values (?, ?, ?, ?)", "p1", "k1", "v1", "sv1");
-        execute("INSERT INTO %s(p, c, v) values (?, ?, ?)", "p1", "k2", "v2");
-        execute("INSERT INTO %s(p, s) values (?, ?)", "p2", "sv2");
-
-
-        ColumnFamilyStore cfs = Keyspace.open(CQLTester.KEYSPACE).getColumnFamilyStore(currentTable());
-        ColumnFamilyStore indexCfs = cfs.indexManager.getIndexesBackedByCfs().iterator().next();
-        cfs.forceBlockingFlush();
-
-        Assert.assertEquals(0.99, cfs.metadata.compressionParameters.getCrcCheckChance());
-        Assert.assertEquals(0.99, cfs.getSSTables().iterator().next().getCompressionMetadata().parameters.getCrcCheckChance());
-        Assert.assertEquals(0.99, indexCfs.metadata.compressionParameters.getCrcCheckChance());
-        Assert.assertEquals(0.99, indexCfs.getSSTables().iterator().next().getCompressionMetadata().parameters.getCrcCheckChance());
-
-        //Test for stack overflow
-        cfs.setCrcCheckChance(0.99);
-
-        assertRows(execute("SELECT * FROM %s WHERE p=?", "p1"),
-                row("p1", "k1", "sv1", "v1"),
-                row("p1", "k2", "sv1", "v2")
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE v=?", "v1"),
-                row("p1", "k1", "sv1", "v1")
-        );
-
-
-
-        //Write a few SSTables then Compact
-
-        execute("INSERT INTO %s(p, c, v, s) values (?, ?, ?, ?)", "p1", "k1", "v1", "sv1");
-        execute("INSERT INTO %s(p, c, v) values (?, ?, ?)", "p1", "k2", "v2");
-        execute("INSERT INTO %s(p, s) values (?, ?)", "p2", "sv2");
-
-        cfs.forceBlockingFlush();
-
-
-        execute("INSERT INTO %s(p, c, v, s) values (?, ?, ?, ?)", "p1", "k1", "v1", "sv1");
-        execute("INSERT INTO %s(p, c, v) values (?, ?, ?)", "p1", "k2", "v2");
-        execute("INSERT INTO %s(p, s) values (?, ?)", "p2", "sv2");
-
-        cfs.forceBlockingFlush();
-
-        execute("INSERT INTO %s(p, c, v, s) values (?, ?, ?, ?)", "p1", "k1", "v1", "sv1");
-        execute("INSERT INTO %s(p, c, v) values (?, ?, ?)", "p1", "k2", "v2");
-        execute("INSERT INTO %s(p, s) values (?, ?)", "p2", "sv2");
-
-        cfs.forceBlockingFlush();
-
-        cfs.forceMajorCompaction();
-
-        //Verify when we alter the value the live sstable readers hold the new one
-        alterTable("ALTER TABLE %s WITH compression = {'sstable_compression': 'LZ4Compressor', 'crc_check_chance': 0.01}");
-
-        Assert.assertEquals( 0.01, cfs.metadata.compressionParameters.getCrcCheckChance());
-        Assert.assertEquals( 0.01, cfs.getSSTables().iterator().next().getCompressionMetadata().parameters.getCrcCheckChance());
-        Assert.assertEquals( 0.01, indexCfs.metadata.compressionParameters.getCrcCheckChance());
-        Assert.assertEquals( 0.01, indexCfs.getSSTables().iterator().next().getCompressionMetadata().parameters.getCrcCheckChance());
-
-        assertRows(execute("SELECT * FROM %s WHERE p=?", "p1"),
-                row("p1", "k1", "sv1", "v1"),
-                row("p1", "k2", "sv1", "v2")
-        );
-
-        assertRows(execute("SELECT * FROM %s WHERE v=?", "v1"),
-                row("p1", "k1", "sv1", "v1")
-        );
-
-
-        //Verify the call used by JMX still works
-        cfs.setCrcCheckChance(0.03);
-        Assert.assertEquals( 0.03, cfs.metadata.compressionParameters.getCrcCheckChance());
-        Assert.assertEquals( 0.03, cfs.getSSTables().iterator().next().getCompressionMetadata().parameters.getCrcCheckChance());
-        Assert.assertEquals( 0.03, indexCfs.metadata.compressionParameters.getCrcCheckChance());
-        Assert.assertEquals( 0.03, indexCfs.getSSTables().iterator().next().getCompressionMetadata().parameters.getCrcCheckChance());
-
-    }
-
-
-    @Test
-    public void testDropDuringCompaction() throws Throwable
-    {
-        CompactionManager.instance.disableAutoCompaction();
-
-        //Start with crc_check_chance of 99%
-        createTable("CREATE TABLE %s (p text, c text, v text, s text static, PRIMARY KEY (p, c)) WITH compression = {'sstable_compression': 'LZ4Compressor', 'crc_check_chance' : 0.99}");
-
-        ColumnFamilyStore cfs = Keyspace.open(CQLTester.KEYSPACE).getColumnFamilyStore(currentTable());
-
-        //Write a few SSTables then Compact, and drop
-        for (int i = 0; i < 100; i++)
-        {
-            execute("INSERT INTO %s(p, c, v, s) values (?, ?, ?, ?)", "p1", "k1", "v1", "sv1");
-            execute("INSERT INTO %s(p, c, v) values (?, ?, ?)", "p1", "k2", "v2");
-            execute("INSERT INTO %s(p, s) values (?, ?)", "p2", "sv2");
-
-            cfs.forceBlockingFlush();
-        }
-
-        DatabaseDescriptor.setCompactionThroughputMbPerSec(1);
-        List<Future<?>> futures = CompactionManager.instance.submitMaximal(cfs, CompactionManager.getDefaultGcBefore(cfs), false); 
-        execute("DROP TABLE %s");
-
-        try
-        {
-            FBUtilities.waitOnFutures(futures);
-        }
-        catch (Throwable t)
-        {
-            if (!(t.getCause() instanceof ExecutionException) || !(t.getCause().getCause() instanceof CompactionInterruptedException))
-                throw t;
-        }
-    }
-}
-

http://git-wip-us.apache.org/repos/asf/cassandra/blob/01115f72/test/unit/org/apache/cassandra/cql3/CreateAndAlterKeyspaceTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/CreateAndAlterKeyspaceTest.java b/test/unit/org/apache/cassandra/cql3/CreateAndAlterKeyspaceTest.java
deleted file mode 100644
index 9e0ca21..0000000
--- a/test/unit/org/apache/cassandra/cql3/CreateAndAlterKeyspaceTest.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.cassandra.cql3;
-
-import org.junit.Test;
-
-public class CreateAndAlterKeyspaceTest extends CQLTester
-{
-    @Test
-    // tests CASSANDRA-9565
-    public void testCreateAndAlterWithDoubleWith() throws Throwable
-    {
-        String[] stmts = new String[] {"ALTER KEYSPACE WITH WITH DURABLE_WRITES = true",
-                                       "ALTER KEYSPACE ks WITH WITH DURABLE_WRITES = true",
-                                       "CREATE KEYSPACE WITH WITH DURABLE_WRITES = true",
-                                       "CREATE KEYSPACE ks WITH WITH DURABLE_WRITES = true"};
-
-        for (String stmt : stmts) {
-            assertInvalidSyntaxMessage("no viable alternative at input 'WITH'", stmt);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/01115f72/test/unit/org/apache/cassandra/cql3/CreateIndexStatementTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/CreateIndexStatementTest.java b/test/unit/org/apache/cassandra/cql3/CreateIndexStatementTest.java
deleted file mode 100644
index 1e2e084..0000000
--- a/test/unit/org/apache/cassandra/cql3/CreateIndexStatementTest.java
+++ /dev/null
@@ -1,101 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.cassandra.cql3;
-
-import java.util.Locale;
-
-import org.apache.commons.lang3.StringUtils;
-
-import org.junit.Test;
-
-public class CreateIndexStatementTest extends CQLTester
-{
-    @Test
-    public void testCreateAndDropIndex() throws Throwable
-    {
-        testCreateAndDropIndex("test", false);
-        testCreateAndDropIndex("test2", true);
-    }
-
-    @Test
-    public void testCreateAndDropIndexWithQuotedIdentifier() throws Throwable
-    {
-        testCreateAndDropIndex("\"quoted_ident\"", false);
-        testCreateAndDropIndex("\"quoted_ident2\"", true);
-    }
-
-    @Test
-    public void testCreateAndDropIndexWithCamelCaseIdentifier() throws Throwable
-    {
-        testCreateAndDropIndex("CamelCase", false);
-        testCreateAndDropIndex("CamelCase2", true);
-    }
-
-    /**
-     * Test creating and dropping an index with the specified name.
-     *
-     * @param indexName the index name
-     * @param addKeyspaceOnDrop add the keyspace name in the drop statement
-     * @throws Throwable if an error occurs
-     */
-    private void testCreateAndDropIndex(String indexName, boolean addKeyspaceOnDrop) throws Throwable
-    {
-        execute("USE system");
-        assertInvalidMessage("Index '" + removeQuotes(indexName.toLowerCase(Locale.US)) + "' could not be found", "DROP INDEX " + indexName + ";");
-
-        createTable("CREATE TABLE %s (a int primary key, b int);");
-        createIndex("CREATE INDEX " + indexName + " ON %s(b);");
-        createIndex("CREATE INDEX IF NOT EXISTS " + indexName + " ON %s(b);");
-
-        assertInvalidMessage("Index already exists", "CREATE INDEX " + indexName + " ON %s(b)");
-
-        execute("INSERT INTO %s (a, b) values (?, ?);", 0, 0);
-        execute("INSERT INTO %s (a, b) values (?, ?);", 1, 1);
-        execute("INSERT INTO %s (a, b) values (?, ?);", 2, 2);
-        execute("INSERT INTO %s (a, b) values (?, ?);", 3, 1);
-
-        assertRows(execute("SELECT * FROM %s where b = ?", 1), row(1, 1), row(3, 1));
-        assertInvalidMessage("Index '" + removeQuotes(indexName.toLowerCase(Locale.US)) + "' could not be found in any of the tables of keyspace 'system'", "DROP INDEX " + indexName);
-
-        if (addKeyspaceOnDrop)
-        {
-            dropIndex("DROP INDEX " + KEYSPACE + "." + indexName);
-        }
-        else
-        {
-            execute("USE " + KEYSPACE);
-            execute("DROP INDEX " + indexName);
-        }
-
-        assertInvalidMessage("No secondary indexes on the restricted columns support the provided operators",
-                             "SELECT * FROM %s where b = ?", 1);
-        dropIndex("DROP INDEX IF EXISTS " + indexName);
-        assertInvalidMessage("Index '" + removeQuotes(indexName.toLowerCase(Locale.US)) + "' could not be found", "DROP INDEX " + indexName);
-    }
-
-    /**
-     * Removes the quotes from the specified index name.
-     *
-     * @param indexName the index name from which the quotes must be removed.
-     * @return the unquoted index name.
-     */
-    private static String removeQuotes(String indexName)
-    {
-        return StringUtils.remove(indexName, '\"');
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/01115f72/test/unit/org/apache/cassandra/cql3/CreateTableTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/CreateTableTest.java b/test/unit/org/apache/cassandra/cql3/CreateTableTest.java
deleted file mode 100644
index d14e87b..0000000
--- a/test/unit/org/apache/cassandra/cql3/CreateTableTest.java
+++ /dev/null
@@ -1,69 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.cassandra.cql3;
-
-import org.junit.Test;
-
-import org.apache.cassandra.utils.ByteBufferUtil;
-
-import static junit.framework.Assert.assertFalse;
-
-public class CreateTableTest extends CQLTester
-{
-    @Test
-    public void testCQL3PartitionKeyOnlyTable()
-    {
-        createTable("CREATE TABLE %s (id text PRIMARY KEY);");
-        assertFalse(currentTableMetadata().isThriftCompatible());
-    }
-
-    @Test
-    public void testCreateTableWithSmallintColumns() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a text, b smallint, c smallint, primary key (a, b));");
-        execute("INSERT INTO %s (a, b, c) VALUES ('1', 1, 2)");
-        execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", "2", Short.MAX_VALUE, Short.MIN_VALUE);
-
-        assertRows(execute("SELECT * FROM %s"),
-                   row("1", (short) 1, (short) 2),
-                   row("2", Short.MAX_VALUE, Short.MIN_VALUE));
-
-        assertInvalidMessage("Expected 2 bytes for a smallint (4)",
-                             "INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", "3", 1, 2);
-        assertInvalidMessage("Expected 2 bytes for a smallint (0)",
-                             "INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", "3", (short) 1, ByteBufferUtil.EMPTY_BYTE_BUFFER);
-     }
-
-    @Test
-    public void testCreateTinyintColumns() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a text, b tinyint, c tinyint, primary key (a, b));");
-        execute("INSERT INTO %s (a, b, c) VALUES ('1', 1, 2)");
-        execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", "2", Byte.MAX_VALUE, Byte.MIN_VALUE);
-
-        assertRows(execute("SELECT * FROM %s"),
-                   row("1", (byte) 1, (byte) 2),
-                   row("2", Byte.MAX_VALUE, Byte.MIN_VALUE));
-
-        assertInvalidMessage("Expected 1 byte for a tinyint (4)",
-                             "INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", "3", 1, 2);
-
-        assertInvalidMessage("Expected 1 byte for a tinyint (0)",
-                             "INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", "3", (byte) 1, ByteBufferUtil.EMPTY_BYTE_BUFFER);
-     }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/01115f72/test/unit/org/apache/cassandra/cql3/CreateTriggerStatementTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/CreateTriggerStatementTest.java b/test/unit/org/apache/cassandra/cql3/CreateTriggerStatementTest.java
deleted file mode 100644
index 6557c16..0000000
--- a/test/unit/org/apache/cassandra/cql3/CreateTriggerStatementTest.java
+++ /dev/null
@@ -1,121 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.cassandra.cql3;
-
-import java.nio.ByteBuffer;
-import java.util.Collection;
-import java.util.Collections;
-
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.Schema;
-import org.apache.cassandra.config.TriggerDefinition;
-import org.apache.cassandra.db.ColumnFamily;
-import org.apache.cassandra.db.Mutation;
-import org.apache.cassandra.triggers.ITrigger;
-import org.junit.Test;
-
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-public class CreateTriggerStatementTest extends CQLTester
-{
-    @Test
-    public void testCreateTrigger() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a int, b int, c int, PRIMARY KEY (a))");
-        execute("CREATE TRIGGER trigger_1 ON %s USING '" + TestTrigger.class.getName() + "'");
-        assertTriggerExists("trigger_1", TestTrigger.class);
-        execute("CREATE TRIGGER trigger_2 ON %s USING '" + TestTrigger.class.getName() + "'");
-        assertTriggerExists("trigger_2", TestTrigger.class);
-        assertInvalid("CREATE TRIGGER trigger_1 ON %s USING '" + TestTrigger.class.getName() + "'");
-        execute("CREATE TRIGGER \"Trigger 3\" ON %s USING '" + TestTrigger.class.getName() + "'");
-        assertTriggerExists("Trigger 3", TestTrigger.class);
-    }
-
-    @Test
-    public void testCreateTriggerIfNotExists() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a int, b int, c int, PRIMARY KEY (a, b))");
-
-        execute("CREATE TRIGGER IF NOT EXISTS trigger_1 ON %s USING '" + TestTrigger.class.getName() + "'");
-        assertTriggerExists("trigger_1", TestTrigger.class);
-
-        execute("CREATE TRIGGER IF NOT EXISTS trigger_1 ON %s USING '" + TestTrigger.class.getName() + "'");
-        assertTriggerExists("trigger_1", TestTrigger.class);
-    }
-
-    @Test
-    public void testDropTrigger() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a int, b int, c int, PRIMARY KEY (a))");
-
-        execute("CREATE TRIGGER trigger_1 ON %s USING '" + TestTrigger.class.getName() + "'");
-        assertTriggerExists("trigger_1", TestTrigger.class);
-
-        execute("DROP TRIGGER trigger_1 ON %s");
-        assertTriggerDoesNotExists("trigger_1", TestTrigger.class);
-
-        execute("CREATE TRIGGER trigger_1 ON %s USING '" + TestTrigger.class.getName() + "'");
-        assertTriggerExists("trigger_1", TestTrigger.class);
-
-        assertInvalid("DROP TRIGGER trigger_2 ON %s");
-        
-        execute("CREATE TRIGGER \"Trigger 3\" ON %s USING '" + TestTrigger.class.getName() + "'");
-        assertTriggerExists("Trigger 3", TestTrigger.class);
-
-        execute("DROP TRIGGER \"Trigger 3\" ON %s");
-        assertTriggerDoesNotExists("Trigger 3", TestTrigger.class);
-    }
-
-    @Test
-    public void testDropTriggerIfExists() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a int, b int, c int, PRIMARY KEY (a))");
-
-        execute("DROP TRIGGER IF EXISTS trigger_1 ON %s");
-        assertTriggerDoesNotExists("trigger_1", TestTrigger.class);
-
-        execute("CREATE TRIGGER trigger_1 ON %s USING '" + TestTrigger.class.getName() + "'");
-        assertTriggerExists("trigger_1", TestTrigger.class);
-
-        execute("DROP TRIGGER IF EXISTS trigger_1 ON %s");
-        assertTriggerDoesNotExists("trigger_1", TestTrigger.class);
-    }
-
-    private void assertTriggerExists(String name, Class<?> clazz)
-    {
-        CFMetaData cfm = Schema.instance.getCFMetaData(keyspace(), currentTable()).copy();
-        assertTrue("the trigger does not exist", cfm.containsTriggerDefinition(TriggerDefinition.create(name,
-                clazz.getName())));
-    }
-
-    private void assertTriggerDoesNotExists(String name, Class<?> clazz)
-    {
-        CFMetaData cfm = Schema.instance.getCFMetaData(keyspace(), currentTable()).copy();
-        assertFalse("the trigger exists", cfm.containsTriggerDefinition(TriggerDefinition.create(name,
-                clazz.getName())));
-    }
-
-    public static class TestTrigger implements ITrigger
-    {
-        public Collection<Mutation> augment(ByteBuffer key, ColumnFamily update)
-        {
-            return Collections.emptyList();
-        }
-    }
-}