You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by if...@apache.org on 2017/11/06 15:50:35 UTC

[25/25] cassandra git commit: Disallow COMPACT STORAGE syntax, avoid starting when compact tables are present

Disallow COMPACT STORAGE syntax, avoid starting when compact tables are present

Patch by Alex Petrov; reviewed by Sylvain Lebresne for CASSANDRA-10857

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

Branch: refs/heads/trunk
Commit: 07fbd8ee6042797aaade90357d625ba9d79c31e0
Parents: 38bd20e
Author: Alex Petrov <ol...@gmail.com>
Authored: Tue Oct 17 11:17:41 2017 +0200
Committer: Alex Petrov <ol...@gmail.com>
Committed: Mon Nov 6 16:44:50 2017 +0100

----------------------------------------------------------------------
 NEWS.txt                                        |    4 +
 src/antlr/Parser.g                              |    2 +-
 .../org/apache/cassandra/db/SystemKeyspace.java |    4 +-
 .../index/internal/keys/KeysSearcher.java       |    1 -
 .../apache/cassandra/schema/SchemaKeyspace.java |    9 +-
 .../apache/cassandra/schema/TableMetadata.java  |   32 +-
 .../cassandra/service/CassandraDaemon.java      |   13 +-
 .../unit/org/apache/cassandra/SchemaLoader.java |   18 -
 .../cassandra/cql3/LargeCompactValueTest.java   |   48 -
 .../apache/cassandra/cql3/SimpleQueryTest.java  |   45 -
 .../validation/entities/CollectionsTest.java    |   10 -
 .../cql3/validation/entities/CountersTest.java  |  259 +-
 .../entities/FrozenCollectionsTest.java         |  503 ++-
 .../validation/entities/SecondaryIndexTest.java |   97 -
 .../validation/miscellaneous/OverflowTest.java  |   30 -
 .../validation/operations/AggregationTest.java  |   20 -
 .../cql3/validation/operations/BatchTest.java   |   33 -
 .../cql3/validation/operations/CreateTest.java  |  227 +-
 .../cql3/validation/operations/DeleteTest.java  | 1057 +++---
 .../cql3/validation/operations/InsertTest.java  |  101 -
 .../operations/InsertUpdateIfConditionTest.java |   18 -
 .../operations/SelectGroupByTest.java           | 1600 +++++----
 .../validation/operations/SelectLimitTest.java  |  134 +-
 .../SelectMultiColumnRelationTest.java          | 3026 +++++++++---------
 .../operations/SelectOrderByTest.java           |  206 +-
 .../SelectOrderedPartitionerTest.java           |   13 -
 .../SelectSingleColumnRelationTest.java         |   51 +-
 .../cql3/validation/operations/SelectTest.java  | 2366 ++------------
 .../cql3/validation/operations/UpdateTest.java  |  548 ++--
 .../db/ColumnFamilyStoreCQLHelperTest.java      |  226 --
 .../unit/org/apache/cassandra/db/ScrubTest.java |   27 -
 .../db/partition/PartitionUpdateTest.java       |   14 -
 .../index/SecondaryIndexManagerTest.java        |    2 +-
 .../index/internal/CassandraIndexTest.java      |   19 +-
 .../cassandra/io/sstable/LegacySSTableTest.java |  185 +-
 .../stress/settings/SettingsSchema.java         |    4 +-
 36 files changed, 3893 insertions(+), 7059 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/07fbd8ee/NEWS.txt
----------------------------------------------------------------------
diff --git a/NEWS.txt b/NEWS.txt
index 30cad13..de7d58a 100644
--- a/NEWS.txt
+++ b/NEWS.txt
@@ -33,6 +33,10 @@ New features
 
 Upgrading
 ---------
+    - Cassandra 4.0 removed support for COMPACT STORAGE tables. All Compact Tables
+      have to be migrated using `ALTER ... DROP COMPACT STORAGE` statement in 3.0/3.11.
+      Cassandra starting 4.0 will not start if flags indicate that the table is non-CQL.
+      Syntax for creating compact tables is also deprecated.
     - Support for legacy auth tables in the system_auth keyspace (users,
       permissions, credentials) and the migration code has been removed. Migration
       of these legacy auth tables must have been completed before the upgrade to

http://git-wip-us.apache.org/repos/asf/cassandra/blob/07fbd8ee/src/antlr/Parser.g
----------------------------------------------------------------------
diff --git a/src/antlr/Parser.g b/src/antlr/Parser.g
index 553d643..d6ea210 100644
--- a/src/antlr/Parser.g
+++ b/src/antlr/Parser.g
@@ -793,7 +793,7 @@ pkDef[CreateTableStatement.RawStatement expr]
 
 cfamProperty[CFProperties props]
     : property[props.properties]
-    | K_COMPACT K_STORAGE { $props.setCompactStorage(); }
+    | K_COMPACT K_STORAGE { throw new SyntaxException("Compact tables are not allowed in Cassandra starting with 4.0 version."); }
     | K_CLUSTERING K_ORDER K_BY '(' cfamOrdering[props] (',' cfamOrdering[props])* ')'
     ;
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/07fbd8ee/src/java/org/apache/cassandra/db/SystemKeyspace.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/SystemKeyspace.java b/src/java/org/apache/cassandra/db/SystemKeyspace.java
index 6b278a5..2ffae11 100644
--- a/src/java/org/apache/cassandra/db/SystemKeyspace.java
+++ b/src/java/org/apache/cassandra/db/SystemKeyspace.java
@@ -139,8 +139,8 @@ public final class SystemKeyspace
               "CREATE TABLE \"%s\" ("
               + "table_name text," // table_name here is the name of the keyspace - don't be fooled
               + "index_name text,"
-              + "PRIMARY KEY ((table_name), index_name)) "
-              + "WITH COMPACT STORAGE")
+              + "value blob," // Table used to be compact in previous versions
+              + "PRIMARY KEY ((table_name), index_name)) ")
               .build();
 
     private static final TableMetadata Local =

http://git-wip-us.apache.org/repos/asf/cassandra/blob/07fbd8ee/src/java/org/apache/cassandra/index/internal/keys/KeysSearcher.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/internal/keys/KeysSearcher.java b/src/java/org/apache/cassandra/index/internal/keys/KeysSearcher.java
index 0f4fcb2..16dcd20 100644
--- a/src/java/org/apache/cassandra/index/internal/keys/KeysSearcher.java
+++ b/src/java/org/apache/cassandra/index/internal/keys/KeysSearcher.java
@@ -145,7 +145,6 @@ public class KeysSearcher extends CassandraIndexSearcher
                                                 OpOrder.Group writeOp,
                                                 int nowInSec)
     {
-        assert iterator.metadata().isCompactTable();
         Row data = iterator.staticRow();
         if (index.isStale(data, indexedValue, nowInSec))
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/07fbd8ee/src/java/org/apache/cassandra/schema/SchemaKeyspace.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/schema/SchemaKeyspace.java b/src/java/org/apache/cassandra/schema/SchemaKeyspace.java
index 460fb9f..cd03a40 100644
--- a/src/java/org/apache/cassandra/schema/SchemaKeyspace.java
+++ b/src/java/org/apache/cassandra/schema/SchemaKeyspace.java
@@ -969,8 +969,15 @@ public final class SchemaKeyspace
             throw new RuntimeException(String.format("%s:%s not found in the schema definitions keyspace.", keyspaceName, tableName));
         UntypedResultSet.Row row = rows.one();
 
+        Set<TableMetadata.Flag> flags = TableMetadata.Flag.fromStringSet(row.getFrozenSet("flags", UTF8Type.instance));
+
+        if (!TableMetadata.Flag.isCQLCompatible(flags))
+        {
+            throw new IllegalArgumentException(TableMetadata.COMPACT_STORAGE_HALT_MESSAGE);
+        }
+
         return TableMetadata.builder(keyspaceName, tableName, TableId.fromUUID(row.getUUID("id")))
-                            .flags(TableMetadata.Flag.fromStringSet(row.getFrozenSet("flags", UTF8Type.instance)))
+                            .flags(flags)
                             .params(createTableParamsFromRow(row))
                             .addColumns(fetchColumns(keyspaceName, tableName, types))
                             .droppedColumns(fetchDroppedColumns(keyspaceName, tableName))

http://git-wip-us.apache.org/repos/asf/cassandra/blob/07fbd8ee/src/java/org/apache/cassandra/schema/TableMetadata.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/schema/TableMetadata.java b/src/java/org/apache/cassandra/schema/TableMetadata.java
index cf4d6bd..206a69d 100644
--- a/src/java/org/apache/cassandra/schema/TableMetadata.java
+++ b/src/java/org/apache/cassandra/schema/TableMetadata.java
@@ -24,6 +24,9 @@ import java.util.Objects;
 import com.google.common.base.MoreObjects;
 import com.google.common.collect.*;
 
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 import org.apache.cassandra.auth.DataResource;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.cql3.ColumnIdentifier;
@@ -44,10 +47,29 @@ import static org.apache.cassandra.schema.IndexMetadata.isNameValid;
 @Unmetered
 public final class TableMetadata
 {
+    private static final Logger logger = LoggerFactory.getLogger(TableMetadata.class);
+    private static final ImmutableSet<Flag> DEFAULT_CQL_FLAGS = ImmutableSet.of(Flag.COMPOUND);
+    private static final ImmutableSet<Flag> DEPRECATED_CS_FLAGS = ImmutableSet.of(Flag.DENSE, Flag.SUPER);
+
+    public static final String COMPACT_STORAGE_HALT_MESSAGE =
+             "Compact Tables are not allowed in Cassandra starting with 4.0 version. " +
+             "Use `ALTER ... DROP COMPACT STORAGE` command supplied in 3.x/3.11 Cassandra " +
+             "in order to migrate off Compact Storage.";
+
+    private static final String COMPACT_STORAGE_DEPRECATION_MESSAGE =
+             "Incorrect set of flags is was detected in table {}.{}: '{}'. \n" +
+             "Starting with version 4.0, '{}' flags are deprecated and every table has to have COMPOUND flag. \n" +
+             "Forcing the following set of flags: '{}'";
+
     public enum Flag
     {
         SUPER, COUNTER, DENSE, COMPOUND;
 
+        public static boolean isCQLCompatible(Set<Flag> flags)
+        {
+            return !flags.contains(Flag.DENSE) && !flags.contains(Flag.SUPER) && flags.contains(Flag.COMPOUND);
+        }
+
         public static Set<Flag> fromStringSet(Set<String> strings)
         {
             return strings.stream().map(String::toUpperCase).map(Flag::valueOf).collect(toSet());
@@ -102,13 +124,21 @@ public final class TableMetadata
 
     private TableMetadata(Builder builder)
     {
+        if (!Flag.isCQLCompatible(builder.flags))
+        {
+            flags = ImmutableSet.copyOf(Sets.union(Sets.difference(builder.flags, DEPRECATED_CS_FLAGS), DEFAULT_CQL_FLAGS));
+            logger.warn(COMPACT_STORAGE_DEPRECATION_MESSAGE, builder.keyspace, builder.name,  builder.flags, DEPRECATED_CS_FLAGS, flags);
+        }
+        else
+        {
+            flags = Sets.immutableEnumSet(builder.flags);
+        }
         keyspace = builder.keyspace;
         name = builder.name;
         id = builder.id;
 
         partitioner = builder.partitioner;
         params = builder.params.build();
-        flags = Sets.immutableEnumSet(builder.flags);
         isView = builder.isView;
 
         indexName = name.contains(".")

http://git-wip-us.apache.org/repos/asf/cassandra/blob/07fbd8ee/src/java/org/apache/cassandra/service/CassandraDaemon.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/CassandraDaemon.java b/src/java/org/apache/cassandra/service/CassandraDaemon.java
index 88d812d..d8309b4 100644
--- a/src/java/org/apache/cassandra/service/CassandraDaemon.java
+++ b/src/java/org/apache/cassandra/service/CassandraDaemon.java
@@ -234,8 +234,17 @@ public class CassandraDaemon
         // Populate token metadata before flushing, for token-aware sstable partitioning (#6696)
         StorageService.instance.populateTokenMetadata();
 
-        // load schema from disk
-        Schema.instance.loadFromDisk();
+        try
+        {
+            // load schema from disk
+            Schema.instance.loadFromDisk();
+        }
+        catch (Exception e)
+        {
+            logger.error("Error while loading schema: ", e);
+            throw e;
+        }
+
 
         // clean up debris in the rest of the keyspaces
         for (String keyspaceName : Schema.instance.getKeyspaces())

http://git-wip-us.apache.org/repos/asf/cassandra/blob/07fbd8ee/test/unit/org/apache/cassandra/SchemaLoader.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/SchemaLoader.java b/test/unit/org/apache/cassandra/SchemaLoader.java
index d8f21e8..0633fb5 100644
--- a/test/unit/org/apache/cassandra/SchemaLoader.java
+++ b/test/unit/org/apache/cassandra/SchemaLoader.java
@@ -228,24 +228,6 @@ public class SchemaLoader
                                    + "c text,"
                                    + "v text,"
                                    + "PRIMARY KEY (k, c))", ks_cql)
-                            .build(),
-
-        CreateTableStatement.parse("CREATE TABLE foo ("
-                                   + "bar text, "
-                                   + "baz text, "
-                                   + "qux text, "
-                                   + "PRIMARY KEY(bar, baz) ) "
-                                   + "WITH COMPACT STORAGE", ks_cql)
-                            .build(),
-
-        CreateTableStatement.parse("CREATE TABLE foofoo ("
-                                   + "bar text, "
-                                   + "baz text, "
-                                   + "qux text, "
-                                   + "quz text, "
-                                   + "foo text, "
-                                   + "PRIMARY KEY((bar, baz), qux, quz) ) "
-                                   + "WITH COMPACT STORAGE", ks_cql)
                             .build()
         )));
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/07fbd8ee/test/unit/org/apache/cassandra/cql3/LargeCompactValueTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/LargeCompactValueTest.java b/test/unit/org/apache/cassandra/cql3/LargeCompactValueTest.java
deleted file mode 100644
index 93b16ce..0000000
--- a/test/unit/org/apache/cassandra/cql3/LargeCompactValueTest.java
+++ /dev/null
@@ -1,48 +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 org.junit.Before;
-import org.junit.Test;
-
-public class LargeCompactValueTest extends CQLTester
-{
-    @Before
-    public void before()
-    {
-        createTable("CREATE TABLE %s (key TEXT, column TEXT, value BLOB, PRIMARY KEY (key, column)) WITH COMPACT STORAGE");
-    }
-
-    @Test
-    public void testInsertAndQuery() throws Throwable
-    {
-        ByteBuffer largeBytes = ByteBuffer.wrap(new byte[100000]);
-        execute("INSERT INTO %s (key, column, value) VALUES (?, ?, ?)", "test", "a", largeBytes);
-        ByteBuffer smallBytes = ByteBuffer.wrap(new byte[10]);
-        execute("INSERT INTO %s (key, column, value) VALUES (?, ?, ?)", "test", "c", smallBytes);
-
-        flush();
-
-        assertRows(execute("SELECT column FROM %s WHERE key = ? AND column IN (?, ?, ?)", "test", "c", "a", "b"),
-                   row("a"),
-                   row("c"));
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/07fbd8ee/test/unit/org/apache/cassandra/cql3/SimpleQueryTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/SimpleQueryTest.java b/test/unit/org/apache/cassandra/cql3/SimpleQueryTest.java
index c2de52a..ce8c695 100644
--- a/test/unit/org/apache/cassandra/cql3/SimpleQueryTest.java
+++ b/test/unit/org/apache/cassandra/cql3/SimpleQueryTest.java
@@ -22,31 +22,6 @@ import org.junit.Test;
 public class SimpleQueryTest extends CQLTester
 {
     @Test
-    public void testStaticCompactTables() throws Throwable
-    {
-        createTable("CREATE TABLE %s (k text PRIMARY KEY, v1 int, v2 text) WITH COMPACT STORAGE");
-
-        execute("INSERT INTO %s (k, v1, v2) values (?, ?, ?)", "first", 1, "value1");
-        execute("INSERT INTO %s (k, v1, v2) values (?, ?, ?)", "second", 2, "value2");
-        execute("INSERT INTO %s (k, v1, v2) values (?, ?, ?)", "third", 3, "value3");
-
-        assertRows(execute("SELECT * FROM %s WHERE k = ?", "first"),
-            row("first", 1, "value1")
-        );
-
-        assertRows(execute("SELECT v2 FROM %s WHERE k = ?", "second"),
-            row("value2")
-        );
-
-        // Murmur3 order
-        assertRows(execute("SELECT * FROM %s"),
-            row("third",  3, "value3"),
-            row("second", 2, "value2"),
-            row("first",  1, "value1")
-        );
-    }
-
-    @Test
     public void testDynamicCompactTables() throws Throwable
     {
         createTable("CREATE TABLE %s (k text, t int, v text, PRIMARY KEY (k, t));");
@@ -502,26 +477,6 @@ public class SimpleQueryTest extends CQLTester
     }
 
     @Test
-    public void testCompactStorageUpdateWithNull() throws Throwable
-    {
-        createTable("CREATE TABLE %s (partitionKey int," +
-                "clustering_1 int," +
-                "value int," +
-                " PRIMARY KEY (partitionKey, clustering_1)) WITH COMPACT STORAGE");
-
-        execute("INSERT INTO %s (partitionKey, clustering_1, value) VALUES (0, 0, 0)");
-        execute("INSERT INTO %s (partitionKey, clustering_1, value) VALUES (0, 1, 1)");
-
-        flush();
-
-        execute("UPDATE %s SET value = ? WHERE partitionKey = ? AND clustering_1 = ?", null, 0, 0);
-
-        assertRows(execute("SELECT * FROM %s WHERE partitionKey = ? AND (clustering_1) IN ((?), (?))", 0, 0, 1),
-            row(0, 1, 1)
-        );
-    }
-
-    @Test
     public void test2ndaryIndexBug() throws Throwable
     {
         createTable("CREATE TABLE %s (k int, c1 int, c2 int, v int, PRIMARY KEY(k, c1, c2))");

http://git-wip-us.apache.org/repos/asf/cassandra/blob/07fbd8ee/test/unit/org/apache/cassandra/cql3/validation/entities/CollectionsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/CollectionsTest.java b/test/unit/org/apache/cassandra/cql3/validation/entities/CollectionsTest.java
index 6f67609..e26f207 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/entities/CollectionsTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/CollectionsTest.java
@@ -576,16 +576,6 @@ public class CollectionsTest extends CQLTester
     }
 
     /**
-     * Migrated from cql_tests.py:TestCQL.collection_compact_test()
-     */
-    @Test
-    public void testCompactCollections() throws Throwable
-    {
-        String tableName = KEYSPACE + "." + createTableName();
-        assertInvalid(String.format("CREATE TABLE %s (user ascii PRIMARY KEY, mails list < text >) WITH COMPACT STORAGE;", tableName));
-    }
-
-    /**
      * Migrated from cql_tests.py:TestCQL.collection_function_test()
      */
     @Test

http://git-wip-us.apache.org/repos/asf/cassandra/blob/07fbd8ee/test/unit/org/apache/cassandra/cql3/validation/entities/CountersTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/CountersTest.java b/test/unit/org/apache/cassandra/cql3/validation/entities/CountersTest.java
index 69ad4d3..f4543ea 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/entities/CountersTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/CountersTest.java
@@ -26,48 +26,6 @@ import org.apache.cassandra.exceptions.InvalidRequestException;
 public class CountersTest extends CQLTester
 {
     /**
-     * Check for a table with counters,
-     * migrated from cql_tests.py:TestCQL.counters_test()
-     */
-    @Test
-    public void testCounters() throws Throwable
-    {
-        createTable("CREATE TABLE %s (userid int, url text, total counter, PRIMARY KEY (userid, url)) WITH COMPACT STORAGE");
-
-        execute("UPDATE %s SET total = total + 1 WHERE userid = 1 AND url = 'http://foo.com'");
-        assertRows(execute("SELECT total FROM %s WHERE userid = 1 AND url = 'http://foo.com'"),
-                   row(1L));
-
-        execute("UPDATE %s SET total = total - 4 WHERE userid = 1 AND url = 'http://foo.com'");
-        assertRows(execute("SELECT total FROM %s WHERE userid = 1 AND url = 'http://foo.com'"),
-                   row(-3L));
-
-        execute("UPDATE %s SET total = total+1 WHERE userid = 1 AND url = 'http://foo.com'");
-        assertRows(execute("SELECT total FROM %s WHERE userid = 1 AND url = 'http://foo.com'"),
-                   row(-2L));
-
-        execute("UPDATE %s SET total = total -2 WHERE userid = 1 AND url = 'http://foo.com'");
-        assertRows(execute("SELECT total FROM %s WHERE userid = 1 AND url = 'http://foo.com'"),
-                   row(-4L));
-
-        execute("UPDATE %s SET total += 6 WHERE userid = 1 AND url = 'http://foo.com'");
-        assertRows(execute("SELECT total FROM %s WHERE userid = 1 AND url = 'http://foo.com'"),
-                   row(2L));
-
-        execute("UPDATE %s SET total -= 1 WHERE userid = 1 AND url = 'http://foo.com'");
-        assertRows(execute("SELECT total FROM %s WHERE userid = 1 AND url = 'http://foo.com'"),
-                   row(1L));
-
-        execute("UPDATE %s SET total += -2 WHERE userid = 1 AND url = 'http://foo.com'");
-        assertRows(execute("SELECT total FROM %s WHERE userid = 1 AND url = 'http://foo.com'"),
-                   row(-1L));
-
-        execute("UPDATE %s SET total -= -2 WHERE userid = 1 AND url = 'http://foo.com'");
-        assertRows(execute("SELECT total FROM %s WHERE userid = 1 AND url = 'http://foo.com'"),
-                   row(1L));
-    }
-
-    /**
      * Test for the validation bug of #4706,
      * migrated from cql_tests.py:TestCQL.validate_counter_regular_test()
      */
@@ -131,75 +89,69 @@ public class CountersTest extends CQLTester
     @Test
     public void testCounterFiltering() throws Throwable
     {
-        for (String compactStorageClause : new String[]{ "", " WITH COMPACT STORAGE" })
-        {
-            createTable("CREATE TABLE %s (k int PRIMARY KEY, a counter)" + compactStorageClause);
-
-            for (int i = 0; i < 10; i++)
-                execute("UPDATE %s SET a = a + ? WHERE k = ?", (long) i, i);
-
-            execute("UPDATE %s SET a = a + ? WHERE k = ?", 6L, 10);
-
-            // GT
-            assertRowsIgnoringOrder(execute("SELECT * FROM %s WHERE a > ? ALLOW FILTERING", 5L),
-                                    row(6, 6L),
-                                    row(7, 7L),
-                                    row(8, 8L),
-                                    row(9, 9L),
-                                    row(10, 6L));
-
-            // GTE
-            assertRowsIgnoringOrder(execute("SELECT * FROM %s WHERE a >= ? ALLOW FILTERING", 6L),
-                                    row(6, 6L),
-                                    row(7, 7L),
-                                    row(8, 8L),
-                                    row(9, 9L),
-                                    row(10, 6L));
-
-            // LT
-            assertRowsIgnoringOrder(execute("SELECT * FROM %s WHERE a < ? ALLOW FILTERING", 3L),
-                                    row(0, 0L),
-                                    row(1, 1L),
-                                    row(2, 2L));
-
-            // LTE
-            assertRowsIgnoringOrder(execute("SELECT * FROM %s WHERE a <= ? ALLOW FILTERING", 3L),
-                                    row(0, 0L),
-                                    row(1, 1L),
-                                    row(2, 2L),
-                                    row(3, 3L));
-
-            // EQ
-            assertRowsIgnoringOrder(execute("SELECT * FROM %s WHERE a = ? ALLOW FILTERING", 6L),
-                                    row(6, 6L),
-                                    row(10, 6L));
-        }
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, a counter)");
+
+        for (int i = 0; i < 10; i++)
+            execute("UPDATE %s SET a = a + ? WHERE k = ?", (long) i, i);
+
+        execute("UPDATE %s SET a = a + ? WHERE k = ?", 6L, 10);
+
+        // GT
+        assertRowsIgnoringOrder(execute("SELECT * FROM %s WHERE a > ? ALLOW FILTERING", 5L),
+                                row(6, 6L),
+                                row(7, 7L),
+                                row(8, 8L),
+                                row(9, 9L),
+                                row(10, 6L));
+
+        // GTE
+        assertRowsIgnoringOrder(execute("SELECT * FROM %s WHERE a >= ? ALLOW FILTERING", 6L),
+                                row(6, 6L),
+                                row(7, 7L),
+                                row(8, 8L),
+                                row(9, 9L),
+                                row(10, 6L));
+
+        // LT
+        assertRowsIgnoringOrder(execute("SELECT * FROM %s WHERE a < ? ALLOW FILTERING", 3L),
+                                row(0, 0L),
+                                row(1, 1L),
+                                row(2, 2L));
+
+        // LTE
+        assertRowsIgnoringOrder(execute("SELECT * FROM %s WHERE a <= ? ALLOW FILTERING", 3L),
+                                row(0, 0L),
+                                row(1, 1L),
+                                row(2, 2L),
+                                row(3, 3L));
+
+        // EQ
+        assertRowsIgnoringOrder(execute("SELECT * FROM %s WHERE a = ? ALLOW FILTERING", 6L),
+                                row(6, 6L),
+                                row(10, 6L));
     }
 
     @Test
     public void testCounterFilteringWithNull() throws Throwable
     {
-        for (String compactStorageClause : new String[]{ "", " WITH COMPACT STORAGE" })
-        {
-            createTable("CREATE TABLE %s (k int PRIMARY KEY, a counter, b counter)" + compactStorageClause);
-            execute("UPDATE %s SET a = a + ? WHERE k = ?", 1L, 1);
-
-            assertRows(execute("SELECT * FROM %s WHERE a > ? ALLOW FILTERING", 0L),
-                       row(1, 1L, null));
-            // GT
-            assertEmpty(execute("SELECT * FROM %s WHERE b > ? ALLOW FILTERING", 1L));
-            // GTE
-            assertEmpty(execute("SELECT * FROM %s WHERE b >= ? ALLOW FILTERING", 1L));
-            // LT
-            assertEmpty(execute("SELECT * FROM %s WHERE b < ? ALLOW FILTERING", 1L));
-            // LTE
-            assertEmpty(execute("SELECT * FROM %s WHERE b <= ? ALLOW FILTERING", 1L));
-            // EQ
-            assertEmpty(execute("SELECT * FROM %s WHERE b = ? ALLOW FILTERING", 1L));
-            // with null
-            assertInvalidMessage("Invalid null value for counter increment/decrement",
-                                 "SELECT * FROM %s WHERE b = null ALLOW FILTERING");
-        }
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, a counter, b counter)");
+        execute("UPDATE %s SET a = a + ? WHERE k = ?", 1L, 1);
+
+        assertRows(execute("SELECT * FROM %s WHERE a > ? ALLOW FILTERING", 0L),
+                   row(1, 1L, null));
+        // GT
+        assertEmpty(execute("SELECT * FROM %s WHERE b > ? ALLOW FILTERING", 1L));
+        // GTE
+        assertEmpty(execute("SELECT * FROM %s WHERE b >= ? ALLOW FILTERING", 1L));
+        // LT
+        assertEmpty(execute("SELECT * FROM %s WHERE b < ? ALLOW FILTERING", 1L));
+        // LTE
+        assertEmpty(execute("SELECT * FROM %s WHERE b <= ? ALLOW FILTERING", 1L));
+        // EQ
+        assertEmpty(execute("SELECT * FROM %s WHERE b = ? ALLOW FILTERING", 1L));
+        // with null
+        assertInvalidMessage("Invalid null value for counter increment/decrement",
+                             "SELECT * FROM %s WHERE b = null ALLOW FILTERING");
     }
 
     /**
@@ -211,99 +163,4 @@ public class CountersTest extends CQLTester
         assertInvalidThrowMessage("counter type is not supported for PRIMARY KEY part a",
                                   InvalidRequestException.class, String.format("CREATE TABLE %s.%s (a counter, b int, PRIMARY KEY (b, a)) WITH CLUSTERING ORDER BY (a desc);", KEYSPACE, createTableName()));
     }
-
-    /**
-     * Test for the bug of #11726.
-     */
-    @Test
-    public void testCounterAndColumnSelection() throws Throwable
-    {
-        for (String compactStorageClause : new String[]{ "", " WITH COMPACT STORAGE" })
-        {
-            createTable("CREATE TABLE %s (k int PRIMARY KEY, c counter)" + compactStorageClause);
-
-            // Flush 2 updates in different sstable so that the following select does a merge, which is what triggers
-            // the problem from #11726
-
-            execute("UPDATE %s SET c = c + ? WHERE k = ?", 1L, 0);
-
-            flush();
-
-            execute("UPDATE %s SET c = c + ? WHERE k = ?", 1L, 0);
-
-            flush();
-
-            // Querying, but not including the counter. Pre-CASSANDRA-11726, this made us query the counter but include
-            // it's value, which broke at merge (post-CASSANDRA-11726 are special cases to never skip values).
-            assertRows(execute("SELECT k FROM %s"), row(0));
-        }
-    }
-
-    /*
-     * Check that a counter batch works as intended
-     */
-    @Test
-    public void testCounterBatch() throws Throwable
-    {
-        createTable("CREATE TABLE %s (userid int, url text, total counter, PRIMARY KEY (userid, url)) WITH COMPACT STORAGE");
-
-        // Ensure we handle updates to the same CQL row in the same partition properly
-        execute("BEGIN UNLOGGED BATCH " +
-                "UPDATE %1$s SET total = total + 1 WHERE userid = 1 AND url = 'http://foo.com'; " +
-                "UPDATE %1$s SET total = total + 1 WHERE userid = 1 AND url = 'http://foo.com'; " +
-                "UPDATE %1$s SET total = total + 1 WHERE userid = 1 AND url = 'http://foo.com'; " +
-                "APPLY BATCH; ");
-        assertRows(execute("SELECT total FROM %s WHERE userid = 1 AND url = 'http://foo.com'"),
-                row(3L));
-
-        // Ensure we handle different CQL rows in the same partition properly
-        execute("BEGIN UNLOGGED BATCH " +
-                "UPDATE %1$s SET total = total + 1 WHERE userid = 1 AND url = 'http://bar.com'; " +
-                "UPDATE %1$s SET total = total + 1 WHERE userid = 1 AND url = 'http://baz.com'; " +
-                "UPDATE %1$s SET total = total + 1 WHERE userid = 1 AND url = 'http://bad.com'; " +
-                "APPLY BATCH; ");
-        assertRows(execute("SELECT url, total FROM %s WHERE userid = 1"),
-                row("http://bad.com", 1L),
-                row("http://bar.com", 1L),
-                row("http://baz.com", 1L),
-                row("http://foo.com", 3L)); // from previous batch
-
-        // Different counters in the same CQL Row
-        createTable("CREATE TABLE %s (userid int, url text, first counter, second counter, third counter, PRIMARY KEY (userid, url))");
-        execute("BEGIN UNLOGGED BATCH " +
-                "UPDATE %1$s SET first = first + 1 WHERE userid = 1 AND url = 'http://foo.com'; " +
-                "UPDATE %1$s SET first = first + 1 WHERE userid = 1 AND url = 'http://foo.com'; " +
-                "UPDATE %1$s SET second = second + 1 WHERE userid = 1 AND url = 'http://foo.com'; " +
-                "APPLY BATCH; ");
-        assertRows(execute("SELECT first, second, third FROM %s WHERE userid = 1 AND url = 'http://foo.com'"),
-                row(2L, 1L, null));
-
-        // Different counters in different CQL Rows
-        execute("BEGIN UNLOGGED BATCH " +
-                "UPDATE %1$s SET first = first + 1 WHERE userid = 1 AND url = 'http://bad.com'; " +
-                "UPDATE %1$s SET first = first + 1, second = second + 1 WHERE userid = 1 AND url = 'http://bar.com'; " +
-                "UPDATE %1$s SET first = first - 1, second = second - 1 WHERE userid = 1 AND url = 'http://bar.com'; " +
-                "UPDATE %1$s SET second = second + 1 WHERE userid = 1 AND url = 'http://baz.com'; " +
-                "APPLY BATCH; ");
-        assertRows(execute("SELECT url, first, second, third FROM %s WHERE userid = 1"),
-                row("http://bad.com", 1L, null, null),
-                row("http://bar.com", 0L, 0L, null),
-                row("http://baz.com", null, 1L, null),
-                row("http://foo.com", 2L, 1L, null)); // from previous batch
-
-
-        // Different counters in different partitions
-        execute("BEGIN UNLOGGED BATCH " +
-                "UPDATE %1$s SET first = first + 1 WHERE userid = 2 AND url = 'http://bad.com'; " +
-                "UPDATE %1$s SET first = first + 1, second = second + 1 WHERE userid = 3 AND url = 'http://bar.com'; " +
-                "UPDATE %1$s SET first = first - 1, second = second - 1 WHERE userid = 4 AND url = 'http://bar.com'; " +
-                "UPDATE %1$s SET second = second + 1 WHERE userid = 5 AND url = 'http://baz.com'; " +
-                "APPLY BATCH; ");
-        assertRowsIgnoringOrder(execute("SELECT userid, url, first, second, third FROM %s WHERE userid IN (2, 3, 4, 5)"),
-                row(2, "http://bad.com", 1L, null, null),
-                row(3, "http://bar.com", 1L, 1L, null),
-                row(4, "http://bar.com", -1L, -1L, null),
-                row(5, "http://baz.com", null, 1L, null));
-
-    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/07fbd8ee/test/unit/org/apache/cassandra/cql3/validation/entities/FrozenCollectionsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/FrozenCollectionsTest.java b/test/unit/org/apache/cassandra/cql3/validation/entities/FrozenCollectionsTest.java
index e50528b..c493267 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/entities/FrozenCollectionsTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/FrozenCollectionsTest.java
@@ -183,272 +183,263 @@ public class FrozenCollectionsTest extends CQLTester
     @Test
     public void testClusteringKeyUsage() throws Throwable
     {
-        for (String option : Arrays.asList("", " WITH COMPACT STORAGE"))
-        {
-            createTable("CREATE TABLE %s (a int, b frozen<set<int>>, c int, PRIMARY KEY (a, b))" + option);
-
-            execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, set(), 1);
-            execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, set(1, 2, 3), 1);
-            execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, set(4, 5, 6), 0);
-            execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, set(7, 8, 9), 0);
-
-            // overwrite with an update
-            execute("UPDATE %s SET c=? WHERE a=? AND b=?", 0, 0, set());
-            execute("UPDATE %s SET c=? WHERE a=? AND b=?", 0, 0, set(1, 2, 3));
-
-            assertRows(execute("SELECT * FROM %s"),
-                row(0, set(), 0),
-                row(0, set(1, 2, 3), 0),
-                row(0, set(4, 5, 6), 0),
-                row(0, set(7, 8, 9), 0)
-            );
-
-            assertRows(execute("SELECT b FROM %s"),
-                row(set()),
-                row(set(1, 2, 3)),
-                row(set(4, 5, 6)),
-                row(set(7, 8, 9))
-            );
-
-            assertRows(execute("SELECT * FROM %s LIMIT 2"),
-                row(0, set(), 0),
-                row(0, set(1, 2, 3), 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a=? AND b=?", 0, set(4, 5, 6)),
-                row(0, set(4, 5, 6), 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a=? AND b=?", 0, set()),
-                row(0, set(), 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a=? AND b IN ?", 0, list(set(4, 5, 6), set())),
-                row(0, set(), 0),
-                row(0, set(4, 5, 6), 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a=? AND b > ?", 0, set(4, 5, 6)),
-                row(0, set(7, 8, 9), 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a=? AND b >= ?", 0, set(4, 5, 6)),
-                row(0, set(4, 5, 6), 0),
-                row(0, set(7, 8, 9), 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a=? AND b < ?", 0, set(4, 5, 6)),
-                row(0, set(), 0),
-                row(0, set(1, 2, 3), 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a=? AND b <= ?", 0, set(4, 5, 6)),
-                row(0, set(), 0),
-                row(0, set(1, 2, 3), 0),
-                row(0, set(4, 5, 6), 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a=? AND b > ? AND b <= ?", 0, set(1, 2, 3), set(4, 5, 6)),
-                row(0, set(4, 5, 6), 0)
-            );
-
-            execute("DELETE FROM %s WHERE a=? AND b=?", 0, set());
-            execute("DELETE FROM %s WHERE a=? AND b=?", 0, set(4, 5, 6));
-            assertRows(execute("SELECT * FROM %s"),
-                row(0, set(1, 2, 3), 0),
-                row(0, set(7, 8, 9), 0)
-            );
-        }
+        createTable("CREATE TABLE %s (a int, b frozen<set<int>>, c int, PRIMARY KEY (a, b))");
+
+        execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, set(), 1);
+        execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, set(1, 2, 3), 1);
+        execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, set(4, 5, 6), 0);
+        execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, set(7, 8, 9), 0);
+
+        // overwrite with an update
+        execute("UPDATE %s SET c=? WHERE a=? AND b=?", 0, 0, set());
+        execute("UPDATE %s SET c=? WHERE a=? AND b=?", 0, 0, set(1, 2, 3));
+
+        assertRows(execute("SELECT * FROM %s"),
+                   row(0, set(), 0),
+                   row(0, set(1, 2, 3), 0),
+                   row(0, set(4, 5, 6), 0),
+                   row(0, set(7, 8, 9), 0)
+        );
+
+        assertRows(execute("SELECT b FROM %s"),
+                   row(set()),
+                   row(set(1, 2, 3)),
+                   row(set(4, 5, 6)),
+                   row(set(7, 8, 9))
+        );
+
+        assertRows(execute("SELECT * FROM %s LIMIT 2"),
+                   row(0, set(), 0),
+                   row(0, set(1, 2, 3), 0)
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE a=? AND b=?", 0, set(4, 5, 6)),
+                   row(0, set(4, 5, 6), 0)
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE a=? AND b=?", 0, set()),
+                   row(0, set(), 0)
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE a=? AND b IN ?", 0, list(set(4, 5, 6), set())),
+                   row(0, set(), 0),
+                   row(0, set(4, 5, 6), 0)
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE a=? AND b > ?", 0, set(4, 5, 6)),
+                   row(0, set(7, 8, 9), 0)
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE a=? AND b >= ?", 0, set(4, 5, 6)),
+                   row(0, set(4, 5, 6), 0),
+                   row(0, set(7, 8, 9), 0)
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE a=? AND b < ?", 0, set(4, 5, 6)),
+                   row(0, set(), 0),
+                   row(0, set(1, 2, 3), 0)
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE a=? AND b <= ?", 0, set(4, 5, 6)),
+                   row(0, set(), 0),
+                   row(0, set(1, 2, 3), 0),
+                   row(0, set(4, 5, 6), 0)
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE a=? AND b > ? AND b <= ?", 0, set(1, 2, 3), set(4, 5, 6)),
+                   row(0, set(4, 5, 6), 0)
+        );
+
+        execute("DELETE FROM %s WHERE a=? AND b=?", 0, set());
+        execute("DELETE FROM %s WHERE a=? AND b=?", 0, set(4, 5, 6));
+        assertRows(execute("SELECT * FROM %s"),
+                   row(0, set(1, 2, 3), 0),
+                   row(0, set(7, 8, 9), 0)
+        );
     }
 
     @Test
     public void testNestedClusteringKeyUsage() throws Throwable
     {
-        for (String option : Arrays.asList("", " WITH COMPACT STORAGE"))
-        {
-            createTable("CREATE TABLE %s (a int, b frozen<map<set<int>, list<int>>>, c frozen<set<int>>, d int, PRIMARY KEY (a, b, c))" + option);
-
-            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, map(), set(), 0);
-            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, map(set(), list(1, 2, 3)), set(), 0);
-            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, map(set(1, 2, 3), list(1, 2, 3)), set(1, 2, 3), 0);
-            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, map(set(4, 5, 6), list(1, 2, 3)), set(1, 2, 3), 0);
-            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, map(set(7, 8, 9), list(1, 2, 3)), set(1, 2, 3), 0);
-
-            assertRows(execute("SELECT * FROM %s"),
-                row(0, map(), set(), 0),
-                row(0, map(set(), list(1, 2, 3)), set(), 0),
-                row(0, map(set(1, 2, 3), list(1, 2, 3)), set(1, 2, 3), 0),
-                row(0, map(set(4, 5, 6), list(1, 2, 3)), set(1, 2, 3), 0),
-                row(0, map(set(7, 8, 9), list(1, 2, 3)), set(1, 2, 3), 0)
-            );
-
-            assertRows(execute("SELECT b FROM %s"),
-                row(map()),
-                row(map(set(), list(1, 2, 3))),
-                row(map(set(1, 2, 3), list(1, 2, 3))),
-                row(map(set(4, 5, 6), list(1, 2, 3))),
-                row(map(set(7, 8, 9), list(1, 2, 3)))
-            );
-
-            assertRows(execute("SELECT c FROM %s"),
-                row(set()),
-                row(set()),
-                row(set(1, 2, 3)),
-                row(set(1, 2, 3)),
-                row(set(1, 2, 3))
-            );
-
-            assertRows(execute("SELECT * FROM %s LIMIT 3"),
-                row(0, map(), set(), 0),
-                row(0, map(set(), list(1, 2, 3)), set(), 0),
-                row(0, map(set(1, 2, 3), list(1, 2, 3)), set(1, 2, 3), 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a=0 ORDER BY b DESC LIMIT 4"),
-                row(0, map(set(7, 8, 9), list(1, 2, 3)), set(1, 2, 3), 0),
-                row(0, map(set(4, 5, 6), list(1, 2, 3)), set(1, 2, 3), 0),
-                row(0, map(set(1, 2, 3), list(1, 2, 3)), set(1, 2, 3), 0),
-                row(0, map(set(), list(1, 2, 3)), set(), 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a=? AND b=?", 0, map()),
-                row(0, map(), set(), 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a=? AND b=?", 0, map(set(), list(1, 2, 3))),
-                row(0, map(set(), list(1, 2, 3)), set(), 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a=? AND b=?", 0, map(set(1, 2, 3), list(1, 2, 3))),
-                row(0, map(set(1, 2, 3), list(1, 2, 3)), set(1, 2, 3), 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a=? AND b=? AND c=?", 0, map(set(), list(1, 2, 3)), set()),
-                    row(0, map(set(), list(1, 2, 3)), set(), 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a=? AND (b, c) IN ?", 0, list(tuple(map(set(4, 5, 6), list(1, 2, 3)), set(1, 2, 3)),
-                                                                                     tuple(map(), set()))),
-                row(0, map(), set(), 0),
-                row(0, map(set(4, 5, 6), list(1, 2, 3)), set(1, 2, 3), 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a=? AND b > ?", 0, map(set(4, 5, 6), list(1, 2, 3))),
-                row(0, map(set(7, 8, 9), list(1, 2, 3)), set(1, 2, 3), 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a=? AND b >= ?", 0, map(set(4, 5, 6), list(1, 2, 3))),
-                row(0, map(set(4, 5, 6), list(1, 2, 3)), set(1, 2, 3), 0),
-                row(0, map(set(7, 8, 9), list(1, 2, 3)), set(1, 2, 3), 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a=? AND b < ?", 0, map(set(4, 5, 6), list(1, 2, 3))),
-                row(0, map(), set(), 0),
-                row(0, map(set(), list(1, 2, 3)), set(), 0),
-                row(0, map(set(1, 2, 3), list(1, 2, 3)), set(1, 2, 3), 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a=? AND b <= ?", 0, map(set(4, 5, 6), list(1, 2, 3))),
-                row(0, map(), set(), 0),
-                row(0, map(set(), list(1, 2, 3)), set(), 0),
-                row(0, map(set(1, 2, 3), list(1, 2, 3)), set(1, 2, 3), 0),
-                row(0, map(set(4, 5, 6), list(1, 2, 3)), set(1, 2, 3), 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a=? AND b > ? AND b <= ?", 0, map(set(1, 2, 3), list(1, 2, 3)), map(set(4, 5, 6), list(1, 2, 3))),
-                row(0, map(set(4, 5, 6), list(1, 2, 3)), set(1, 2, 3), 0)
-            );
-
-            execute("DELETE FROM %s WHERE a=? AND b=? AND c=?", 0, map(), set());
-            assertEmpty(execute("SELECT * FROM %s WHERE a=? AND b=? AND c=?", 0, map(), set()));
-
-            execute("DELETE FROM %s WHERE a=? AND b=? AND c=?", 0, map(set(), list(1, 2, 3)), set());
-            assertEmpty(execute("SELECT * FROM %s WHERE a=? AND b=? AND c=?", 0, map(set(), list(1, 2, 3)), set()));
-
-            execute("DELETE FROM %s WHERE a=? AND b=? AND c=?", 0, map(set(4, 5, 6), list(1, 2, 3)), set(1, 2, 3));
-            assertEmpty(execute("SELECT * FROM %s WHERE a=? AND b=? AND c=?", 0, map(set(4, 5, 6), list(1, 2, 3)), set(1, 2, 3)));
-
-            assertRows(execute("SELECT * FROM %s"),
-                    row(0, map(set(1, 2, 3), list(1, 2, 3)), set(1, 2, 3), 0),
-                    row(0, map(set(7, 8, 9), list(1, 2, 3)), set(1, 2, 3), 0)
-            );
-        }
+        createTable("CREATE TABLE %s (a int, b frozen<map<set<int>, list<int>>>, c frozen<set<int>>, d int, PRIMARY KEY (a, b, c))");
+
+        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, map(), set(), 0);
+        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, map(set(), list(1, 2, 3)), set(), 0);
+        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, map(set(1, 2, 3), list(1, 2, 3)), set(1, 2, 3), 0);
+        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, map(set(4, 5, 6), list(1, 2, 3)), set(1, 2, 3), 0);
+        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, map(set(7, 8, 9), list(1, 2, 3)), set(1, 2, 3), 0);
+
+        assertRows(execute("SELECT * FROM %s"),
+                   row(0, map(), set(), 0),
+                   row(0, map(set(), list(1, 2, 3)), set(), 0),
+                   row(0, map(set(1, 2, 3), list(1, 2, 3)), set(1, 2, 3), 0),
+                   row(0, map(set(4, 5, 6), list(1, 2, 3)), set(1, 2, 3), 0),
+                   row(0, map(set(7, 8, 9), list(1, 2, 3)), set(1, 2, 3), 0)
+        );
+
+        assertRows(execute("SELECT b FROM %s"),
+                   row(map()),
+                   row(map(set(), list(1, 2, 3))),
+                   row(map(set(1, 2, 3), list(1, 2, 3))),
+                   row(map(set(4, 5, 6), list(1, 2, 3))),
+                   row(map(set(7, 8, 9), list(1, 2, 3)))
+        );
+
+        assertRows(execute("SELECT c FROM %s"),
+                   row(set()),
+                   row(set()),
+                   row(set(1, 2, 3)),
+                   row(set(1, 2, 3)),
+                   row(set(1, 2, 3))
+        );
+
+        assertRows(execute("SELECT * FROM %s LIMIT 3"),
+                   row(0, map(), set(), 0),
+                   row(0, map(set(), list(1, 2, 3)), set(), 0),
+                   row(0, map(set(1, 2, 3), list(1, 2, 3)), set(1, 2, 3), 0)
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE a=0 ORDER BY b DESC LIMIT 4"),
+                   row(0, map(set(7, 8, 9), list(1, 2, 3)), set(1, 2, 3), 0),
+                   row(0, map(set(4, 5, 6), list(1, 2, 3)), set(1, 2, 3), 0),
+                   row(0, map(set(1, 2, 3), list(1, 2, 3)), set(1, 2, 3), 0),
+                   row(0, map(set(), list(1, 2, 3)), set(), 0)
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE a=? AND b=?", 0, map()),
+                   row(0, map(), set(), 0)
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE a=? AND b=?", 0, map(set(), list(1, 2, 3))),
+                   row(0, map(set(), list(1, 2, 3)), set(), 0)
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE a=? AND b=?", 0, map(set(1, 2, 3), list(1, 2, 3))),
+                   row(0, map(set(1, 2, 3), list(1, 2, 3)), set(1, 2, 3), 0)
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE a=? AND b=? AND c=?", 0, map(set(), list(1, 2, 3)), set()),
+                   row(0, map(set(), list(1, 2, 3)), set(), 0)
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE a=? AND (b, c) IN ?", 0, list(tuple(map(set(4, 5, 6), list(1, 2, 3)), set(1, 2, 3)),
+                                                                                 tuple(map(), set()))),
+                   row(0, map(), set(), 0),
+                   row(0, map(set(4, 5, 6), list(1, 2, 3)), set(1, 2, 3), 0)
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE a=? AND b > ?", 0, map(set(4, 5, 6), list(1, 2, 3))),
+                   row(0, map(set(7, 8, 9), list(1, 2, 3)), set(1, 2, 3), 0)
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE a=? AND b >= ?", 0, map(set(4, 5, 6), list(1, 2, 3))),
+                   row(0, map(set(4, 5, 6), list(1, 2, 3)), set(1, 2, 3), 0),
+                   row(0, map(set(7, 8, 9), list(1, 2, 3)), set(1, 2, 3), 0)
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE a=? AND b < ?", 0, map(set(4, 5, 6), list(1, 2, 3))),
+                   row(0, map(), set(), 0),
+                   row(0, map(set(), list(1, 2, 3)), set(), 0),
+                   row(0, map(set(1, 2, 3), list(1, 2, 3)), set(1, 2, 3), 0)
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE a=? AND b <= ?", 0, map(set(4, 5, 6), list(1, 2, 3))),
+                   row(0, map(), set(), 0),
+                   row(0, map(set(), list(1, 2, 3)), set(), 0),
+                   row(0, map(set(1, 2, 3), list(1, 2, 3)), set(1, 2, 3), 0),
+                   row(0, map(set(4, 5, 6), list(1, 2, 3)), set(1, 2, 3), 0)
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE a=? AND b > ? AND b <= ?", 0, map(set(1, 2, 3), list(1, 2, 3)), map(set(4, 5, 6), list(1, 2, 3))),
+                   row(0, map(set(4, 5, 6), list(1, 2, 3)), set(1, 2, 3), 0)
+        );
+
+        execute("DELETE FROM %s WHERE a=? AND b=? AND c=?", 0, map(), set());
+        assertEmpty(execute("SELECT * FROM %s WHERE a=? AND b=? AND c=?", 0, map(), set()));
+
+        execute("DELETE FROM %s WHERE a=? AND b=? AND c=?", 0, map(set(), list(1, 2, 3)), set());
+        assertEmpty(execute("SELECT * FROM %s WHERE a=? AND b=? AND c=?", 0, map(set(), list(1, 2, 3)), set()));
+
+        execute("DELETE FROM %s WHERE a=? AND b=? AND c=?", 0, map(set(4, 5, 6), list(1, 2, 3)), set(1, 2, 3));
+        assertEmpty(execute("SELECT * FROM %s WHERE a=? AND b=? AND c=?", 0, map(set(4, 5, 6), list(1, 2, 3)), set(1, 2, 3)));
+
+        assertRows(execute("SELECT * FROM %s"),
+                   row(0, map(set(1, 2, 3), list(1, 2, 3)), set(1, 2, 3), 0),
+                   row(0, map(set(7, 8, 9), list(1, 2, 3)), set(1, 2, 3), 0)
+        );
     }
 
     @Test
     public void testNormalColumnUsage() throws Throwable
     {
-        for (String option : Arrays.asList("", " WITH COMPACT STORAGE"))
-        {
-            createTable("CREATE TABLE %s (a int PRIMARY KEY, b frozen<map<set<int>, list<int>>>, c frozen<set<int>>)" + option);
-
-            execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, map(), set());
-            execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 1, map(set(), list(99999, 999999, 99999)), set());
-            execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 2, map(set(1, 2, 3), list(1, 2, 3)), set(1, 2, 3));
-            execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 3, map(set(4, 5, 6), list(1, 2, 3)), set(1, 2, 3));
-            execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 4, map(set(7, 8, 9), list(1, 2, 3)), set(1, 2, 3));
-
-            // overwrite with update
-            execute ("UPDATE %s SET b=? WHERE a=?", map(set(), list(1, 2, 3)), 1);
-
-            assertRows(execute("SELECT * FROM %s"),
-                row(0, map(), set()),
-                row(1, map(set(), list(1, 2, 3)), set()),
-                row(2, map(set(1, 2, 3), list(1, 2, 3)), set(1, 2, 3)),
-                row(3, map(set(4, 5, 6), list(1, 2, 3)), set(1, 2, 3)),
-                row(4, map(set(7, 8, 9), list(1, 2, 3)), set(1, 2, 3))
-            );
-
-            assertRows(execute("SELECT b FROM %s"),
-                row(map()),
-                row(map(set(), list(1, 2, 3))),
-                row(map(set(1, 2, 3), list(1, 2, 3))),
-                row(map(set(4, 5, 6), list(1, 2, 3))),
-                row(map(set(7, 8, 9), list(1, 2, 3)))
-            );
-
-            assertRows(execute("SELECT c FROM %s"),
-                row(set()),
-                row(set()),
-                row(set(1, 2, 3)),
-                row(set(1, 2, 3)),
-                row(set(1, 2, 3))
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a=?", 3),
-                row(3, map(set(4, 5, 6), list(1, 2, 3)), set(1, 2, 3))
-            );
-
-            execute("UPDATE %s SET b=? WHERE a=?", null, 1);
-            assertRows(execute("SELECT * FROM %s WHERE a=?", 1),
-                row(1, null, set())
-            );
-
-            execute("UPDATE %s SET b=? WHERE a=?", map(), 1);
-            assertRows(execute("SELECT * FROM %s WHERE a=?", 1),
-                row(1, map(), set())
-            );
-
-            execute("UPDATE %s SET c=? WHERE a=?", null, 2);
-            assertRows(execute("SELECT * FROM %s WHERE a=?", 2),
-                row(2, map(set(1, 2, 3), list(1, 2, 3)), null)
-            );
-
-            execute("UPDATE %s SET c=? WHERE a=?", set(), 2);
-            assertRows(execute("SELECT * FROM %s WHERE a=?", 2),
-                    row(2, map(set(1, 2, 3), list(1, 2, 3)), set())
-            );
-
-            execute("DELETE b FROM %s WHERE a=?", 3);
-            assertRows(execute("SELECT * FROM %s WHERE a=?", 3),
-                row(3, null, set(1, 2, 3))
-            );
-
-            execute("DELETE c FROM %s WHERE a=?", 4);
-            assertRows(execute("SELECT * FROM %s WHERE a=?", 4),
-                row(4, map(set(7, 8, 9), list(1, 2, 3)), null)
-            );
-        }
+        createTable("CREATE TABLE %s (a int PRIMARY KEY, b frozen<map<set<int>, list<int>>>, c frozen<set<int>>)");
+
+        execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, map(), set());
+        execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 1, map(set(), list(99999, 999999, 99999)), set());
+        execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 2, map(set(1, 2, 3), list(1, 2, 3)), set(1, 2, 3));
+        execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 3, map(set(4, 5, 6), list(1, 2, 3)), set(1, 2, 3));
+        execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 4, map(set(7, 8, 9), list(1, 2, 3)), set(1, 2, 3));
+
+        // overwrite with update
+        execute("UPDATE %s SET b=? WHERE a=?", map(set(), list(1, 2, 3)), 1);
+
+        assertRows(execute("SELECT * FROM %s"),
+                   row(0, map(), set()),
+                   row(1, map(set(), list(1, 2, 3)), set()),
+                   row(2, map(set(1, 2, 3), list(1, 2, 3)), set(1, 2, 3)),
+                   row(3, map(set(4, 5, 6), list(1, 2, 3)), set(1, 2, 3)),
+                   row(4, map(set(7, 8, 9), list(1, 2, 3)), set(1, 2, 3))
+        );
+
+        assertRows(execute("SELECT b FROM %s"),
+                   row(map()),
+                   row(map(set(), list(1, 2, 3))),
+                   row(map(set(1, 2, 3), list(1, 2, 3))),
+                   row(map(set(4, 5, 6), list(1, 2, 3))),
+                   row(map(set(7, 8, 9), list(1, 2, 3)))
+        );
+
+        assertRows(execute("SELECT c FROM %s"),
+                   row(set()),
+                   row(set()),
+                   row(set(1, 2, 3)),
+                   row(set(1, 2, 3)),
+                   row(set(1, 2, 3))
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE a=?", 3),
+                   row(3, map(set(4, 5, 6), list(1, 2, 3)), set(1, 2, 3))
+        );
+
+        execute("UPDATE %s SET b=? WHERE a=?", null, 1);
+        assertRows(execute("SELECT * FROM %s WHERE a=?", 1),
+                   row(1, null, set())
+        );
+
+        execute("UPDATE %s SET b=? WHERE a=?", map(), 1);
+        assertRows(execute("SELECT * FROM %s WHERE a=?", 1),
+                   row(1, map(), set())
+        );
+
+        execute("UPDATE %s SET c=? WHERE a=?", null, 2);
+        assertRows(execute("SELECT * FROM %s WHERE a=?", 2),
+                   row(2, map(set(1, 2, 3), list(1, 2, 3)), null)
+        );
+
+        execute("UPDATE %s SET c=? WHERE a=?", set(), 2);
+        assertRows(execute("SELECT * FROM %s WHERE a=?", 2),
+                   row(2, map(set(1, 2, 3), list(1, 2, 3)), set())
+        );
+
+        execute("DELETE b FROM %s WHERE a=?", 3);
+        assertRows(execute("SELECT * FROM %s WHERE a=?", 3),
+                   row(3, null, set(1, 2, 3))
+        );
+
+        execute("DELETE c FROM %s WHERE a=?", 4);
+        assertRows(execute("SELECT * FROM %s WHERE a=?", 4),
+                   row(4, map(set(7, 8, 9), list(1, 2, 3)), null)
+        );
     }
 
     @Test

http://git-wip-us.apache.org/repos/asf/cassandra/blob/07fbd8ee/test/unit/org/apache/cassandra/cql3/validation/entities/SecondaryIndexTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/SecondaryIndexTest.java b/test/unit/org/apache/cassandra/cql3/validation/entities/SecondaryIndexTest.java
index 1c7576a..c763b9b 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/entities/SecondaryIndexTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/SecondaryIndexTest.java
@@ -651,23 +651,6 @@ public class SecondaryIndexTest extends CQLTester
     }
 
     @Test
-    public void testCompactTableWithValueOver64k() throws Throwable
-    {
-        createTable("CREATE TABLE %s(a int, b blob, PRIMARY KEY (a)) WITH COMPACT STORAGE");
-        createIndex("CREATE INDEX ON %s(b)");
-        failInsert("INSERT INTO %s (a, b) VALUES (0, ?)", ByteBuffer.allocate(TOO_BIG));
-        failInsert("INSERT INTO %s (a, b) VALUES (0, ?) IF NOT EXISTS", ByteBuffer.allocate(TOO_BIG));
-        failInsert("BEGIN BATCH\n" +
-                   "INSERT INTO %s (a, b) VALUES (0, ?);\n" +
-                   "APPLY BATCH",
-                   ByteBuffer.allocate(TOO_BIG));
-        failInsert("BEGIN BATCH\n" +
-                   "INSERT INTO %s (a, b) VALUES (0, ?) IF NOT EXISTS;\n" +
-                   "APPLY BATCH",
-                   ByteBuffer.allocate(TOO_BIG));
-    }
-
-    @Test
     public void testIndexOnPartitionKeyInsertValueOver64k() throws Throwable
     {
         createTable("CREATE TABLE %s(a int, b int, c blob, PRIMARY KEY ((a, b)))");
@@ -881,22 +864,6 @@ public class SecondaryIndexTest extends CQLTester
                    row("B"), row("E"));
     }
 
-    /**
-     * Migrated from cql_tests.py:TestCQL.invalid_clustering_indexing_test()
-     */
-    @Test
-    public void testIndexesOnClusteringInvalid() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a int, b int, c int, d int, PRIMARY KEY ((a, b))) WITH COMPACT STORAGE");
-        assertInvalid("CREATE INDEX ON %s (a)");
-        assertInvalid("CREATE INDEX ON %s (b)");
-
-        createTable("CREATE TABLE %s (a int, b int, c int, PRIMARY KEY (a, b)) WITH COMPACT STORAGE");
-        assertInvalid("CREATE INDEX ON %s (a)");
-        assertInvalid("CREATE INDEX ON %s (b)");
-        assertInvalid("CREATE INDEX ON %s (c)");
-    }
-
     @Test
     public void testMultipleIndexesOnOneColumn() throws Throwable
     {
@@ -1233,27 +1200,6 @@ public class SecondaryIndexTest extends CQLTester
     }
 
     @Test
-    public void testEmptyRestrictionValueWithSecondaryIndexAndCompactTables() throws Throwable
-    {
-        createTable("CREATE TABLE %s (pk blob, c blob, v blob, PRIMARY KEY ((pk), c)) WITH COMPACT STORAGE");
-        assertInvalidMessage("Secondary indexes are not supported on COMPACT STORAGE tables that have clustering columns",
-                            "CREATE INDEX on %s(c)");
-
-        createTable("CREATE TABLE %s (pk blob PRIMARY KEY, v blob) WITH COMPACT STORAGE");
-        createIndex("CREATE INDEX on %s(v)");
-
-        execute("INSERT INTO %s (pk, v) VALUES (?, ?)", bytes("foo123"), bytes("1"));
-
-        // Test restrictions on non-primary key value
-        assertEmpty(execute("SELECT * FROM %s WHERE pk = textAsBlob('foo123') AND v = textAsBlob('');"));
-
-        execute("INSERT INTO %s (pk, v) VALUES (?, ?)", bytes("foo124"), EMPTY_BYTE_BUFFER);
-
-        assertRows(execute("SELECT * FROM %s WHERE v = textAsBlob('');"),
-                   row(bytes("foo124"), EMPTY_BYTE_BUFFER));
-    }
-
-    @Test
     public void testPartitionKeyWithIndex() throws Throwable
     {
         createTable("CREATE TABLE %s (a int, b int, c int, PRIMARY KEY ((a, b)))");
@@ -1566,49 +1512,6 @@ public class SecondaryIndexTest extends CQLTester
         assertEmpty(execute("SELECT * FROM %s WHERE a = 5"));
     }
 
-    @Test
-    public void testIndicesOnCompactTable() throws Throwable
-    {
-        assertInvalidMessage("COMPACT STORAGE with composite PRIMARY KEY allows no more than one column not part of the PRIMARY KEY (got: v1, v2)",
-                             "CREATE TABLE test (pk int, c int, v1 int, v2 int, PRIMARY KEY(pk, c)) WITH COMPACT STORAGE");
-
-        createTable("CREATE TABLE %s (pk int, c int, v int, PRIMARY KEY(pk, c)) WITH COMPACT STORAGE");
-        assertInvalidMessage("Secondary indexes are not supported on COMPACT STORAGE tables that have clustering columns",
-                             "CREATE INDEX ON %s(v)");
-
-        createTable("CREATE TABLE %s (pk int PRIMARY KEY, v int) WITH COMPACT STORAGE");
-        createIndex("CREATE INDEX ON %s(v)");
-
-        execute("INSERT INTO %s (pk, v) VALUES (?, ?)", 1, 1);
-        execute("INSERT INTO %s (pk, v) VALUES (?, ?)", 2, 1);
-        execute("INSERT INTO %s (pk, v) VALUES (?, ?)", 3, 3);
-
-        assertRows(execute("SELECT pk, v FROM %s WHERE v = 1"),
-                   row(1, 1),
-                   row(2, 1));
-
-        assertRows(execute("SELECT pk, v FROM %s WHERE v = 3"),
-                   row(3, 3));
-
-        assertEmpty(execute("SELECT pk, v FROM %s WHERE v = 5"));
-
-        createTable("CREATE TABLE %s (pk int PRIMARY KEY, v1 int, v2 int) WITH COMPACT STORAGE");
-        createIndex("CREATE INDEX ON %s(v1)");
-
-        execute("INSERT INTO %s (pk, v1, v2) VALUES (?, ?, ?)", 1, 1, 1);
-        execute("INSERT INTO %s (pk, v1, v2) VALUES (?, ?, ?)", 2, 1, 2);
-        execute("INSERT INTO %s (pk, v1, v2) VALUES (?, ?, ?)", 3, 3, 3);
-
-        assertRows(execute("SELECT pk, v2 FROM %s WHERE v1 = 1"),
-                   row(1, 1),
-                   row(2, 2));
-
-        assertRows(execute("SELECT pk, v2 FROM %s WHERE v1 = 3"),
-                   row(3, 3));
-
-        assertEmpty(execute("SELECT pk, v2 FROM %s WHERE v1 = 5"));
-    }
-
     private ResultMessage.Prepared prepareStatement(String cql)
     {
         return QueryProcessor.prepare(String.format(cql, KEYSPACE, currentTable()),

http://git-wip-us.apache.org/repos/asf/cassandra/blob/07fbd8ee/test/unit/org/apache/cassandra/cql3/validation/miscellaneous/OverflowTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/miscellaneous/OverflowTest.java b/test/unit/org/apache/cassandra/cql3/validation/miscellaneous/OverflowTest.java
index 6fdedc2..406f27a 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/miscellaneous/OverflowTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/miscellaneous/OverflowTest.java
@@ -167,20 +167,6 @@ public class OverflowTest extends CQLTester
     }
 
     /**
-     * Test regression from #5189,
-     * migrated from cql_tests.py:TestCQL.compact_metadata_test()
-     */
-    @Test
-    public void testCompactMetadata() throws Throwable
-    {
-        createTable("CREATE TABLE %s (id int primary key, i int ) WITH COMPACT STORAGE");
-
-        execute("INSERT INTO %s (id, i) VALUES (1, 2)");
-        assertRows(execute("SELECT * FROM %s"),
-                   row(1, 2));
-    }
-
-    /**
      * Migrated from cql_tests.py:TestCQL.conversion_functions_test()
      */
     @Test
@@ -237,22 +223,6 @@ public class OverflowTest extends CQLTester
         // Test empty IN() in UPDATE
         execute("UPDATE %s SET v = 3 WHERE k1 IN () AND k2 = 2");
         assertArrayEquals(rows, getRows(execute("SELECT * FROM %s")));
-
-        // Same test, but for compact
-        createTable("CREATE TABLE %s (k1 int, k2 int, v int, PRIMARY KEY (k1, k2)) WITH COMPACT STORAGE");
-
-        rows = fill();
-
-        assertEmpty(execute("SELECT v FROM %s WHERE k1 IN ()"));
-        assertEmpty(execute("SELECT v FROM %s WHERE k1 = 0 AND k2 IN ()"));
-
-        // Test empty IN() in DELETE
-        execute("DELETE FROM %s WHERE k1 IN ()");
-        assertArrayEquals(rows, getRows(execute("SELECT * FROM %s")));
-
-        // Test empty IN() in UPDATE
-        execute("UPDATE %s SET v = 3 WHERE k1 IN () AND k2 = 2");
-        assertArrayEquals(rows, getRows(execute("SELECT * FROM %s")));
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/cassandra/blob/07fbd8ee/test/unit/org/apache/cassandra/cql3/validation/operations/AggregationTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/AggregationTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/AggregationTest.java
index a1e9c8f..dea3803 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/AggregationTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/AggregationTest.java
@@ -321,26 +321,6 @@ public class AggregationTest extends CQLTester
     }
 
     @Test
-    public void testFunctionsWithCompactStorage() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a int , b int, c double, primary key(a, b) ) WITH COMPACT STORAGE");
-
-        execute("INSERT INTO %s (a, b, c) VALUES (1, 1, 11.5)");
-        execute("INSERT INTO %s (a, b, c) VALUES (1, 2, 9.5)");
-        execute("INSERT INTO %s (a, b, c) VALUES (1, 3, 9.0)");
-
-        assertRows(execute("SELECT max(b), min(b), sum(b), avg(b) , max(c), sum(c), avg(c) FROM %s"),
-                   row(3, 1, 6, 2, 11.5, 30.0, 10.0));
-
-        assertRows(execute("SELECT COUNT(*) FROM %s"), row(3L));
-        assertRows(execute("SELECT COUNT(1) FROM %s"), row(3L));
-        assertRows(execute("SELECT COUNT(*) FROM %s WHERE a = 1 AND b > 1"), row(2L));
-        assertRows(execute("SELECT COUNT(1) FROM %s WHERE a = 1 AND b > 1"), row(2L));
-        assertRows(execute("SELECT max(b), min(b), sum(b), avg(b) , max(c), sum(c), avg(c) FROM %s WHERE a = 1 AND b > 1"),
-                   row(3, 2, 5, 2, 9.5, 18.5, 9.25));
-    }
-
-    @Test
     public void testInvalidCalls() throws Throwable
     {
         createTable("CREATE TABLE %s (a int, b int, c int, primary key (a, b))");

http://git-wip-us.apache.org/repos/asf/cassandra/blob/07fbd8ee/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
index ae93b6e..3790305 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/BatchTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/BatchTest.java
@@ -104,39 +104,6 @@ public class BatchTest extends CQLTester
     }
 
     @Test
-    public void testBatchRangeDelete() throws Throwable
-    {
-        createTable("CREATE TABLE %s (partitionKey int," +
-                "clustering int," +
-                "value int," +
-                " PRIMARY KEY (partitionKey, clustering)) WITH COMPACT STORAGE");
-
-        int value = 0;
-        for (int partitionKey = 0; partitionKey < 4; partitionKey++)
-            for (int clustering1 = 0; clustering1 < 5; clustering1++)
-                execute("INSERT INTO %s (partitionKey, clustering, value) VALUES (?, ?, ?)",
-                        partitionKey, clustering1, value++);
-
-        execute("BEGIN BATCH " +
-                "DELETE FROM %1$s WHERE partitionKey = 1;" +
-                "DELETE FROM %1$s WHERE partitionKey = 0 AND  clustering >= 4;" +
-                "DELETE FROM %1$s WHERE partitionKey = 0 AND clustering <= 0;" +
-                "DELETE FROM %1$s WHERE partitionKey = 2 AND clustering >= 0 AND clustering <= 3;" +
-                "DELETE FROM %1$s WHERE partitionKey = 2 AND clustering <= 3 AND clustering >= 4;" +
-                "DELETE FROM %1$s WHERE partitionKey = 3 AND (clustering) >= (3) AND (clustering) <= (6);" +
-                "APPLY BATCH;");
-
-        assertRows(execute("SELECT * FROM %s"),
-                   row(0, 1, 1),
-                   row(0, 2, 2),
-                   row(0, 3, 3),
-                   row(2, 4, 14),
-                   row(3, 0, 15),
-                   row(3, 1, 16),
-                   row(3, 2, 17));
-    }
-
-    @Test
     public void testBatchUpdate() throws Throwable
     {
         createTable("CREATE TABLE %s (partitionKey int," +

http://git-wip-us.apache.org/repos/asf/cassandra/blob/07fbd8ee/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
index e3c66c5..ac261ca 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/CreateTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/CreateTest.java
@@ -283,142 +283,7 @@ public class CreateTest extends CQLTester
                    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()
      */
@@ -462,8 +327,6 @@ public class CreateTest extends CQLTester
 
         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");
     }
 
     /**
@@ -551,32 +414,12 @@ public class CreateTest extends CQLTester
     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),)");
+        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");
     }
 
     /**
@@ -662,38 +505,6 @@ public class CreateTest extends CQLTester
     }
 
     @Test
-    public void testCreateIndexOnCompactTableWithClusteringColumns() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a int, b int , c int, PRIMARY KEY (a, b)) WITH COMPACT STORAGE;");
-
-        assertInvalidMessage("Secondary indexes are not supported on COMPACT STORAGE tables that have clustering columns",
-                             "CREATE INDEX ON %s (a);");
-
-        assertInvalidMessage("Secondary indexes are not supported on COMPACT STORAGE tables that have clustering columns",
-                             "CREATE INDEX ON %s (b);");
-
-        assertInvalidMessage("Secondary indexes are not supported on COMPACT STORAGE tables that have clustering columns",
-                             "CREATE INDEX ON %s (c);");
-    }
-
-    @Test
-    public void testCreateIndexOnCompactTableWithoutClusteringColumns() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a int PRIMARY KEY, b int) WITH COMPACT STORAGE;");
-
-        assertInvalidMessage("Secondary indexes are not supported on PRIMARY KEY columns in COMPACT STORAGE tables",
-                             "CREATE INDEX ON %s (a);");
-
-        createIndex("CREATE INDEX ON %s (b);");
-
-        execute("INSERT INTO %s (a, b) values (1, 1)");
-        execute("INSERT INTO %s (a, b) values (2, 4)");
-        execute("INSERT INTO %s (a, b) values (3, 6)");
-
-        assertRows(execute("SELECT * FROM %s WHERE b = ?", 4), row(2, 4));
-    }
-
-    @Test
     // tests CASSANDRA-4278
     public void testHyphenDatacenters() throws Throwable
     {
@@ -865,19 +676,27 @@ public class CreateTest extends CQLTester
                                             + " WITH compression = { 'class' : 'SnappyCompressor', 'unknownOption' : 32 };");
     }
 
-     private void assertThrowsConfigurationException(String errorMsg, String createStmt) {
-         try
-         {
-             createTable(createStmt);
-             fail("Query should be invalid but no error was thrown. Query is: " + createStmt);
-         }
-         catch (RuntimeException e)
-         {
-             Throwable cause = e.getCause();
-             assertTrue("The exception should be a ConfigurationException", cause instanceof ConfigurationException);
-             assertEquals(errorMsg, cause.getMessage());
-         }
-     }
+    @Test
+    public void compactTableTest() throws Throwable
+    {
+        assertInvalidMessage("Compact tables are not allowed in Cassandra starting with 4.0 version.",
+                             "CREATE TABLE compact_table_create (id text PRIMARY KEY, content text) WITH COMPACT STORAGE;");
+    }
+
+    private void assertThrowsConfigurationException(String errorMsg, String createStmt)
+    {
+        try
+        {
+            createTable(createStmt);
+            fail("Query should be invalid but no error was thrown. Query is: " + createStmt);
+        }
+        catch (RuntimeException e)
+        {
+            Throwable cause = e.getCause();
+            assertTrue("The exception should be a ConfigurationException", cause instanceof ConfigurationException);
+            assertEquals(errorMsg, cause.getMessage());
+        }
+    }
 
     private void assertTriggerExists(String name)
     {


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@cassandra.apache.org
For additional commands, e-mail: commits-help@cassandra.apache.org