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

[01/25] cassandra git commit: Allow dropping COMPACT STORAGE flag

Repository: cassandra
Updated Branches:
  refs/heads/cassandra-3.0 b8697441d -> 6c29ee84a
  refs/heads/cassandra-3.11 c8a3b58bd -> 6d429cd03
  refs/heads/trunk 5b09543f6 -> 07fbd8ee6


http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/test/unit/org/apache/cassandra/cql3/validation/operations/DropCompactStorageThriftTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/DropCompactStorageThriftTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/DropCompactStorageThriftTest.java
new file mode 100644
index 0000000..dde3e7b
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/DropCompactStorageThriftTest.java
@@ -0,0 +1,525 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.cql3.validation.operations;
+
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+
+import org.junit.Test;
+
+import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.cql3.ColumnSpecification;
+import org.apache.cassandra.cql3.UntypedResultSet;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.AsciiType;
+import org.apache.cassandra.db.marshal.BytesType;
+import org.apache.cassandra.db.marshal.CompositeType;
+import org.apache.cassandra.db.marshal.EmptyType;
+import org.apache.cassandra.db.marshal.Int32Type;
+import org.apache.cassandra.db.marshal.MapType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.locator.SimpleStrategy;
+import org.apache.cassandra.thrift.Cassandra;
+import org.apache.cassandra.thrift.CfDef;
+import org.apache.cassandra.thrift.Column;
+import org.apache.cassandra.thrift.ColumnDef;
+import org.apache.cassandra.thrift.ColumnOrSuperColumn;
+import org.apache.cassandra.thrift.ColumnParent;
+import org.apache.cassandra.thrift.IndexType;
+import org.apache.cassandra.thrift.KsDef;
+import org.apache.cassandra.thrift.Mutation;
+import org.apache.cassandra.thrift.SuperColumn;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+import static org.apache.cassandra.thrift.ConsistencyLevel.ONE;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+public class DropCompactStorageThriftTest extends ThriftCQLTester
+{
+    @Test
+    public void thriftCreatedTableTest() throws Throwable
+    {
+        final String KEYSPACE = "thrift_created_table_test_ks";
+        final String TABLE = "test_table_1";
+
+        CfDef cfDef = new CfDef().setDefault_validation_class(Int32Type.instance.toString())
+                                 .setKey_validation_class(AsciiType.instance.toString())
+                                 .setComparator_type(AsciiType.instance.toString())
+                                 .setColumn_metadata(Arrays.asList(new ColumnDef(ByteBufferUtil.bytes("col1"),
+                                                                                 AsciiType.instance.toString())
+                                                                   .setIndex_name("col1Index")
+                                                                   .setIndex_type(IndexType.KEYS),
+                                                                   new ColumnDef(ByteBufferUtil.bytes("col2"),
+                                                                                 AsciiType.instance.toString())
+                                                                   .setIndex_name("col2Index")
+                                                                   .setIndex_type(IndexType.KEYS)))
+                                 .setKeyspace(KEYSPACE)
+                                 .setName(TABLE);
+
+        KsDef ksDef = new KsDef(KEYSPACE,
+                                SimpleStrategy.class.getName(),
+                                Arrays.asList(cfDef));
+        ksDef.setStrategy_options(Collections.singletonMap("replication_factor", "1"));
+
+        Cassandra.Client client = getClient();
+        client.system_add_keyspace(ksDef);
+        client.set_keyspace(KEYSPACE);
+
+        client.insert(UTF8Type.instance.decompose("key1"),
+                      new ColumnParent(TABLE),
+                      getColumnForInsert(ByteBufferUtil.bytes("col1"), ByteBufferUtil.bytes("val1")),
+                      ONE);
+
+        client.insert(UTF8Type.instance.decompose("key1"),
+                      new ColumnParent(TABLE),
+                      getColumnForInsert(ByteBufferUtil.bytes("col2"), ByteBufferUtil.bytes("val2")),
+                      ONE);
+
+        client.insert(UTF8Type.instance.decompose("key1"),
+                      new ColumnParent(TABLE),
+                      getColumnForInsert(ByteBufferUtil.bytes("dynamicKey1"), ByteBufferUtil.bytes(100)),
+                      ONE);
+
+        client.insert(UTF8Type.instance.decompose("key1"),
+                      new ColumnParent(TABLE),
+                      getColumnForInsert(ByteBufferUtil.bytes("dynamicKey2"), ByteBufferUtil.bytes(200)),
+                      ONE);
+
+
+        execute(String.format("ALTER TABLE %s.%s DROP COMPACT STORAGE;", KEYSPACE, TABLE));
+        UntypedResultSet resultSet = execute(String.format("select * from %s.%s",
+                                                           KEYSPACE, TABLE));
+        assertColumnType(AsciiType.instance, resultSet, "key");
+        assertColumnType(AsciiType.instance, resultSet, "column1");
+        assertColumnType(Int32Type.instance, resultSet, "value");
+        assertColumnType(AsciiType.instance, resultSet, "col1");
+        assertColumnType(AsciiType.instance, resultSet, "col2");
+
+        assertRows(resultSet,
+                   row("key1", "dynamicKey1", "val1", "val2", 100),
+                   row("key1", "dynamicKey2", "val1", "val2", 200));
+    }
+
+    @Test
+    public void thriftStaticCompatTableTest() throws Throwable
+    {
+        String KEYSPACE = keyspace();
+        String TABLE = createTable("CREATE TABLE %s (key ascii PRIMARY KEY, val ascii) WITH COMPACT STORAGE");
+
+        Cassandra.Client client = getClient();
+        client.set_keyspace(KEYSPACE);
+
+        client.insert(UTF8Type.instance.decompose("key1"),
+                      new ColumnParent(TABLE),
+                      getColumnForInsert(ByteBufferUtil.bytes("val"), ByteBufferUtil.bytes("val1")),
+                      ONE);
+
+        client.insert(UTF8Type.instance.decompose("key1"),
+                      new ColumnParent(TABLE),
+                      getColumnForInsert(ByteBufferUtil.bytes("dynamicKey1"), ByteBufferUtil.bytes("dynamicValue1")),
+                      ONE);
+
+        client.insert(UTF8Type.instance.decompose("key1"),
+                      new ColumnParent(TABLE),
+                      getColumnForInsert(ByteBufferUtil.bytes("dynamicKey2"), ByteBufferUtil.bytes("dynamicValue2")),
+                      ONE);
+
+        execute(String.format("ALTER TABLE %s.%s DROP COMPACT STORAGE;", KEYSPACE, TABLE));
+        UntypedResultSet resultSet = execute(String.format("select * from %s.%s",
+                                                           KEYSPACE, TABLE));
+        assertColumnType(AsciiType.instance, resultSet, "key");
+        assertColumnType(UTF8Type.instance, resultSet, "column1");
+        assertColumnType(AsciiType.instance, resultSet, "val");
+        assertColumnType(BytesType.instance, resultSet, "value");
+
+        // Values are interpreted as bytes by default:
+        assertRows(resultSet,
+                   row("key1", "dynamicKey1", "val1", ByteBufferUtil.bytes("dynamicValue1")),
+                   row("key1", "dynamicKey2", "val1", ByteBufferUtil.bytes("dynamicValue2")));
+    }
+
+    @Test
+    public void testSparseCompactTableIndex() throws Throwable
+    {
+        createTable("CREATE TABLE %s (key ascii PRIMARY KEY, val ascii) WITH COMPACT STORAGE");
+
+        // Indexes are allowed only on the sparse compact tables
+        createIndex("CREATE INDEX ON %s(val)");
+        for (int i = 0; i < 10; i++)
+            execute("INSERT INTO %s (key, val) VALUES (?, ?)", Integer.toString(i), Integer.toString(i * 10));
+
+        alterTable("ALTER TABLE %s DROP COMPACT STORAGE");
+
+        assertRows(execute("SELECT * FROM %s WHERE val = '50'"),
+                   row("5", null, "50", null));
+        assertRows(execute("SELECT * FROM %s WHERE key = '5'"),
+                   row("5", null, "50", null));
+    }
+
+    @Test
+    public void thriftCompatTableTest() throws Throwable
+    {
+        String KEYSPACE = keyspace();
+        String TABLE = createTable("CREATE TABLE %s (pkey ascii, ckey ascii, PRIMARY KEY (pkey, ckey)) WITH COMPACT STORAGE");
+
+        Cassandra.Client client = getClient();
+        client.set_keyspace(KEYSPACE);
+
+        client.insert(UTF8Type.instance.decompose("key1"),
+                      new ColumnParent(TABLE),
+                      getColumnForInsert(ByteBufferUtil.bytes("ckeyValue1"), ByteBufferUtil.EMPTY_BYTE_BUFFER),
+                      ONE);
+
+        client.insert(UTF8Type.instance.decompose("key1"),
+                      new ColumnParent(TABLE),
+                      getColumnForInsert(ByteBufferUtil.bytes("ckeyValue2"), ByteBufferUtil.EMPTY_BYTE_BUFFER),
+                      ONE);
+
+        execute(String.format("ALTER TABLE %s.%s DROP COMPACT STORAGE;", KEYSPACE, TABLE));
+        UntypedResultSet resultSet = execute(String.format("select * from %s.%s",
+                                                           KEYSPACE, TABLE));
+        assertColumnType(AsciiType.instance, resultSet, "pkey");
+        assertColumnType(AsciiType.instance, resultSet, "ckey");
+        assertColumnType(EmptyType.instance, resultSet, "value");
+
+        // Value is always empty
+        assertRows(resultSet,
+                   row("key1", "ckeyValue1", ByteBufferUtil.EMPTY_BYTE_BUFFER),
+                   row("key1", "ckeyValue2", ByteBufferUtil.EMPTY_BYTE_BUFFER));
+    }
+
+    @Test
+    public void thriftDenseTableTest() throws Throwable
+    {
+        String KEYSPACE = keyspace();
+        String TABLE = createTable("CREATE TABLE %s (pkey text, ckey text, v text, PRIMARY KEY (pkey, ckey)) WITH COMPACT STORAGE");
+
+        Cassandra.Client client = getClient();
+        client.set_keyspace(KEYSPACE);
+
+        client.insert(UTF8Type.instance.decompose("key1"),
+                      new ColumnParent(TABLE),
+                      getColumnForInsert(ByteBufferUtil.bytes("ckey1"), ByteBufferUtil.bytes("cvalue1")),
+                      ONE);
+
+        client.insert(UTF8Type.instance.decompose("key1"),
+                      new ColumnParent(TABLE),
+                      getColumnForInsert(ByteBufferUtil.bytes("ckey2"), ByteBufferUtil.bytes("cvalue2")),
+                      ONE);
+
+        execute(String.format("ALTER TABLE %s.%s DROP COMPACT STORAGE;", KEYSPACE, TABLE));
+        UntypedResultSet resultSet = execute(String.format("select * from %s.%s",
+                                                           KEYSPACE, TABLE));
+        assertColumnType(UTF8Type.instance, resultSet, "pkey");
+        assertColumnType(UTF8Type.instance, resultSet, "ckey");
+        assertColumnType(UTF8Type.instance, resultSet, "v");
+
+        assertRows(resultSet,
+                   row("key1", "ckey1", "cvalue1"),
+                   row("key1", "ckey2", "cvalue2"));
+    }
+
+    @Test
+    public void thriftTableWithIntKey() throws Throwable
+    {
+        final String KEYSPACE = "thrift_table_with_int_key_ks";
+        final String TABLE = "test_table_1";
+
+        ByteBuffer columnName = ByteBufferUtil.bytes("columnname");
+        CfDef cfDef = new CfDef().setDefault_validation_class(UTF8Type.instance.toString())
+                                 .setKey_validation_class(BytesType.instance.toString())
+                                 .setComparator_type(BytesType.instance.toString())
+                                 .setColumn_metadata(Arrays.asList(new ColumnDef(columnName,
+                                                                                 Int32Type.instance.toString())
+                                                                   .setIndex_name("col1Index")
+                                                                   .setIndex_type(IndexType.KEYS)))
+                                 .setKeyspace(KEYSPACE)
+                                 .setName(TABLE);
+
+        KsDef ksDef = new KsDef(KEYSPACE,
+                                SimpleStrategy.class.getName(),
+                                Arrays.asList(cfDef));
+        ksDef.setStrategy_options(Collections.singletonMap("replication_factor", "1"));
+
+        Cassandra.Client client = getClient();
+        client.system_add_keyspace(ksDef);
+        client.set_keyspace(KEYSPACE);
+
+        client.insert(UTF8Type.instance.decompose("key1"),
+                      new ColumnParent(TABLE),
+                      getColumnForInsert(columnName, ByteBufferUtil.bytes(100)),
+                      ONE);
+
+        execute(String.format("ALTER TABLE %s.%s DROP COMPACT STORAGE;", KEYSPACE, TABLE));
+        UntypedResultSet resultSet = execute(String.format("select * from %s.%s",
+                                                           KEYSPACE, TABLE));
+
+        assertEquals(resultSet.metadata()
+                              .stream()
+                              .filter((cs) -> cs.name.toString().equals(BytesType.instance.getString(columnName)))
+                              .findFirst()
+                              .get().type,
+                     Int32Type.instance);
+
+        assertRows(resultSet,
+                   row(UTF8Type.instance.decompose("key1"), null, 100, null));
+    }
+
+    @Test
+    public void thriftCompatTableWithSupercolumnsTest() throws Throwable
+    {
+        final String KEYSPACE = "thrift_compact_table_with_supercolumns_test";
+        final String TABLE = "test_table_1";
+
+        CfDef cfDef = new CfDef().setColumn_type("Super")
+                                 .setSubcomparator_type(Int32Type.instance.toString())
+                                 .setComparator_type(AsciiType.instance.toString())
+                                 .setDefault_validation_class(AsciiType.instance.toString())
+                                 .setKey_validation_class(AsciiType.instance.toString())
+                                 .setKeyspace(KEYSPACE)
+                                 .setName(TABLE);
+
+        KsDef ksDef = new KsDef(KEYSPACE,
+                                SimpleStrategy.class.getName(),
+                                Arrays.asList(cfDef));
+        ksDef.setStrategy_options(Collections.singletonMap("replication_factor", "1"));
+
+        Cassandra.Client client = getClient();
+        client.system_add_keyspace(ksDef);
+
+        client.set_keyspace(KEYSPACE);
+
+        Mutation mutation = new Mutation();
+        ColumnOrSuperColumn csoc = new ColumnOrSuperColumn();
+        csoc.setSuper_column(getSuperColumnForInsert(ByteBufferUtil.bytes("val1"),
+                                                     Arrays.asList(getColumnForInsert(ByteBufferUtil.bytes(1), ByteBufferUtil.bytes("value1")),
+                                                                   getColumnForInsert(ByteBufferUtil.bytes(2), ByteBufferUtil.bytes("value2")),
+                                                                   getColumnForInsert(ByteBufferUtil.bytes(3), ByteBufferUtil.bytes("value3")))));
+        mutation.setColumn_or_supercolumn(csoc);
+
+        Mutation mutation2 = new Mutation();
+        ColumnOrSuperColumn csoc2 = new ColumnOrSuperColumn();
+        csoc2.setSuper_column(getSuperColumnForInsert(ByteBufferUtil.bytes("val2"),
+                                                     Arrays.asList(getColumnForInsert(ByteBufferUtil.bytes(4), ByteBufferUtil.bytes("value7")),
+                                                                   getColumnForInsert(ByteBufferUtil.bytes(5), ByteBufferUtil.bytes("value8")),
+                                                                   getColumnForInsert(ByteBufferUtil.bytes(6), ByteBufferUtil.bytes("value9")))));
+        mutation2.setColumn_or_supercolumn(csoc2);
+
+        client.batch_mutate(Collections.singletonMap(ByteBufferUtil.bytes("key1"),
+                                                     Collections.singletonMap(TABLE, Arrays.asList(mutation, mutation2))),
+                            ONE);
+
+        execute(String.format("ALTER TABLE %s.%s DROP COMPACT STORAGE;", KEYSPACE, TABLE));
+        UntypedResultSet resultSet = execute(String.format("select * from %s.%s",
+                                                           KEYSPACE, TABLE));
+        assertColumnType(AsciiType.instance, resultSet, "key");
+        assertColumnType(AsciiType.instance, resultSet, "column1");
+        assertColumnType(MapType.getInstance(Int32Type.instance, AsciiType.instance, true), resultSet, "");
+
+        assertRows(resultSet,
+                   row("key1", "val1", map(1, "value1", 2, "value2", 3, "value3")),
+                   row("key1", "val2", map(4, "value7", 5, "value8", 6, "value9")));
+
+        assertRows(execute(String.format("SELECT \"\" FROM %s.%s;", KEYSPACE, TABLE)),
+                   row(map(1, "value1", 2, "value2", 3, "value3")),
+                   row(map(4, "value7", 5, "value8", 6, "value9")));
+
+        assertInvalidMessage("Range deletions are not supported for specific columns",
+                             String.format("DELETE \"\" FROM %s.%s WHERE key=?;", KEYSPACE, TABLE),
+                             "key1");
+
+        execute(String.format("TRUNCATE %s.%s;", KEYSPACE, TABLE));
+
+        execute(String.format("INSERT INTO %s.%s (key, column1, \"\") VALUES (?, ?, ?);", KEYSPACE, TABLE),
+                "key3", "val1", map(7, "value7", 8, "value8"));
+
+        assertRows(execute(String.format("SELECT \"\" FROM %s.%s;", KEYSPACE, TABLE)),
+                   row(map(7, "value7", 8, "value8")));
+    }
+
+    @Test
+    public void thriftCreatedTableWithCompositeColumnsTest() throws Throwable
+    {
+        final String KEYSPACE = "thrift_created_table_with_composites_test_ks";
+        final String TABLE = "test_table_1";
+
+        CompositeType type = CompositeType.getInstance(AsciiType.instance, AsciiType.instance, AsciiType.instance);
+        CfDef cfDef = new CfDef().setDefault_validation_class(AsciiType.instance.toString())
+                                 .setComparator_type(type.toString())
+                                 .setKey_validation_class(AsciiType.instance.toString())
+                                 .setKeyspace(KEYSPACE)
+                                 .setName(TABLE);
+
+        KsDef ksDef = new KsDef(KEYSPACE,
+                                SimpleStrategy.class.getName(),
+                                Arrays.asList(cfDef));
+        ksDef.setStrategy_options(Collections.singletonMap("replication_factor", "1"));
+
+        Cassandra.Client client = getClient();
+        client.system_add_keyspace(ksDef);
+        client.set_keyspace(KEYSPACE);
+
+        client.insert(UTF8Type.instance.decompose("key1"),
+                      new ColumnParent(TABLE),
+                      getColumnForInsert(type.decompose("a", "b", "c"), ByteBufferUtil.bytes("val1")),
+                      ONE);
+
+        client.insert(UTF8Type.instance.decompose("key1"),
+                      new ColumnParent(TABLE),
+                      getColumnForInsert(type.decompose("d", "e", "f"), ByteBufferUtil.bytes("val2")),
+                      ONE);
+
+        execute(String.format("ALTER TABLE %s.%s DROP COMPACT STORAGE;", KEYSPACE, TABLE));
+        UntypedResultSet resultSet = execute(String.format("select * from %s.%s",
+                                                           KEYSPACE, TABLE));
+
+        assertColumnType(AsciiType.instance, resultSet, "key");
+        assertColumnType(AsciiType.instance, resultSet, "column1");
+        assertColumnType(AsciiType.instance, resultSet, "column2");
+        assertColumnType(AsciiType.instance, resultSet, "column3");
+        assertColumnType(AsciiType.instance, resultSet, "value");
+
+        assertRows(resultSet,
+                   row("key1", "a", "b", "c", "val1"),
+                   row("key1", "d", "e", "f", "val2"));
+    }
+
+    @Test
+    public void compactTableWithoutClusteringKeyTest() throws Throwable
+    {
+        String KEYSPACE = keyspace();
+        String TABLE = createTable("CREATE TABLE %s (pkey text PRIMARY KEY, s1 text, s2 text) WITH COMPACT STORAGE");
+
+        Cassandra.Client client = getClient();
+        client.set_keyspace(KEYSPACE);
+
+        client.insert(UTF8Type.instance.decompose("key1"),
+                      new ColumnParent(TABLE),
+                      getColumnForInsert(ByteBufferUtil.bytes("ckey1"), ByteBufferUtil.bytes("val1")),
+                      ONE);
+
+        client.insert(UTF8Type.instance.decompose("key1"),
+                      new ColumnParent(TABLE),
+                      getColumnForInsert(ByteBufferUtil.bytes("ckey2"), ByteBufferUtil.bytes("val2")),
+                      ONE);
+
+        client.insert(UTF8Type.instance.decompose("key1"),
+                      new ColumnParent(TABLE),
+                      getColumnForInsert(ByteBufferUtil.bytes("s1"), ByteBufferUtil.bytes("s1Val")),
+                      ONE);
+
+        client.insert(UTF8Type.instance.decompose("key1"),
+                      new ColumnParent(TABLE),
+                      getColumnForInsert(ByteBufferUtil.bytes("s2"), ByteBufferUtil.bytes("s2Val")),
+                      ONE);
+
+        execute(String.format("ALTER TABLE %s.%s DROP COMPACT STORAGE;", KEYSPACE, TABLE));
+        UntypedResultSet resultSet = execute(String.format("select * from %s.%s",
+                                                           KEYSPACE, TABLE));
+
+        assertColumnType(UTF8Type.instance, resultSet, "pkey");
+        assertColumnType(UTF8Type.instance, resultSet, "s1");
+        assertColumnType(UTF8Type.instance, resultSet, "s2");
+        assertColumnType(UTF8Type.instance, resultSet, "column1");
+        assertColumnType(BytesType.instance, resultSet, "value");
+
+        assertRows(resultSet,
+                   row("key1", "ckey1", "s1Val", "s2Val", ByteBufferUtil.bytes("val1")),
+                   row("key1", "ckey2", "s1Val", "s2Val", ByteBufferUtil.bytes("val2")));
+    }
+
+    @Test
+    public void denseTableTestTest() throws Throwable
+    {
+        String KEYSPACE = keyspace();
+        String TABLE = createTable("CREATE TABLE %s (pkey text PRIMARY KEY, s text) WITH COMPACT STORAGE");
+
+        Cassandra.Client client = getClient();
+        client.set_keyspace(KEYSPACE);
+
+        client.insert(UTF8Type.instance.decompose("key1"),
+                      new ColumnParent(TABLE),
+                      getColumnForInsert(ByteBufferUtil.bytes("ckey1"), ByteBufferUtil.bytes("val1")),
+                      ONE);
+
+        client.insert(UTF8Type.instance.decompose("key1"),
+                      new ColumnParent(TABLE),
+                      getColumnForInsert(ByteBufferUtil.bytes("ckey2"), ByteBufferUtil.bytes("val2")),
+                      ONE);
+
+        client.insert(UTF8Type.instance.decompose("key1"),
+                      new ColumnParent(TABLE),
+                      getColumnForInsert(ByteBufferUtil.bytes("s"), ByteBufferUtil.bytes("sval1")),
+                      ONE);
+
+        client.insert(UTF8Type.instance.decompose("key1"),
+                      new ColumnParent(TABLE),
+                      getColumnForInsert(ByteBufferUtil.bytes("s"), ByteBufferUtil.bytes("sval2")),
+                      ONE);
+
+        // `s` becomes static, `column1` becomes a clustering key, `value` becomes visible
+        execute(String.format("ALTER TABLE %s.%s DROP COMPACT STORAGE;", KEYSPACE, TABLE));
+        UntypedResultSet resultSet = execute(String.format("select * from %s.%s",
+                                                           KEYSPACE, TABLE));
+        assertColumnType(UTF8Type.instance, resultSet, "pkey");
+        assertColumnType(UTF8Type.instance, resultSet, "s");
+        assertColumnType(UTF8Type.instance, resultSet, "column1");
+        assertColumnType(BytesType.instance, resultSet, "value");
+
+        assertRows(resultSet,
+                   row("key1", "ckey1", "sval2", ByteBufferUtil.bytes("val1")),
+                   row("key1", "ckey2", "sval2", ByteBufferUtil.bytes("val2")));
+    }
+
+    private Column getColumnForInsert(ByteBuffer columnName, ByteBuffer value)
+    {
+        Column column = new Column();
+        column.setName(columnName);
+        column.setValue(value);
+        column.setTimestamp(System.currentTimeMillis());
+        return column;
+    }
+
+    private SuperColumn getSuperColumnForInsert(ByteBuffer columnName, List<Column> columns)
+    {
+        SuperColumn column = new SuperColumn();
+        column.setName(columnName);
+        for (Column c : columns)
+            column.addToColumns(c);
+        return column;
+    }
+
+    private static void assertColumnType(AbstractType t, UntypedResultSet resultSet, String columnName)
+    {
+        for (ColumnSpecification columnSpecification : resultSet.metadata())
+        {
+            if (columnSpecification.name.toString().equals(columnName))
+            {
+                assertEquals(t, columnSpecification.type);
+                return;
+            }
+        }
+
+        fail(String.format("Could not find a column with name '%s'", columnName));
+    }
+}


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


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

Posted by if...@apache.org.
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


[12/25] cassandra git commit: Merge branch 'cassandra-3.0' into cassandra-3.11

Posted by if...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d429cd0/tools/stress/src/org/apache/cassandra/io/sstable/StressCQLSSTableWriter.java
----------------------------------------------------------------------
diff --cc tools/stress/src/org/apache/cassandra/io/sstable/StressCQLSSTableWriter.java
index 56f0297,0000000..5a285e1
mode 100644,000000..100644
--- a/tools/stress/src/org/apache/cassandra/io/sstable/StressCQLSSTableWriter.java
+++ b/tools/stress/src/org/apache/cassandra/io/sstable/StressCQLSSTableWriter.java
@@@ -1,676 -1,0 +1,676 @@@
 +/*
 + * 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.io.sstable;
 +
 +import java.io.Closeable;
 +import java.io.File;
 +import java.io.IOException;
 +import java.nio.ByteBuffer;
 +import java.util.*;
 +import java.util.stream.Collectors;
 +
 +import com.datastax.driver.core.ProtocolVersion;
 +import com.datastax.driver.core.TypeCodec;
 +import org.antlr.runtime.RecognitionException;
 +import org.apache.cassandra.config.CFMetaData;
 +import org.apache.cassandra.config.DatabaseDescriptor;
 +import org.apache.cassandra.config.Schema;
 +import org.apache.cassandra.cql3.CQLFragmentParser;
 +import org.apache.cassandra.cql3.ColumnSpecification;
 +import org.apache.cassandra.cql3.CqlParser;
 +import org.apache.cassandra.cql3.QueryOptions;
 +import org.apache.cassandra.cql3.UpdateParameters;
 +import org.apache.cassandra.cql3.functions.UDHelper;
 +import org.apache.cassandra.cql3.statements.CreateTableStatement;
 +import org.apache.cassandra.cql3.statements.CreateTypeStatement;
 +import org.apache.cassandra.cql3.statements.ParsedStatement;
 +import org.apache.cassandra.cql3.statements.UpdateStatement;
 +import org.apache.cassandra.db.*;
 +import org.apache.cassandra.db.marshal.UserType;
 +import org.apache.cassandra.db.partitions.Partition;
 +import org.apache.cassandra.dht.IPartitioner;
 +import org.apache.cassandra.dht.Murmur3Partitioner;
 +import org.apache.cassandra.exceptions.InvalidRequestException;
 +import org.apache.cassandra.exceptions.RequestValidationException;
 +import org.apache.cassandra.exceptions.SyntaxException;
 +import org.apache.cassandra.io.sstable.format.SSTableFormat;
 +import org.apache.cassandra.schema.KeyspaceMetadata;
 +import org.apache.cassandra.schema.KeyspaceParams;
 +import org.apache.cassandra.schema.Types;
 +import org.apache.cassandra.service.ClientState;
 +import org.apache.cassandra.utils.ByteBufferUtil;
 +import org.apache.cassandra.utils.Pair;
 +
 +/**
 + * Utility to write SSTables.
 + * <p>
 + * Typical usage looks like:
 + * <pre>
 + *   String type = CREATE TYPE myKs.myType (a int, b int)";
 + *   String schema = "CREATE TABLE myKs.myTable ("
 + *                 + "  k int PRIMARY KEY,"
 + *                 + "  v1 text,"
 + *                 + "  v2 int,"
 + *                 + "  v3 myType,"
 + *                 + ")";
 + *   String insert = "INSERT INTO myKs.myTable (k, v1, v2, v3) VALUES (?, ?, ?, ?)";
 + *
 + *   // Creates a new writer. You need to provide at least the directory where to write the created sstable,
 + *   // the schema for the sstable to write and a (prepared) insert statement to use. If you do not use the
 + *   // default partitioner (Murmur3Partitioner), you will also need to provide the partitioner in use, see
 + *   // StressCQLSSTableWriter.Builder for more details on the available options.
 + *   StressCQLSSTableWriter writer = StressCQLSSTableWriter.builder()
 + *                                             .inDirectory("path/to/directory")
 + *                                             .withType(type)
 + *                                             .forTable(schema)
 + *                                             .using(insert).build();
 + *
 + *   UserType myType = writer.getUDType("myType");
 + *   // Adds a nember of rows to the resulting sstable
 + *   writer.addRow(0, "test1", 24, myType.newValue().setInt("a", 10).setInt("b", 20));
 + *   writer.addRow(1, "test2", null, null);
 + *   writer.addRow(2, "test3", 42, myType.newValue().setInt("a", 30).setInt("b", 40));
 + *
 + *   // Close the writer, finalizing the sstable
 + *   writer.close();
 + * </pre>
 + *
 + * Please note that {@code StressCQLSSTableWriter} is <b>not</b> thread-safe (multiple threads cannot access the
 + * same instance). It is however safe to use multiple instances in parallel (even if those instance write
 + * sstables for the same table).
 + */
 +public class StressCQLSSTableWriter implements Closeable
 +{
 +    public static final ByteBuffer UNSET_VALUE = ByteBufferUtil.UNSET_BYTE_BUFFER;
 +
 +    static
 +    {
 +        DatabaseDescriptor.clientInitialization(false);
 +        // Partitioner is not set in client mode.
 +        if (DatabaseDescriptor.getPartitioner() == null)
 +            DatabaseDescriptor.setPartitionerUnsafe(Murmur3Partitioner.instance);
 +    }
 +
 +    private final AbstractSSTableSimpleWriter writer;
 +    private final UpdateStatement insert;
 +    private final List<ColumnSpecification> boundNames;
 +    private final List<TypeCodec> typeCodecs;
 +    private final ColumnFamilyStore cfs;
 +
 +    private StressCQLSSTableWriter(ColumnFamilyStore cfs, AbstractSSTableSimpleWriter writer, UpdateStatement insert, List<ColumnSpecification> boundNames)
 +    {
 +        this.cfs = cfs;
 +        this.writer = writer;
 +        this.insert = insert;
 +        this.boundNames = boundNames;
 +        this.typeCodecs = boundNames.stream().map(bn ->  UDHelper.codecFor(UDHelper.driverType(bn.type)))
 +                                             .collect(Collectors.toList());
 +    }
 +
 +    /**
 +     * Returns a new builder for a StressCQLSSTableWriter.
 +     *
 +     * @return the new builder.
 +     */
 +    public static Builder builder()
 +    {
 +        return new Builder();
 +    }
 +
 +    /**
 +     * Adds a new row to the writer.
 +     * <p>
 +     * This is a shortcut for {@code addRow(Arrays.asList(values))}.
 +     *
 +     * @param values the row values (corresponding to the bind variables of the
 +     * insertion statement used when creating by this writer).
 +     * @return this writer.
 +     */
 +    public StressCQLSSTableWriter addRow(Object... values)
 +    throws InvalidRequestException, IOException
 +    {
 +        return addRow(Arrays.asList(values));
 +    }
 +
 +    /**
 +     * Adds a new row to the writer.
 +     * <p>
 +     * Each provided value type should correspond to the types of the CQL column
 +     * the value is for. The correspondance between java type and CQL type is the
 +     * same one than the one documented at
 +     * www.datastax.com/drivers/java/2.0/apidocs/com/datastax/driver/core/DataType.Name.html#asJavaClass().
 +     * <p>
 +     * If you prefer providing the values directly as binary, use
 +     * {@link #rawAddRow} instead.
 +     *
 +     * @param values the row values (corresponding to the bind variables of the
 +     * insertion statement used when creating by this writer).
 +     * @return this writer.
 +     */
 +    public StressCQLSSTableWriter addRow(List<Object> values)
 +    throws InvalidRequestException, IOException
 +    {
 +        int size = Math.min(values.size(), boundNames.size());
 +        List<ByteBuffer> rawValues = new ArrayList<>(size);
 +
 +        for (int i = 0; i < size; i++)
 +        {
 +            Object value = values.get(i);
 +            rawValues.add(serialize(value, typeCodecs.get(i)));
 +        }
 +
 +        return rawAddRow(rawValues);
 +    }
 +
 +    /**
 +     * Adds a new row to the writer.
 +     * <p>
 +     * This is equivalent to the other addRow methods, but takes a map whose
 +     * keys are the names of the columns to add instead of taking a list of the
 +     * values in the order of the insert statement used during construction of
 +     * this write.
 +     * <p>
 +     * Please note that the column names in the map keys must be in lowercase unless
 +     * the declared column name is a
 +     * <a href="http://cassandra.apache.org/doc/cql3/CQL.html#identifiers">case-sensitive quoted identifier</a>
 +     * (in which case the map key must use the exact case of the column).
 +     *
 +     * @param values a map of colum name to column values representing the new
 +     * row to add. Note that if a column is not part of the map, it's value will
 +     * be {@code null}. If the map contains keys that does not correspond to one
 +     * of the column of the insert statement used when creating this writer, the
 +     * the corresponding value is ignored.
 +     * @return this writer.
 +     */
 +    public StressCQLSSTableWriter addRow(Map<String, Object> values)
 +    throws InvalidRequestException, IOException
 +    {
 +        int size = boundNames.size();
 +        List<ByteBuffer> rawValues = new ArrayList<>(size);
 +        for (int i = 0; i < size; i++)
 +        {
 +            ColumnSpecification spec = boundNames.get(i);
 +            Object value = values.get(spec.name.toString());
 +            rawValues.add(serialize(value, typeCodecs.get(i)));
 +        }
 +        return rawAddRow(rawValues);
 +    }
 +
 +    /**
 +     * Adds a new row to the writer given already serialized values.
 +     *
 +     * @param values the row values (corresponding to the bind variables of the
 +     * insertion statement used when creating by this writer) as binary.
 +     * @return this writer.
 +     */
 +    public StressCQLSSTableWriter rawAddRow(ByteBuffer... values)
 +    throws InvalidRequestException, IOException
 +    {
 +        return rawAddRow(Arrays.asList(values));
 +    }
 +
 +    /**
 +     * Adds a new row to the writer given already serialized values.
 +     * <p>
 +     * This is a shortcut for {@code rawAddRow(Arrays.asList(values))}.
 +     *
 +     * @param values the row values (corresponding to the bind variables of the
 +     * insertion statement used when creating by this writer) as binary.
 +     * @return this writer.
 +     */
 +    public StressCQLSSTableWriter rawAddRow(List<ByteBuffer> values)
 +    throws InvalidRequestException, IOException
 +    {
 +        if (values.size() != boundNames.size())
 +            throw new InvalidRequestException(String.format("Invalid number of arguments, expecting %d values but got %d", boundNames.size(), values.size()));
 +
 +        QueryOptions options = QueryOptions.forInternalCalls(null, values);
 +        List<ByteBuffer> keys = insert.buildPartitionKeyNames(options);
 +        SortedSet<Clustering> clusterings = insert.createClustering(options);
 +
 +        long now = System.currentTimeMillis() * 1000;
 +        // Note that we asks indexes to not validate values (the last 'false' arg below) because that triggers a 'Keyspace.open'
 +        // and that forces a lot of initialization that we don't want.
 +        UpdateParameters params = new UpdateParameters(insert.cfm,
 +                                                       insert.updatedColumns(),
 +                                                       options,
 +                                                       insert.getTimestamp(now, options),
 +                                                       insert.getTimeToLive(options),
 +                                                       Collections.<DecoratedKey, Partition>emptyMap());
 +
 +        try
 +        {
 +            for (ByteBuffer key : keys)
 +            {
 +                for (Clustering clustering : clusterings)
 +                    insert.addUpdateForKey(writer.getUpdateFor(key), clustering, params);
 +            }
 +            return this;
 +        }
 +        catch (SSTableSimpleUnsortedWriter.SyncException e)
 +        {
 +            // If we use a BufferedWriter and had a problem writing to disk, the IOException has been
 +            // wrapped in a SyncException (see BufferedWriter below). We want to extract that IOE.
 +            throw (IOException)e.getCause();
 +        }
 +    }
 +
 +    /**
 +     * Adds a new row to the writer given already serialized values.
 +     * <p>
 +     * This is equivalent to the other rawAddRow methods, but takes a map whose
 +     * keys are the names of the columns to add instead of taking a list of the
 +     * values in the order of the insert statement used during construction of
 +     * this write.
 +     *
 +     * @param values a map of colum name to column values representing the new
 +     * row to add. Note that if a column is not part of the map, it's value will
 +     * be {@code null}. If the map contains keys that does not correspond to one
 +     * of the column of the insert statement used when creating this writer, the
 +     * the corresponding value is ignored.
 +     * @return this writer.
 +     */
 +    public StressCQLSSTableWriter rawAddRow(Map<String, ByteBuffer> values)
 +    throws InvalidRequestException, IOException
 +    {
 +        int size = Math.min(values.size(), boundNames.size());
 +        List<ByteBuffer> rawValues = new ArrayList<>(size);
 +        for (int i = 0; i < size; i++) 
 +        {
 +            ColumnSpecification spec = boundNames.get(i);
 +            rawValues.add(values.get(spec.name.toString()));
 +        }
 +        return rawAddRow(rawValues);
 +    }
 +
 +    /**
 +     * Returns the User Defined type, used in this SSTable Writer, that can
 +     * be used to create UDTValue instances.
 +     *
 +     * @param dataType name of the User Defined type
 +     * @return user defined type
 +     */
 +    public com.datastax.driver.core.UserType getUDType(String dataType)
 +    {
 +        KeyspaceMetadata ksm = Schema.instance.getKSMetaData(insert.keyspace());
 +        UserType userType = ksm.types.getNullable(ByteBufferUtil.bytes(dataType));
 +        return (com.datastax.driver.core.UserType) UDHelper.driverType(userType);
 +    }
 +
 +    /**
 +     * Close this writer.
 +     * <p>
 +     * This method should be called, otherwise the produced sstables are not
 +     * guaranteed to be complete (and won't be in practice).
 +     */
 +    public void close() throws IOException
 +    {
 +        writer.close();
 +    }
 +
 +    private ByteBuffer serialize(Object value, TypeCodec codec)
 +    {
 +        if (value == null || value == UNSET_VALUE)
 +            return (ByteBuffer) value;
 +
 +        return codec.serialize(value, ProtocolVersion.NEWEST_SUPPORTED);
 +    }
 +    /**
 +     * The writer loads data in directories corresponding to how they laid out on the server.
 +     * <p>
 +     * {keyspace}/{table-cfid}/
 +     *
 +     * This method can be used to fetch the innermost directory with the sstable components
 +     * @return The directory containing the sstable components
 +     */
 +    public File getInnermostDirectory()
 +    {
 +        return cfs.getDirectories().getDirectoryForNewSSTables();
 +    }
 +
 +    /**
 +     * A Builder for a StressCQLSSTableWriter object.
 +     */
 +    public static class Builder
 +    {
 +        private final List<File> directoryList;
 +        private ColumnFamilyStore cfs;
 +
 +        protected SSTableFormat.Type formatType = null;
 +
 +        private Boolean makeRangeAware = false;
 +
 +        private CreateTableStatement.RawStatement schemaStatement;
 +        private final List<CreateTypeStatement> typeStatements;
 +        private UpdateStatement.ParsedInsert insertStatement;
 +        private IPartitioner partitioner;
 +
 +        private boolean sorted = false;
 +        private long bufferSizeInMB = 128;
 +
 +        protected Builder()
 +        {
 +            this.typeStatements = new ArrayList<>();
 +            this.directoryList = new ArrayList<>();
 +        }
 +
 +        /**
 +         * The directory where to write the sstables.
 +         * <p>
 +         * This is a mandatory option.
 +         *
 +         * @param directory the directory to use, which should exists and be writable.
 +         * @return this builder.
 +         *
 +         * @throws IllegalArgumentException if {@code directory} doesn't exist or is not writable.
 +         */
 +        public Builder inDirectory(String directory)
 +        {
 +            return inDirectory(new File(directory));
 +        }
 +
 +        /**
 +         * The directory where to write the sstables (mandatory option).
 +         * <p>
 +         * This is a mandatory option.
 +         *
 +         * @param directory the directory to use, which should exist and be writable.
 +         * @return this builder.
 +         *
 +         * @throws IllegalArgumentException if {@code directory} doesn't exist or is not writable.
 +         */
 +        public Builder inDirectory(File directory)
 +        {
 +            if (!directory.exists())
 +                throw new IllegalArgumentException(directory + " doesn't exists");
 +            if (!directory.canWrite())
 +                throw new IllegalArgumentException(directory + " exists but is not writable");
 +
 +            directoryList.add(directory);
 +            return this;
 +        }
 +
 +        /**
 +         * A pre-instanciated ColumnFamilyStore
 +         * <p>
 +         * This is can be used in place of inDirectory and forTable
 +         *
 +         * @see #inDirectory(File)
 +         *
 +         * @param cfs the list of directories to use, which should exist and be writable.
 +         * @return this builder.
 +         *
 +         * @throws IllegalArgumentException if a directory doesn't exist or is not writable.
 +         */
 +        public Builder withCfs(ColumnFamilyStore cfs)
 +        {
 +            this.cfs = cfs;
 +            return this;
 +        }
 +
 +
 +        public Builder withType(String typeDefinition) throws SyntaxException
 +        {
 +            typeStatements.add(parseStatement(typeDefinition, CreateTypeStatement.class, "CREATE TYPE"));
 +            return this;
 +        }
 +
 +        /**
 +         * The schema (CREATE TABLE statement) for the table for which sstable are to be created.
 +         * <p>
 +         * Please note that the provided CREATE TABLE statement <b>must</b> use a fully-qualified
 +         * table name, one that include the keyspace name.
 +         * <p>
 +         * This is a mandatory option.
 +         *
 +         * @param schema the schema of the table for which sstables are to be created.
 +         * @return this builder.
 +         *
 +         * @throws IllegalArgumentException if {@code schema} is not a valid CREATE TABLE statement
 +         * or does not have a fully-qualified table name.
 +         */
 +        public Builder forTable(String schema)
 +        {
 +            this.schemaStatement = parseStatement(schema, CreateTableStatement.RawStatement.class, "CREATE TABLE");
 +            return this;
 +        }
 +
 +        /**
 +         * The partitioner to use.
 +         * <p>
 +         * By default, {@code Murmur3Partitioner} will be used. If this is not the partitioner used
 +         * by the cluster for which the SSTables are created, you need to use this method to
 +         * provide the correct partitioner.
 +         *
 +         * @param partitioner the partitioner to use.
 +         * @return this builder.
 +         */
 +        public Builder withPartitioner(IPartitioner partitioner)
 +        {
 +            this.partitioner = partitioner;
 +            return this;
 +        }
 +
 +
 +        /**
 +         * Specify if the sstable writer should be vnode range aware.
 +         * This will create a sstable per vnode range.
 +         *
 +         * @param makeRangeAware
 +         * @return
 +         */
 +        public Builder rangeAware(boolean makeRangeAware)
 +        {
 +            this.makeRangeAware = makeRangeAware;
 +            return this;
 +        }
 +
 +        /**
 +         * The INSERT statement defining the order of the values to add for a given CQL row.
 +         * <p>
 +         * Please note that the provided INSERT statement <b>must</b> use a fully-qualified
 +         * table name, one that include the keyspace name. Morewover, said statement must use
 +         * bind variables since it is those bind variables that will be bound to values by the
 +         * resulting writer.
 +         * <p>
 +         * This is a mandatory option, and this needs to be called after foTable().
 +         *
 +         * @param insert an insertion statement that defines the order
 +         * of column values to use.
 +         * @return this builder.
 +         *
 +         * @throws IllegalArgumentException if {@code insertStatement} is not a valid insertion
 +         * statement, does not have a fully-qualified table name or have no bind variables.
 +         */
 +        public Builder using(String insert)
 +        {
 +            this.insertStatement = parseStatement(insert, UpdateStatement.ParsedInsert.class, "INSERT");
 +            return this;
 +        }
 +
 +        /**
 +         * The size of the buffer to use.
 +         * <p>
 +         * This defines how much data will be buffered before being written as
 +         * a new SSTable. This correspond roughly to the data size that will have the created
 +         * sstable.
 +         * <p>
 +         * The default is 128MB, which should be reasonable for a 1GB heap. If you experience
 +         * OOM while using the writer, you should lower this value.
 +         *
 +         * @param size the size to use in MB.
 +         * @return this builder.
 +         */
 +        public Builder withBufferSizeInMB(int size)
 +        {
 +            this.bufferSizeInMB = size;
 +            return this;
 +        }
 +
 +        /**
 +         * Creates a StressCQLSSTableWriter that expects sorted inputs.
 +         * <p>
 +         * If this option is used, the resulting writer will expect rows to be
 +         * added in SSTable sorted order (and an exception will be thrown if that
 +         * is not the case during insertion). The SSTable sorted order means that
 +         * rows are added such that their partition key respect the partitioner
 +         * order.
 +         * <p>
 +         * You should thus only use this option is you know that you can provide
 +         * the rows in order, which is rarely the case. If you can provide the
 +         * rows in order however, using this sorted might be more efficient.
 +         * <p>
 +         * Note that if used, some option like withBufferSizeInMB will be ignored.
 +         *
 +         * @return this builder.
 +         */
 +        public Builder sorted()
 +        {
 +            this.sorted = true;
 +            return this;
 +        }
 +
 +        @SuppressWarnings("resource")
 +        public StressCQLSSTableWriter build()
 +        {
 +            if (directoryList.isEmpty() && cfs == null)
 +                throw new IllegalStateException("No output directories specified, you should provide a directory with inDirectory()");
 +            if (schemaStatement == null && cfs == null)
 +                throw new IllegalStateException("Missing schema, you should provide the schema for the SSTable to create with forTable()");
 +            if (insertStatement == null)
 +                throw new IllegalStateException("No insert statement specified, you should provide an insert statement through using()");
 +
 +            synchronized (StressCQLSSTableWriter.class)
 +            {
 +                if (cfs == null)
 +                    cfs = createOfflineTable(schemaStatement, typeStatements, directoryList);
 +
 +                if (partitioner == null)
 +                    partitioner = cfs.getPartitioner();
 +
 +                Pair<UpdateStatement, List<ColumnSpecification>> preparedInsert = prepareInsert();
 +                AbstractSSTableSimpleWriter writer = sorted
 +                                                     ? new SSTableSimpleWriter(cfs.getDirectories().getDirectoryForNewSSTables(), cfs.metadata, preparedInsert.left.updatedColumns())
 +                                                     : new SSTableSimpleUnsortedWriter(cfs.getDirectories().getDirectoryForNewSSTables(), cfs.metadata, preparedInsert.left.updatedColumns(), bufferSizeInMB);
 +
 +                if (formatType != null)
 +                    writer.setSSTableFormatType(formatType);
 +
 +                writer.setRangeAwareWriting(makeRangeAware);
 +
 +                return new StressCQLSSTableWriter(cfs, writer, preparedInsert.left, preparedInsert.right);
 +            }
 +        }
 +
 +        private static void createTypes(String keyspace, List<CreateTypeStatement> typeStatements)
 +        {
 +            KeyspaceMetadata ksm = Schema.instance.getKSMetaData(keyspace);
 +            Types.RawBuilder builder = Types.rawBuilder(keyspace);
 +            for (CreateTypeStatement st : typeStatements)
 +                st.addToRawBuilder(builder);
 +
 +            ksm = ksm.withSwapped(builder.build());
 +            Schema.instance.setKeyspaceMetadata(ksm);
 +        }
 +
 +        public static ColumnFamilyStore createOfflineTable(String schema, List<File> directoryList)
 +        {
 +            return createOfflineTable(parseStatement(schema, CreateTableStatement.RawStatement.class, "CREATE TABLE"), Collections.EMPTY_LIST, directoryList);
 +        }
 +
 +        /**
 +         * Creates the table according to schema statement
 +         * with specified data directories
 +         */
 +        public static ColumnFamilyStore createOfflineTable(CreateTableStatement.RawStatement schemaStatement, List<CreateTypeStatement> typeStatements, List<File> directoryList)
 +        {
 +            String keyspace = schemaStatement.keyspace();
 +
 +            if (Schema.instance.getKSMetaData(keyspace) == null)
 +                Schema.instance.load(KeyspaceMetadata.create(keyspace, KeyspaceParams.simple(1)));
 +
 +            createTypes(keyspace, typeStatements);
 +
 +            KeyspaceMetadata ksm = Schema.instance.getKSMetaData(keyspace);
 +
 +            CFMetaData cfMetaData = ksm.tables.getNullable(schemaStatement.columnFamily());
 +
 +            if (cfMetaData != null)
 +                return Schema.instance.getColumnFamilyStoreInstance(cfMetaData.cfId);
 +
 +            CreateTableStatement statement = (CreateTableStatement) schemaStatement.prepare(ksm.types).statement;
 +            statement.validate(ClientState.forInternalCalls());
 +
 +            //Build metatdata with a portable cfId
 +            cfMetaData = statement.metadataBuilder()
 +                                  .withId(CFMetaData.generateLegacyCfId(keyspace, statement.columnFamily()))
 +                                  .build()
 +                                  .params(statement.params());
 +
 +            Keyspace.setInitialized();
 +            Directories directories = new Directories(cfMetaData, directoryList.stream().map(Directories.DataDirectory::new).collect(Collectors.toList()));
 +
 +            Keyspace ks = Keyspace.openWithoutSSTables(keyspace);
 +            ColumnFamilyStore cfs =  ColumnFamilyStore.createColumnFamilyStore(ks, cfMetaData.cfName, cfMetaData, directories, false, false, true);
 +
 +            ks.initCfCustom(cfs);
 +            Schema.instance.load(cfs.metadata);
 +            Schema.instance.setKeyspaceMetadata(ksm.withSwapped(ksm.tables.with(cfs.metadata)));
 +
 +            return cfs;
 +        }
 +
 +        /**
 +         * Prepares insert statement for writing data to SSTable
 +         *
 +         * @return prepared Insert statement and it's bound names
 +         */
 +        private Pair<UpdateStatement, List<ColumnSpecification>> prepareInsert()
 +        {
-             ParsedStatement.Prepared cqlStatement = insertStatement.prepare();
++            ParsedStatement.Prepared cqlStatement = insertStatement.prepare(ClientState.forInternalCalls());
 +            UpdateStatement insert = (UpdateStatement) cqlStatement.statement;
 +            insert.validate(ClientState.forInternalCalls());
 +
 +            if (insert.hasConditions())
 +                throw new IllegalArgumentException("Conditional statements are not supported");
 +            if (insert.isCounter())
 +                throw new IllegalArgumentException("Counter update statements are not supported");
 +            if (cqlStatement.boundNames.isEmpty())
 +                throw new IllegalArgumentException("Provided insert statement has no bind variables");
 +
 +            return Pair.create(insert, cqlStatement.boundNames);
 +        }
 +    }
 +
 +    public static <T extends ParsedStatement> T parseStatement(String query, Class<T> klass, String type)
 +    {
 +        try
 +        {
 +            ParsedStatement stmt = CQLFragmentParser.parseAnyUnhandled(CqlParser::query, query);
 +
 +            if (!stmt.getClass().equals(klass))
 +                throw new IllegalArgumentException("Invalid query, must be a " + type + " statement but was: " + stmt.getClass());
 +
 +            return klass.cast(stmt);
 +        }
 +        catch (RecognitionException | RequestValidationException e)
 +        {
 +            throw new IllegalArgumentException(e.getMessage(), e);
 +        }
 +    }
 +}


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


[02/25] cassandra git commit: Allow dropping COMPACT STORAGE flag

Posted by if...@apache.org.
Allow dropping COMPACT STORAGE flag

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/6c29ee84
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/6c29ee84
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/6c29ee84

Branch: refs/heads/cassandra-3.0
Commit: 6c29ee84a2f62ccd05c328bbaa0c364eb1a7a821
Parents: b869744
Author: Alex Petrov <ol...@gmail.com>
Authored: Sat Sep 30 08:56:22 2017 +0200
Committer: Alex Petrov <ol...@gmail.com>
Committed: Mon Nov 6 15:44:51 2017 +0100

----------------------------------------------------------------------
 NEWS.txt                                        |  17 +
 bin/cqlsh.py                                    |   6 +-
 doc/native_protocol_v4.spec                     |   4 +
 ...dra-driver-internal-only-3.11.0-bb96859b.zip | Bin 0 -> 266661 bytes
 ...driver-internal-only-3.7.1.post0-19c1603.zip | Bin 252027 -> 0 bytes
 .../cassandra/auth/CassandraRoleManager.java    |   3 +-
 .../org/apache/cassandra/config/CFMetaData.java |  59 ++-
 src/java/org/apache/cassandra/cql3/Cql.g        |  27 +-
 .../apache/cassandra/cql3/QueryProcessor.java   |   2 +-
 .../cql3/statements/AlterTableStatement.java    |  42 +-
 .../statements/AuthenticationStatement.java     |   2 +-
 .../cql3/statements/AuthorizationStatement.java |   2 +-
 .../cql3/statements/BatchStatement.java         |   4 +-
 .../statements/CreateAggregateStatement.java    |   4 +-
 .../statements/CreateFunctionStatement.java     |   4 +-
 .../cql3/statements/CreateIndexStatement.java   |   9 +-
 .../cql3/statements/CreateTableStatement.java   |   2 +-
 .../cql3/statements/CreateViewStatement.java    |   2 +-
 .../cql3/statements/DropFunctionStatement.java  |   4 +-
 .../cql3/statements/ModificationStatement.java  |  11 +-
 .../cql3/statements/ParsedStatement.java        |   3 +-
 .../statements/SchemaAlteringStatement.java     |  32 +-
 .../cql3/statements/SelectStatement.java        |   8 +-
 .../cql3/statements/TruncateStatement.java      |   2 +-
 .../cassandra/cql3/statements/UseStatement.java |   2 +-
 src/java/org/apache/cassandra/db/view/View.java |  38 +-
 .../index/internal/keys/KeysSearcher.java       |   8 +-
 .../apache/cassandra/repair/RepairRunnable.java |   3 +-
 .../apache/cassandra/service/ClientState.java   |  16 +
 .../cassandra/thrift/ThriftValidation.java      |  10 +-
 .../transport/messages/StartupMessage.java      |   4 +
 .../org/apache/cassandra/cql3/ViewTest.java     |  15 +-
 .../cql3/validation/entities/UFTest.java        |  25 +
 .../cql3/validation/entities/UserTypesTest.java |  20 +
 .../cql3/validation/operations/AlterTest.java   |  15 +-
 .../DropCompactStorageThriftTest.java           | 525 +++++++++++++++++++
 36 files changed, 843 insertions(+), 87 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/NEWS.txt
----------------------------------------------------------------------
diff --git a/NEWS.txt b/NEWS.txt
index 60cf77c..621866b 100644
--- a/NEWS.txt
+++ b/NEWS.txt
@@ -37,6 +37,23 @@ Upgrading
    - Nothing specific to this release, but please see previous upgrading sections,
      especially if you are upgrading from 2.2.
 
+Compact Storage
+---------------
+    - Starting version 4.0, Thrift and COMPACT STORAGE is no longer supported.
+      'ALTER ... DROP COMPACT STORAGE' statement makes Compact Tables CQL-compatible,
+      exposing internal structure of Thrift/Compact Tables. You can find more details
+      on exposed internal structure under: 
+      http://cassandra.apache.org/doc/latest/cql/appendices.html#appendix-c-dropping-compact-storage
+
+      For uninterrupted cluster upgrades, drivers now support 'NO_COMPACT' startup option.
+      Supplying this flag will have same effect as 'DROP COMPACT STORAGE', but only for the
+      current connection.
+
+      In order to upgrade, clients supporting a non-compact schema view can be rolled out
+      gradually. When all the clients are updated 'ALTER ... DROP COMPACT STORAGE' can be
+      executed. After dropping compact storage, ’NO_COMPACT' option will have no effect
+      after that.
+
 Materialized Views
 -------------------
     - Cassandra will no longer allow dropping columns on tables with Materialized Views.

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/bin/cqlsh.py
----------------------------------------------------------------------
diff --git a/bin/cqlsh.py b/bin/cqlsh.py
index 8eb42a3..8d05d9d 100644
--- a/bin/cqlsh.py
+++ b/bin/cqlsh.py
@@ -205,6 +205,7 @@ parser.add_option("--browser", dest='browser', help="""The browser to use to dis
                                                     - one of the supported browsers in https://docs.python.org/2/library/webbrowser.html.
                                                     - browser path followed by %s, example: /usr/bin/google-chrome-stable %s""")
 parser.add_option('--ssl', action='store_true', help='Use SSL', default=False)
+parser.add_option('--no_compact', action='store_true', help='No Compact', default=False)
 parser.add_option("-u", "--username", help="Authenticate as user.")
 parser.add_option("-p", "--password", help="Authenticate using password.")
 parser.add_option('-k', '--keyspace', help='Authenticate to the given keyspace.')
@@ -702,6 +703,7 @@ class Shell(cmd.Cmd):
                  completekey=DEFAULT_COMPLETEKEY, browser=None, use_conn=None,
                  cqlver=DEFAULT_CQLVER, keyspace=None,
                  tracing_enabled=False, expand_enabled=False,
+                 no_compact=False,
                  display_nanotime_format=DEFAULT_NANOTIME_FORMAT,
                  display_timestamp_format=DEFAULT_TIMESTAMP_FORMAT,
                  display_date_format=DEFAULT_DATE_FORMAT,
@@ -732,7 +734,7 @@ class Shell(cmd.Cmd):
         else:
             self.conn = Cluster(contact_points=(self.hostname,), port=self.port, cql_version=cqlver,
                                 protocol_version=protocol_version,
-                                auth_provider=self.auth_provider,
+                                auth_provider=self.auth_provider, no_compact=no_compact,
                                 ssl_options=sslhandling.ssl_settings(hostname, CONFIG_FILE) if ssl else None,
                                 load_balancing_policy=WhiteListRoundRobinPolicy([self.hostname]),
                                 control_connection_timeout=connect_timeout,
@@ -2486,6 +2488,7 @@ def read_options(cmdlineargs, environment):
     optvalues.debug = False
     optvalues.file = None
     optvalues.ssl = False
+    optvalues.no_compact = False
     optvalues.encoding = option_with_default(configs.get, 'ui', 'encoding', UTF8)
 
     optvalues.tty = option_with_default(configs.getboolean, 'ui', 'tty', sys.stdin.isatty())
@@ -2643,6 +2646,7 @@ def main(options, hostname, port):
                       browser=options.browser,
                       cqlver=options.cqlversion,
                       keyspace=options.keyspace,
+                      no_compact=options.no_compact,
                       display_timestamp_format=options.time_format,
                       display_nanotime_format=options.nanotime_format,
                       display_date_format=options.date_format,

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/doc/native_protocol_v4.spec
----------------------------------------------------------------------
diff --git a/doc/native_protocol_v4.spec b/doc/native_protocol_v4.spec
index 44dac18..2188a33 100644
--- a/doc/native_protocol_v4.spec
+++ b/doc/native_protocol_v4.spec
@@ -271,6 +271,10 @@ Table of Contents
       different from the protocol version.
     - "COMPRESSION": the compression algorithm to use for frames (See section 5).
       This is optional; if not specified no compression will be used.
+    - "NO_COMPACT": whether or not connection has to be established in compatibility
+      mode. This mode will make all Thrift and Compact Tables to be exposed as if
+      they were CQL Tables. This is optional; if not specified, the option will
+      not be used.
 
 
 4.1.2. AUTH_RESPONSE

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/lib/cassandra-driver-internal-only-3.11.0-bb96859b.zip
----------------------------------------------------------------------
diff --git a/lib/cassandra-driver-internal-only-3.11.0-bb96859b.zip b/lib/cassandra-driver-internal-only-3.11.0-bb96859b.zip
new file mode 100644
index 0000000..d31abc3
Binary files /dev/null and b/lib/cassandra-driver-internal-only-3.11.0-bb96859b.zip differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/lib/cassandra-driver-internal-only-3.7.1.post0-19c1603.zip
----------------------------------------------------------------------
diff --git a/lib/cassandra-driver-internal-only-3.7.1.post0-19c1603.zip b/lib/cassandra-driver-internal-only-3.7.1.post0-19c1603.zip
deleted file mode 100644
index 900d64d..0000000
Binary files a/lib/cassandra-driver-internal-only-3.7.1.post0-19c1603.zip and /dev/null differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/src/java/org/apache/cassandra/auth/CassandraRoleManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/auth/CassandraRoleManager.java b/src/java/org/apache/cassandra/auth/CassandraRoleManager.java
index e5b98e4..c6716e5 100644
--- a/src/java/org/apache/cassandra/auth/CassandraRoleManager.java
+++ b/src/java/org/apache/cassandra/auth/CassandraRoleManager.java
@@ -39,6 +39,7 @@ import org.apache.cassandra.db.ConsistencyLevel;
 import org.apache.cassandra.db.marshal.UTF8Type;
 import org.apache.cassandra.exceptions.*;
 import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.service.ClientState;
 import org.apache.cassandra.service.QueryState;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.transport.messages.ResultMessage;
@@ -465,7 +466,7 @@ public class CassandraRoleManager implements IRoleManager
     {
         try
         {
-            return QueryProcessor.parseStatement(String.format(template, keyspace, table)).prepare().statement;
+            return QueryProcessor.parseStatement(String.format(template, keyspace, table)).prepare(ClientState.forInternalCalls()).statement;
         }
         catch (RequestValidationException e)
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/src/java/org/apache/cassandra/config/CFMetaData.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/CFMetaData.java b/src/java/org/apache/cassandra/config/CFMetaData.java
index fd1c9e5..c1b2171 100644
--- a/src/java/org/apache/cassandra/config/CFMetaData.java
+++ b/src/java/org/apache/cassandra/config/CFMetaData.java
@@ -79,10 +79,6 @@ public final class CFMetaData
     public final Pair<String, String> ksAndCFName;
     public final byte[] ksAndCFBytes;
 
-    private final ImmutableSet<Flag> flags;
-    private final boolean isDense;
-    private final boolean isCompound;
-    private final boolean isSuper;
     private final boolean isCounter;
     private final boolean isView;
     private final boolean isIndex;
@@ -94,6 +90,11 @@ public final class CFMetaData
     private final Serializers serializers;
 
     // non-final, for now
+    private volatile ImmutableSet<Flag> flags;
+    private volatile boolean isDense;
+    private volatile boolean isCompound;
+    private volatile boolean isSuper;
+
     public volatile TableParams params = TableParams.DEFAULT;
 
     private volatile Map<ByteBuffer, DroppedColumn> droppedColumns = new HashMap<>();
@@ -127,6 +128,9 @@ public final class CFMetaData
     private volatile ColumnDefinition superCfKeyColumn;
     private volatile ColumnDefinition superCfValueColumn;
 
+    /** Caches a non-compact version of the metadata for compact tables to be used with the NO_COMPACT protocol option. */
+    private volatile CFMetaData nonCompactCopy = null;
+
     public boolean isSuperColumnKeyColumn(ColumnDefinition cd)
     {
         return cd.name.equals(superCfKeyColumn.name);
@@ -330,6 +334,9 @@ public final class CFMetaData
     // are kept because they are often useful in a different format.
     private void rebuild()
     {
+        // A non-compact copy will be created lazily
+        this.nonCompactCopy = null;
+
         if (isCompactTable())
         {
             this.compactValueColumn = isSuper() ?
@@ -505,6 +512,38 @@ public final class CFMetaData
         return params(indexParams.build());
     }
 
+    /**
+     * Returns a cached non-compact version of this table. Cached version has to be invalidated
+     * every time the table is rebuilt.
+     */
+    public CFMetaData asNonCompact()
+    {
+        assert isCompactTable() : "Can't get non-compact version of a CQL table";
+
+        // Note that this is racy, but re-computing the non-compact copy a few times on first uses isn't a big deal so
+        // we don't bother.
+        if (nonCompactCopy == null)
+        {
+            nonCompactCopy = copyOpts(new CFMetaData(ksName,
+                                                     cfName,
+                                                     cfId,
+                                                     false,
+                                                     isCounter,
+                                                     false,
+                                                     true,
+                                                     isView,
+                                                     copy(partitionKeyColumns),
+                                                     copy(clusteringColumns),
+                                                     copy(partitionColumns),
+                                                     partitioner,
+                                                     superCfKeyColumn,
+                                                     superCfValueColumn),
+                                      this);
+        }
+
+        return nonCompactCopy;
+    }
+
     public CFMetaData copy()
     {
         return copy(cfId);
@@ -842,6 +881,12 @@ public final class CFMetaData
         superCfKeyColumn = cfm.superCfKeyColumn;
         superCfValueColumn = cfm.superCfValueColumn;
 
+        isDense = cfm.isDense;
+        isCompound = cfm.isCompound;
+        isSuper = cfm.isSuper;
+
+        flags = cfm.flags;
+
         rebuild();
 
         // compaction thresholds are checked by ThriftValidation. We shouldn't be doing
@@ -874,12 +919,6 @@ public final class CFMetaData
         if (!cfm.cfId.equals(cfId))
             throw new ConfigurationException(String.format("Column family ID mismatch (found %s; expected %s)",
                                                            cfm.cfId, cfId));
-
-        // Dense flag can get set, see CASSANDRA-12373 for details. We have to remove flag from both parts because
-        // there's no guaranteed call order in the call.
-
-        if (!cfm.flags.equals(flags) && (!isSuper() || !Sets.difference(cfm.flags, Sets.immutableEnumSet(Flag.DENSE)).equals(Sets.difference(flags, Sets.immutableEnumSet(Flag.DENSE)))))
-            throw new ConfigurationException("Types do not match: " + cfm.flags + " != " + flags);
     }
 
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/src/java/org/apache/cassandra/cql3/Cql.g
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/Cql.g b/src/java/org/apache/cassandra/cql3/Cql.g
index 3123877..0234327 100644
--- a/src/java/org/apache/cassandra/cql3/Cql.g
+++ b/src/java/org/apache/cassandra/cql3/Cql.g
@@ -781,7 +781,7 @@ createTriggerStatement returns [CreateTriggerStatement expr]
     @init {
         boolean ifNotExists = false;
     }
-    : K_CREATE K_TRIGGER (K_IF K_NOT K_EXISTS { ifNotExists = true; } )? (name=cident)
+    : K_CREATE K_TRIGGER (K_IF K_NOT K_EXISTS { ifNotExists = true; } )? (name=noncol_ident)
         K_ON cf=columnFamilyName K_USING cls=STRING_LITERAL
       { $expr = new CreateTriggerStatement(cf, name.toString(), $cls.text, ifNotExists); }
     ;
@@ -791,7 +791,7 @@ createTriggerStatement returns [CreateTriggerStatement expr]
  */
 dropTriggerStatement returns [DropTriggerStatement expr]
      @init { boolean ifExists = false; }
-    : K_DROP K_TRIGGER (K_IF K_EXISTS { ifExists = true; } )? (name=cident) K_ON cf=columnFamilyName
+    : K_DROP K_TRIGGER (K_IF K_EXISTS { ifExists = true; } )? (name=noncol_ident) K_ON cf=columnFamilyName
       { $expr = new DropTriggerStatement(cf, name.toString(), ifExists); }
     ;
 
@@ -816,20 +816,21 @@ alterTableStatement returns [AlterTableStatement expr]
     @init {
         AlterTableStatement.Type type = null;
         TableAttributes attrs = new TableAttributes();
-        Map<ColumnIdentifier.Raw, ColumnIdentifier.Raw> renames = new HashMap<ColumnIdentifier.Raw, ColumnIdentifier.Raw>();
+        Map<ColumnIdentifier.Raw, ColumnIdentifier> renames = new HashMap<ColumnIdentifier.Raw, ColumnIdentifier>();
         boolean isStatic = false;
         Long dropTimestamp = null;
     }
     : K_ALTER K_COLUMNFAMILY cf=columnFamilyName
-          ( K_ALTER id=cident K_TYPE v=comparatorType { type = AlterTableStatement.Type.ALTER; }
-          | K_ADD   id=cident v=comparatorType ({ isStatic=true; } K_STATIC)? { type = AlterTableStatement.Type.ADD; }
+          ( K_ALTER id=cident K_TYPE v=comparatorType { type = AlterTableStatement.Type.ALTER;  }
+          | K_ADD   aid=ident {id=new ColumnIdentifier.ColumnIdentifierValue(aid);} v=comparatorType ({ isStatic=true; } K_STATIC)? { type = AlterTableStatement.Type.ADD; }
           | K_DROP  id=cident                               { type = AlterTableStatement.Type.DROP; }
           | K_DROP  id=cident K_USING K_TIMESTAMP t=INTEGER { type = AlterTableStatement.Type.DROP;
                                                               dropTimestamp = Long.parseLong(Constants.Literal.integer($t.text).getText()); }
+          | K_DROP  K_COMPACT K_STORAGE                     { type = AlterTableStatement.Type.DROP_COMPACT_STORAGE; }
           | K_WITH  properties[attrs]                       { type = AlterTableStatement.Type.OPTS; }
           | K_RENAME                                        { type = AlterTableStatement.Type.RENAME; }
-               id1=cident K_TO toId1=cident { renames.put(id1, toId1); }
-               ( K_AND idn=cident K_TO toIdn=cident { renames.put(idn, toIdn); } )*
+               id1=cident K_TO toId1=ident { renames.put(id1, toId1); }
+               ( K_AND idn=cident K_TO toIdn=ident { renames.put(idn, toIdn); } )*
           )
     {
         $expr = new AlterTableStatement(cf, type, id, v, attrs, renames, isStatic, dropTimestamp);
@@ -1169,10 +1170,14 @@ userPassword[RoleOptions opts]
 // Column Identifiers.  These need to be treated differently from other
 // identifiers because the underlying comparator is not necessarily text. See
 // CASSANDRA-8178 for details.
+// Also, we need to support the internal of the super column map (for backward
+// compatibility) which is empty (we only want to allow this is queries, not for
+// creating table or other).
 cident returns [ColumnIdentifier.Raw id]
     : t=IDENT              { $id = new ColumnIdentifier.Literal($t.text, false); }
     | t=QUOTED_NAME        { $id = new ColumnIdentifier.Literal($t.text, true); }
     | k=unreserved_keyword { $id = new ColumnIdentifier.Literal(k, false); }
+    | EMPTY_QUOTED_NAME    { $id = new ColumnIdentifier.Literal("", false); }
     ;
 
 // Column identifiers where the comparator is known to be text
@@ -1309,7 +1314,9 @@ intValue returns [Term.Raw value]
     ;
 
 functionName returns [FunctionName s]
-    : (ks=keyspaceName '.')? f=allowedFunctionName   { $s = new FunctionName(ks, f); }
+     // antlr might try to recover and give a null for f. It will still error out in the end, but FunctionName
+     // wouldn't be happy with that so we should bypass this for now or we'll have a weird user-facing error
+    : (ks=keyspaceName '.')? f=allowedFunctionName   { $s = f == null ? null : new FunctionName(ks, f); }
     ;
 
 allowedFunctionName returns [String s]
@@ -1822,6 +1829,10 @@ STRING_LITERAL
       )
     ;
 
+EMPTY_QUOTED_NAME
+    : '\"' '\"'
+    ;
+
 QUOTED_NAME
     @init{ StringBuilder b = new StringBuilder(); }
     @after{ setText(b.toString()); }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/src/java/org/apache/cassandra/cql3/QueryProcessor.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/QueryProcessor.java b/src/java/org/apache/cassandra/cql3/QueryProcessor.java
index af94d3e..ddee6c7 100644
--- a/src/java/org/apache/cassandra/cql3/QueryProcessor.java
+++ b/src/java/org/apache/cassandra/cql3/QueryProcessor.java
@@ -517,7 +517,7 @@ public class QueryProcessor implements QueryHandler
             ((CFStatement)statement).prepareKeyspace(clientState);
 
         Tracing.trace("Preparing statement");
-        return statement.prepare();
+        return statement.prepare(clientState);
     }
 
     public static ParsedStatement parseStatement(String queryStr) throws SyntaxException

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java b/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java
index befdd25..a5fa12d 100644
--- a/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java
@@ -31,8 +31,6 @@ import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.db.marshal.CollectionType;
-import org.apache.cassandra.db.marshal.CounterColumnType;
-import org.apache.cassandra.db.marshal.ReversedType;
 import org.apache.cassandra.db.view.View;
 import org.apache.cassandra.exceptions.*;
 import org.apache.cassandra.schema.IndexMetadata;
@@ -49,14 +47,14 @@ public class AlterTableStatement extends SchemaAlteringStatement
 {
     public enum Type
     {
-        ADD, ALTER, DROP, OPTS, RENAME
+        ADD, ALTER, DROP, DROP_COMPACT_STORAGE, OPTS, RENAME
     }
 
     public final Type oType;
     public final CQL3Type.Raw validator;
     public final ColumnIdentifier.Raw rawColumnName;
     private final TableAttributes attrs;
-    private final Map<ColumnIdentifier.Raw, ColumnIdentifier.Raw> renames;
+    private final Map<ColumnIdentifier.Raw, ColumnIdentifier> renames;
     private final boolean isStatic; // Only for ALTER ADD
     private final Long deleteTimestamp;
 
@@ -65,7 +63,7 @@ public class AlterTableStatement extends SchemaAlteringStatement
                                ColumnIdentifier.Raw columnName,
                                CQL3Type.Raw validator,
                                TableAttributes attrs,
-                               Map<ColumnIdentifier.Raw, ColumnIdentifier.Raw> renames,
+                               Map<ColumnIdentifier.Raw, ColumnIdentifier> renames,
                                boolean isStatic,
                                Long deleteTimestamp)
     {
@@ -95,15 +93,15 @@ public class AlterTableStatement extends SchemaAlteringStatement
         if (meta.isView())
             throw new InvalidRequestException("Cannot use ALTER TABLE on Materialized View");
 
-        CFMetaData cfm = meta.copy();
+        CFMetaData cfm;
 
         CQL3Type validator = this.validator == null ? null : this.validator.prepare(keyspace());
         ColumnIdentifier columnName = null;
         ColumnDefinition def = null;
         if (rawColumnName != null)
         {
-            columnName = rawColumnName.prepare(cfm);
-            def = cfm.getColumnDefinition(columnName);
+            columnName = rawColumnName.prepare(meta);
+            def = meta.getColumnDefinition(columnName);
         }
 
         List<ViewDefinition> viewUpdates = null;
@@ -115,9 +113,11 @@ public class AlterTableStatement extends SchemaAlteringStatement
                 throw new InvalidRequestException("Altering of types is not allowed");
             case ADD:
                 assert columnName != null;
-                if (cfm.isDense())
+                if (meta.isDense())
                     throw new InvalidRequestException("Cannot add new column to a COMPACT STORAGE table");
 
+                cfm = meta.copy();
+
                 if (isStatic)
                 {
                     if (!cfm.isCompound())
@@ -190,11 +190,14 @@ public class AlterTableStatement extends SchemaAlteringStatement
 
             case DROP:
                 assert columnName != null;
-                if (!cfm.isCQLTable())
+                if (!meta.isCQLTable())
                     throw new InvalidRequestException("Cannot drop columns from a non-CQL3 table");
+
                 if (def == null)
                     throw new InvalidRequestException(String.format("Column %s was not found in table %s", columnName, columnFamily()));
 
+                cfm = meta.copy();
+
                 switch (def.kind)
                 {
                     case PARTITION_KEY:
@@ -238,11 +241,19 @@ public class AlterTableStatement extends SchemaAlteringStatement
                                                                     columnName.toString(),
                                                                     keyspace()));
                 break;
+            case DROP_COMPACT_STORAGE:
+                if (!meta.isCompactTable())
+                    throw new InvalidRequestException("Cannot DROP COMPACT STORAGE on table without COMPACT STORAGE");
+
+                cfm = meta.asNonCompact();
+                break;
             case OPTS:
                 if (attrs == null)
                     throw new InvalidRequestException("ALTER TABLE WITH invoked, but no parameters found");
                 attrs.validate();
 
+                cfm = meta.copy();
+
                 TableParams params = attrs.asAlteredTableParams(cfm.params);
 
                 if (!Iterables.isEmpty(views) && params.gcGraceSeconds == 0)
@@ -261,10 +272,13 @@ public class AlterTableStatement extends SchemaAlteringStatement
 
                 break;
             case RENAME:
-                for (Map.Entry<ColumnIdentifier.Raw, ColumnIdentifier.Raw> entry : renames.entrySet())
+                cfm = meta.copy();
+
+                for (Map.Entry<ColumnIdentifier.Raw, ColumnIdentifier> entry : renames.entrySet())
                 {
                     ColumnIdentifier from = entry.getKey().prepare(cfm);
-                    ColumnIdentifier to = entry.getValue().prepare(cfm);
+                    ColumnIdentifier to = entry.getValue();
+
                     cfm.renameColumn(from, to);
 
                     // If the view includes a renamed column, it must be renamed in the view table and the definition.
@@ -274,7 +288,7 @@ public class AlterTableStatement extends SchemaAlteringStatement
 
                         ViewDefinition viewCopy = view.copy();
                         ColumnIdentifier viewFrom = entry.getKey().prepare(viewCopy.metadata);
-                        ColumnIdentifier viewTo = entry.getValue().prepare(viewCopy.metadata);
+                        ColumnIdentifier viewTo = entry.getValue();
                         viewCopy.renameColumn(viewFrom, viewTo);
 
                         if (viewUpdates == null)
@@ -283,6 +297,8 @@ public class AlterTableStatement extends SchemaAlteringStatement
                     }
                 }
                 break;
+            default:
+                throw new InvalidRequestException("Can not alter table: unknown option type " + oType);
         }
 
         MigrationManager.announceColumnFamilyUpdate(cfm, viewUpdates, isLocalOnly);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/src/java/org/apache/cassandra/cql3/statements/AuthenticationStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/AuthenticationStatement.java b/src/java/org/apache/cassandra/cql3/statements/AuthenticationStatement.java
index 151e4f0..30ab6b0 100644
--- a/src/java/org/apache/cassandra/cql3/statements/AuthenticationStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/AuthenticationStatement.java
@@ -31,7 +31,7 @@ import org.apache.cassandra.transport.messages.ResultMessage;
 public abstract class AuthenticationStatement extends ParsedStatement implements CQLStatement
 {
     @Override
-    public Prepared prepare()
+    public Prepared prepare(ClientState clientState)
     {
         return new Prepared(this);
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/src/java/org/apache/cassandra/cql3/statements/AuthorizationStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/AuthorizationStatement.java b/src/java/org/apache/cassandra/cql3/statements/AuthorizationStatement.java
index 098e22c..fa2a993 100644
--- a/src/java/org/apache/cassandra/cql3/statements/AuthorizationStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/AuthorizationStatement.java
@@ -32,7 +32,7 @@ import org.apache.cassandra.transport.messages.ResultMessage;
 public abstract class AuthorizationStatement extends ParsedStatement implements CQLStatement
 {
     @Override
-    public Prepared prepare()
+    public Prepared prepare(ClientState clientState)
     {
         return new Prepared(this);
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java b/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
index cd9358c..1c3cfa6 100644
--- a/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
@@ -516,7 +516,7 @@ public class BatchStatement implements CQLStatement
                 statement.prepareKeyspace(state);
         }
 
-        public ParsedStatement.Prepared prepare() throws InvalidRequestException
+        public ParsedStatement.Prepared prepare(ClientState clientState) throws InvalidRequestException
         {
             VariableSpecifications boundNames = getBoundVariables();
 
@@ -537,7 +537,7 @@ public class BatchStatement implements CQLStatement
                     haveMultipleCFs = !firstKS.equals(parsed.keyspace()) || !firstCF.equals(parsed.columnFamily());
                 }
 
-                statements.add(parsed.prepare(boundNames));
+                statements.add(parsed.prepare(boundNames, clientState));
             }
 
             Attributes prepAttrs = attrs.prepare("[batch]", "[batch]");

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/src/java/org/apache/cassandra/cql3/statements/CreateAggregateStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/CreateAggregateStatement.java b/src/java/org/apache/cassandra/cql3/statements/CreateAggregateStatement.java
index ca0270f..9d91693 100644
--- a/src/java/org/apache/cassandra/cql3/statements/CreateAggregateStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/CreateAggregateStatement.java
@@ -78,7 +78,7 @@ public final class CreateAggregateStatement extends SchemaAlteringStatement
         this.ifNotExists = ifNotExists;
     }
 
-    public Prepared prepare()
+    public Prepared prepare(ClientState clientState)
     {
         argTypes = new ArrayList<>(argRawTypes.size());
         for (CQL3Type.Raw rawType : argRawTypes)
@@ -136,7 +136,7 @@ public final class CreateAggregateStatement extends SchemaAlteringStatement
                 throw new InvalidRequestException("INITCOND must not be empty for all types except TEXT, ASCII, BLOB");
         }
 
-        return super.prepare();
+        return super.prepare(clientState);
     }
 
     private AbstractType<?> prepareType(String typeName, CQL3Type.Raw rawType)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/src/java/org/apache/cassandra/cql3/statements/CreateFunctionStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/CreateFunctionStatement.java b/src/java/org/apache/cassandra/cql3/statements/CreateFunctionStatement.java
index c8d38f5..dfe522b 100644
--- a/src/java/org/apache/cassandra/cql3/statements/CreateFunctionStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/CreateFunctionStatement.java
@@ -76,7 +76,7 @@ public final class CreateFunctionStatement extends SchemaAlteringStatement
         this.ifNotExists = ifNotExists;
     }
 
-    public Prepared prepare() throws InvalidRequestException
+    public Prepared prepare(ClientState clientState) throws InvalidRequestException
     {
         if (new HashSet<>(argNames).size() != argNames.size())
             throw new InvalidRequestException(String.format("duplicate argument names for given function %s with argument names %s",
@@ -87,7 +87,7 @@ public final class CreateFunctionStatement extends SchemaAlteringStatement
             argTypes.add(prepareType("arguments", rawType));
 
         returnType = prepareType("return type", rawReturnType);
-        return super.prepare();
+        return super.prepare(clientState);
     }
 
     public void prepareKeyspace(ClientState state) throws InvalidRequestException

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/src/java/org/apache/cassandra/cql3/statements/CreateIndexStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/CreateIndexStatement.java b/src/java/org/apache/cassandra/cql3/statements/CreateIndexStatement.java
index c21441c..47d54fe 100644
--- a/src/java/org/apache/cassandra/cql3/statements/CreateIndexStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/CreateIndexStatement.java
@@ -103,8 +103,13 @@ public class CreateIndexStatement extends SchemaAlteringStatement
                 throw new InvalidRequestException("No column definition found for column " + target.column);
 
             // TODO: we could lift that limitation
-            if (cfm.isCompactTable() && cd.isPrimaryKeyColumn())
-                throw new InvalidRequestException("Secondary indexes are not supported on PRIMARY KEY columns in COMPACT STORAGE tables");
+            if (cfm.isCompactTable())
+            {
+                if (cd.isPrimaryKeyColumn())
+                    throw new InvalidRequestException("Secondary indexes are not supported on PRIMARY KEY columns in COMPACT STORAGE tables");
+                if (cfm.compactValueColumn().equals(cd))
+                    throw new InvalidRequestException("Secondary indexes are not supported on compact value column of COMPACT STORAGE tables");
+            }
 
             // It would be possible to support 2ndary index on static columns (but not without modifications of at least ExtendedFilter and
             // CompositesIndex) and maybe we should, but that means a query like:

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/src/java/org/apache/cassandra/cql3/statements/CreateTableStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/CreateTableStatement.java b/src/java/org/apache/cassandra/cql3/statements/CreateTableStatement.java
index ef950dc..9f14194 100644
--- a/src/java/org/apache/cassandra/cql3/statements/CreateTableStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/CreateTableStatement.java
@@ -191,7 +191,7 @@ public class CreateTableStatement extends SchemaAlteringStatement
         /**
          * Transform this raw statement into a CreateTableStatement.
          */
-        public ParsedStatement.Prepared prepare() throws RequestValidationException
+        public ParsedStatement.Prepared prepare(ClientState clientState) throws RequestValidationException
         {
             KeyspaceMetadata ksm = Schema.instance.getKSMetaData(keyspace());
             if (ksm == null)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/src/java/org/apache/cassandra/cql3/statements/CreateViewStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/CreateViewStatement.java b/src/java/org/apache/cassandra/cql3/statements/CreateViewStatement.java
index 778a3f4..cce954f 100644
--- a/src/java/org/apache/cassandra/cql3/statements/CreateViewStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/CreateViewStatement.java
@@ -214,7 +214,7 @@ public class CreateViewStatement extends SchemaAlteringStatement
         rawSelect.prepareKeyspace(state);
         rawSelect.setBoundVariables(getBoundVariables());
 
-        ParsedStatement.Prepared prepared = rawSelect.prepare(true);
+        ParsedStatement.Prepared prepared = rawSelect.prepare(true, queryState.getClientState());
         SelectStatement select = (SelectStatement) prepared.statement;
         StatementRestrictions restrictions = select.getRestrictions();
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/src/java/org/apache/cassandra/cql3/statements/DropFunctionStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/DropFunctionStatement.java b/src/java/org/apache/cassandra/cql3/statements/DropFunctionStatement.java
index 138691e..8845a82 100644
--- a/src/java/org/apache/cassandra/cql3/statements/DropFunctionStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/DropFunctionStatement.java
@@ -63,7 +63,7 @@ public final class DropFunctionStatement extends SchemaAlteringStatement
     }
 
     @Override
-    public Prepared prepare() throws InvalidRequestException
+    public Prepared prepare(ClientState clientState) throws InvalidRequestException
     {
         if (Schema.instance.getKSMetaData(functionName.keyspace) != null)
         {
@@ -82,7 +82,7 @@ public final class DropFunctionStatement extends SchemaAlteringStatement
             }
         }
 
-        return super.prepare();
+        return super.prepare(clientState);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java b/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
index 28fc90f..8ae4d64 100644
--- a/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
@@ -793,17 +793,16 @@ public abstract class ModificationStatement implements CQLStatement
             this.ifExists = ifExists;
         }
 
-        public ParsedStatement.Prepared prepare()
+        public ParsedStatement.Prepared prepare(ClientState clientState)
         {
             VariableSpecifications boundNames = getBoundVariables();
-            ModificationStatement statement = prepare(boundNames);
-            CFMetaData cfm = ThriftValidation.validateColumnFamily(keyspace(), columnFamily());
-            return new ParsedStatement.Prepared(statement, boundNames, boundNames.getPartitionKeyBindIndexes(cfm));
+            ModificationStatement statement = prepare(boundNames, clientState);
+            return new ParsedStatement.Prepared(statement, boundNames, boundNames.getPartitionKeyBindIndexes(statement.cfm));
         }
 
-        public ModificationStatement prepare(VariableSpecifications boundNames)
+        public ModificationStatement prepare(VariableSpecifications boundNames, ClientState clientState)
         {
-            CFMetaData metadata = ThriftValidation.validateColumnFamily(keyspace(), columnFamily());
+            CFMetaData metadata = ThriftValidation.validateColumnFamilyWithCompactMode(keyspace(), columnFamily(), clientState.isNoCompactMode());
 
             Attributes preparedAttributes = attrs.prepare(keyspace(), columnFamily());
             preparedAttributes.collectMarkerSpecification(boundNames);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/src/java/org/apache/cassandra/cql3/statements/ParsedStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/ParsedStatement.java b/src/java/org/apache/cassandra/cql3/statements/ParsedStatement.java
index 4c3f8a9..01a1b5e 100644
--- a/src/java/org/apache/cassandra/cql3/statements/ParsedStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/ParsedStatement.java
@@ -23,6 +23,7 @@ import java.util.List;
 import org.apache.cassandra.cql3.*;
 import org.apache.cassandra.cql3.functions.Function;
 import org.apache.cassandra.exceptions.RequestValidationException;
+import org.apache.cassandra.service.ClientState;
 
 public abstract class ParsedStatement
 {
@@ -44,7 +45,7 @@ public abstract class ParsedStatement
         this.variables = variables;
     }
 
-    public abstract Prepared prepare() throws RequestValidationException;
+    public abstract Prepared prepare(ClientState clientState) throws RequestValidationException;
 
     public static class Prepared
     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/src/java/org/apache/cassandra/cql3/statements/SchemaAlteringStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/SchemaAlteringStatement.java b/src/java/org/apache/cassandra/cql3/statements/SchemaAlteringStatement.java
index 62ba0ae..e7ecb14 100644
--- a/src/java/org/apache/cassandra/cql3/statements/SchemaAlteringStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/SchemaAlteringStatement.java
@@ -18,6 +18,8 @@
 package org.apache.cassandra.cql3.statements;
 
 import org.apache.cassandra.auth.AuthenticatedUser;
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.cql3.CFName;
 import org.apache.cassandra.cql3.CQLStatement;
 import org.apache.cassandra.cql3.QueryOptions;
@@ -25,9 +27,12 @@ import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.exceptions.RequestValidationException;
 import org.apache.cassandra.service.ClientState;
 import org.apache.cassandra.service.QueryState;
+import org.apache.cassandra.thrift.ThriftValidation;
 import org.apache.cassandra.transport.Event;
 import org.apache.cassandra.transport.messages.ResultMessage;
 
+import static org.apache.cassandra.cql3.statements.RequestValidations.invalidRequest;
+
 /**
  * Abstract class for statements that alter the schema.
  */
@@ -60,8 +65,33 @@ public abstract class SchemaAlteringStatement extends CFStatement implements CQL
     }
 
     @Override
-    public Prepared prepare()
+    public Prepared prepare(ClientState clientState)
     {
+        // We don't allow schema changes in no-compact mode on compact tables because it feels like unnecessary
+        // complication: applying the change on the non compact version of the table might be unsafe (the table is
+        // still compact in general), and applying it to the compact version in a no-compact connection feels
+        // confusing/unintuitive. If user want to alter the compact version, they can simply do so in a normal
+        // connection; if they want to alter the non-compact version, they should finish their transition and properly
+        // DROP COMPACT STORAGE on the table before doing so.
+        if (isColumnFamilyLevel && clientState.isNoCompactMode())
+        {
+            CFMetaData table = ThriftValidation.validateColumnFamily(keyspace(), columnFamily());
+            if (table.isCompactTable())
+            {
+                throw invalidRequest("Cannot alter schema of compact table %s.%s from a connection in NO-COMPACT mode",
+                                     table.ksName, table.cfName);
+            }
+            else if (table.isView())
+            {
+                CFMetaData baseTable = Schema.instance.getView(table.ksName, table.cfName).baseTableMetadata();
+                if (baseTable.isCompactTable())
+                    throw new InvalidRequestException(String.format("Cannot ALTER schema of view %s.%s on compact table %s from "
+                                                                    + "a connection in NO-COMPACT mode",
+                                                                    table.ksName, table.cfName,
+                                                                    baseTable.ksName, baseTable.cfName));
+            }
+        }
+
         return new Prepared(this);
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
index 2e090fa..1e867bc 100644
--- a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
@@ -887,14 +887,14 @@ public class SelectStatement implements CQLStatement
             this.limit = limit;
         }
 
-        public ParsedStatement.Prepared prepare() throws InvalidRequestException
+        public ParsedStatement.Prepared prepare(ClientState clientState) throws InvalidRequestException
         {
-            return prepare(false);
+            return prepare(false, clientState);
         }
 
-        public ParsedStatement.Prepared prepare(boolean forView) throws InvalidRequestException
+        public ParsedStatement.Prepared prepare(boolean forView, ClientState clientState) throws InvalidRequestException
         {
-            CFMetaData cfm = ThriftValidation.validateColumnFamily(keyspace(), columnFamily());
+            CFMetaData cfm = ThriftValidation.validateColumnFamilyWithCompactMode(keyspace(), columnFamily(), clientState.isNoCompactMode());
             VariableSpecifications boundNames = getBoundVariables();
 
             Selection selection = selectClause.isEmpty()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/src/java/org/apache/cassandra/cql3/statements/TruncateStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/TruncateStatement.java b/src/java/org/apache/cassandra/cql3/statements/TruncateStatement.java
index 66b3da0..b697910 100644
--- a/src/java/org/apache/cassandra/cql3/statements/TruncateStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/TruncateStatement.java
@@ -45,7 +45,7 @@ public class TruncateStatement extends CFStatement implements CQLStatement
         return 0;
     }
 
-    public Prepared prepare() throws InvalidRequestException
+    public Prepared prepare(ClientState clientState) throws InvalidRequestException
     {
         return new Prepared(this);
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/src/java/org/apache/cassandra/cql3/statements/UseStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/UseStatement.java b/src/java/org/apache/cassandra/cql3/statements/UseStatement.java
index fe3d518..e4685cc 100644
--- a/src/java/org/apache/cassandra/cql3/statements/UseStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/UseStatement.java
@@ -39,7 +39,7 @@ public class UseStatement extends ParsedStatement implements CQLStatement
         return 0;
     }
 
-    public Prepared prepare() throws InvalidRequestException
+    public Prepared prepare(ClientState clientState) throws InvalidRequestException
     {
         return new Prepared(this);
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/src/java/org/apache/cassandra/db/view/View.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/view/View.java b/src/java/org/apache/cassandra/db/view/View.java
index 58e2a84..9716dc4 100644
--- a/src/java/org/apache/cassandra/db/view/View.java
+++ b/src/java/org/apache/cassandra/db/view/View.java
@@ -17,32 +17,38 @@
  */
 package org.apache.cassandra.db.view;
 
-import java.nio.ByteBuffer;
-import java.util.*;
-import java.util.concurrent.TimeUnit;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.UUID;
 import java.util.stream.Collectors;
-
 import javax.annotation.Nullable;
 
 import com.google.common.collect.Iterables;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
-import org.apache.cassandra.cql3.*;
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.config.ViewDefinition;
+import org.apache.cassandra.cql3.ColumnIdentifier;
+import org.apache.cassandra.cql3.MultiColumnRelation;
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.Relation;
+import org.apache.cassandra.cql3.SingleColumnRelation;
+import org.apache.cassandra.cql3.Term;
 import org.apache.cassandra.cql3.statements.ParsedStatement;
 import org.apache.cassandra.cql3.statements.SelectStatement;
-import org.apache.cassandra.db.*;
-import org.apache.cassandra.config.*;
-import org.apache.cassandra.cql3.ColumnIdentifier;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.ReadQuery;
 import org.apache.cassandra.db.compaction.CompactionManager;
-import org.apache.cassandra.db.partitions.*;
-import org.apache.cassandra.db.rows.*;
+import org.apache.cassandra.db.rows.Row;
 import org.apache.cassandra.schema.KeyspaceMetadata;
 import org.apache.cassandra.service.ClientState;
-import org.apache.cassandra.service.pager.QueryPager;
-import org.apache.cassandra.transport.Server;
 import org.apache.cassandra.utils.FBUtilities;
-import org.apache.cassandra.utils.btree.BTreeSet;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 /**
  * A View copies data from a base table into a view table which can be queried independently from the
@@ -176,7 +182,7 @@ public class View
             ClientState state = ClientState.forInternalCalls();
             state.setKeyspace(baseCfs.keyspace.getName());
             rawSelect.prepareKeyspace(state);
-            ParsedStatement.Prepared prepared = rawSelect.prepare(true);
+            ParsedStatement.Prepared prepared = rawSelect.prepare(true, ClientState.forInternalCalls());
             select = (SelectStatement) prepared.statement;
         }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/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 c14c5a7..7cf4c51 100644
--- a/src/java/org/apache/cassandra/index/internal/keys/KeysSearcher.java
+++ b/src/java/org/apache/cassandra/index/internal/keys/KeysSearcher.java
@@ -197,7 +197,13 @@ public class KeysSearcher extends CassandraIndexSearcher
         }
         else
         {
-            assert iterator.metadata().isCompactTable();
+            if (!iterator.metadata().isCompactTable())
+            {
+                logger.warn("Non-composite index was used on the table '{}' during the query. Starting from Cassandra 4.0, only " +
+                            "composite indexes will be supported. If compact flags were dropped for this table, drop and re-create " +
+                            "the index.", iterator.metadata().cfName);
+            }
+
             Row data = iterator.staticRow();
             if (index.isStale(data, indexedValue, nowInSec))
             {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/src/java/org/apache/cassandra/repair/RepairRunnable.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/repair/RepairRunnable.java b/src/java/org/apache/cassandra/repair/RepairRunnable.java
index 213e5c5..77726d4 100644
--- a/src/java/org/apache/cassandra/repair/RepairRunnable.java
+++ b/src/java/org/apache/cassandra/repair/RepairRunnable.java
@@ -45,6 +45,7 @@ import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.repair.messages.RepairOption;
 import org.apache.cassandra.service.ActiveRepairService;
+import org.apache.cassandra.service.ClientState;
 import org.apache.cassandra.service.QueryState;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.tracing.TraceKeyspace;
@@ -385,7 +386,7 @@ public class RepairRunnable extends WrappedRunnable implements ProgressEventNoti
 
                 String format = "select event_id, source, activity from %s.%s where session_id = ? and event_id > ? and event_id < ?;";
                 String query = String.format(format, TraceKeyspace.NAME, TraceKeyspace.EVENTS);
-                SelectStatement statement = (SelectStatement) QueryProcessor.parseStatement(query).prepare().statement;
+                SelectStatement statement = (SelectStatement) QueryProcessor.parseStatement(query).prepare(ClientState.forInternalCalls()).statement;
 
                 ByteBuffer sessionIdBytes = ByteBufferUtil.bytes(sessionId);
                 InetAddress source = FBUtilities.getBroadcastAddress();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/src/java/org/apache/cassandra/service/ClientState.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/ClientState.java b/src/java/org/apache/cassandra/service/ClientState.java
index 32849bc..641d174 100644
--- a/src/java/org/apache/cassandra/service/ClientState.java
+++ b/src/java/org/apache/cassandra/service/ClientState.java
@@ -81,6 +81,12 @@ public class ClientState
     private volatile AuthenticatedUser user;
     private volatile String keyspace;
 
+    /**
+     * Force Compact Tables to be represented as CQL ones for the current client session (simulates
+     * ALTER .. DROP COMPACT STORAGE but only for this session)
+     */
+    private volatile boolean noCompactMode;
+
     private static final QueryHandler cqlQueryHandler;
     static
     {
@@ -253,6 +259,16 @@ public class ClientState
         keyspace = ks;
     }
 
+    public void setNoCompactMode()
+    {
+        this.noCompactMode = true;
+    }
+
+    public boolean isNoCompactMode()
+    {
+        return noCompactMode;
+    }
+
     /**
      * Attempts to login the given user.
      */

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/src/java/org/apache/cassandra/thrift/ThriftValidation.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/thrift/ThriftValidation.java b/src/java/org/apache/cassandra/thrift/ThriftValidation.java
index 99aed05..6ad791d 100644
--- a/src/java/org/apache/cassandra/thrift/ThriftValidation.java
+++ b/src/java/org/apache/cassandra/thrift/ThriftValidation.java
@@ -105,6 +105,11 @@ public class ThriftValidation
     // To be used when the operation should be authorized whether this is a counter CF or not
     public static CFMetaData validateColumnFamily(String keyspaceName, String cfName) throws org.apache.cassandra.exceptions.InvalidRequestException
     {
+        return validateColumnFamilyWithCompactMode(keyspaceName, cfName, false);
+    }
+
+    public static CFMetaData validateColumnFamilyWithCompactMode(String keyspaceName, String cfName, boolean noCompactMode) throws org.apache.cassandra.exceptions.InvalidRequestException
+    {
         validateKeyspace(keyspaceName);
         if (cfName.isEmpty())
             throw new org.apache.cassandra.exceptions.InvalidRequestException("non-empty table is required");
@@ -113,7 +118,10 @@ public class ThriftValidation
         if (metadata == null)
             throw new org.apache.cassandra.exceptions.InvalidRequestException("unconfigured table " + cfName);
 
-        return metadata;
+        if (metadata.isCompactTable() && noCompactMode)
+            return metadata.asNonCompact();
+        else
+            return metadata;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/src/java/org/apache/cassandra/transport/messages/StartupMessage.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/transport/messages/StartupMessage.java b/src/java/org/apache/cassandra/transport/messages/StartupMessage.java
index 04d8e62..774be6a 100644
--- a/src/java/org/apache/cassandra/transport/messages/StartupMessage.java
+++ b/src/java/org/apache/cassandra/transport/messages/StartupMessage.java
@@ -35,6 +35,7 @@ public class StartupMessage extends Message.Request
 {
     public static final String CQL_VERSION = "CQL_VERSION";
     public static final String COMPRESSION = "COMPRESSION";
+    public static final String NO_COMPACT = "NO_COMPACT";
 
     public static final Message.Codec<StartupMessage> codec = new Message.Codec<StartupMessage>()
     {
@@ -97,6 +98,9 @@ public class StartupMessage extends Message.Request
             }
         }
 
+        if (options.containsKey(NO_COMPACT) && Boolean.parseBoolean(options.get(NO_COMPACT)))
+            state.getClientState().setNoCompactMode();
+
         if (DatabaseDescriptor.getAuthenticator().requireAuthentication())
             return new AuthenticateMessage(DatabaseDescriptor.getAuthenticator().getClass().getName());
         else

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/test/unit/org/apache/cassandra/cql3/ViewTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/ViewTest.java b/test/unit/org/apache/cassandra/cql3/ViewTest.java
index 4a4fe1a..136ae1c 100644
--- a/test/unit/org/apache/cassandra/cql3/ViewTest.java
+++ b/test/unit/org/apache/cassandra/cql3/ViewTest.java
@@ -48,6 +48,8 @@ import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.SystemKeyspace;
 import org.apache.cassandra.db.compaction.CompactionManager;
 import org.apache.cassandra.db.marshal.AsciiType;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.exceptions.SyntaxException;
 import org.apache.cassandra.schema.KeyspaceParams;
 import org.apache.cassandra.utils.FBUtilities;
 
@@ -1237,7 +1239,6 @@ public class ViewTest extends CQLTester
         catch (Exception e)
         {
         }
-
     }
 
     @Test
@@ -1376,4 +1377,16 @@ public class ViewTest extends CQLTester
         assertRows(execute("SELECT k, toJson(listval) from mv"),
                    row(0, "[[\"a\", \"1\"], [\"b\", \"2\"], [\"c\", \"3\"]]"));
     }
+
+    @Test(expected = SyntaxException.class)
+    public void emptyViewNameTest() throws Throwable
+    {
+        execute("CREATE MATERIALIZED VIEW \"\" AS SELECT a, b FROM tbl WHERE b IS NOT NULL PRIMARY KEY (b, a)");
+    }
+
+     @Test(expected = SyntaxException.class)
+     public void emptyBaseTableNameTest() throws Throwable
+     {
+         execute("CREATE MATERIALIZED VIEW myview AS SELECT a, b FROM \"\" WHERE b IS NOT NULL PRIMARY KEY (b, a)");
+     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/test/unit/org/apache/cassandra/cql3/validation/entities/UFTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/UFTest.java b/test/unit/org/apache/cassandra/cql3/validation/entities/UFTest.java
index 8a743ea..6e6af19 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/entities/UFTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/UFTest.java
@@ -34,6 +34,7 @@ import org.apache.cassandra.cql3.functions.FunctionName;
 import org.apache.cassandra.cql3.functions.UDFunction;
 import org.apache.cassandra.db.marshal.CollectionType;
 import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.exceptions.SyntaxException;
 import org.apache.cassandra.schema.KeyspaceMetadata;
 import org.apache.cassandra.service.ClientState;
 import org.apache.cassandra.transport.Event;
@@ -968,4 +969,28 @@ public class UFTest extends CQLTester
         assertRows(execute("SELECT " + fNameICC + "(empty_int) FROM %s"), row(0));
         assertRows(execute("SELECT " + fNameICN + "(empty_int) FROM %s"), row(new Object[]{ null }));
     }
+
+    @Test(expected = SyntaxException.class)
+    public void testEmptyFunctionName() throws Throwable
+    {
+        execute("CREATE FUNCTION IF NOT EXISTS " + KEYSPACE + ".\"\" (arg int)\n" +
+                "  RETURNS NULL ON NULL INPUT\n" +
+                "  RETURNS int\n" +
+                "  LANGUAGE java\n" +
+                "  AS $$\n" +
+                "    return a;\n" +
+                "  $$");
+    }
+
+    @Test(expected = SyntaxException.class)
+    public void testEmptyArgName() throws Throwable
+    {
+        execute("CREATE FUNCTION IF NOT EXISTS " + KEYSPACE + ".myfn (\"\" int)\n" +
+                "  RETURNS NULL ON NULL INPUT\n" +
+                "  RETURNS int\n" +
+                "  LANGUAGE java\n" +
+                "  AS $$\n" +
+                "    return a;\n" +
+                "  $$");
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/test/unit/org/apache/cassandra/cql3/validation/entities/UserTypesTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/UserTypesTest.java b/test/unit/org/apache/cassandra/cql3/validation/entities/UserTypesTest.java
index dfc2e5e..68c0b8c 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/entities/UserTypesTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/UserTypesTest.java
@@ -24,6 +24,7 @@ import org.junit.Test;
 
 import org.apache.cassandra.cql3.CQLTester;
 import org.apache.cassandra.dht.ByteOrderedPartitioner;
+import org.apache.cassandra.exceptions.*;
 import org.apache.cassandra.service.StorageService;
 
 public class UserTypesTest extends CQLTester
@@ -713,6 +714,25 @@ public class UserTypesTest extends CQLTester
                        row(1, 1,set(userType(1), userType(1, 1), userType(1, 2), userType(2), userType(2, 1)), 2));
     }
 
+    @Test(expected = SyntaxException.class)
+    public void emptyTypeNameTest() throws Throwable
+    {
+        execute("CREATE TYPE \"\" (a int, b int)");
+    }
+
+    @Test(expected = SyntaxException.class)
+    public void emptyFieldNameTest() throws Throwable
+    {
+        execute("CREATE TYPE mytype (\"\" int, b int)");
+    }
+
+    @Test(expected = SyntaxException.class)
+    public void renameColumnToEmpty() throws Throwable
+    {
+        String typeName = createType("CREATE TYPE %s (a int, b int)");
+        execute(String.format("ALTER TYPE %s.%s RENAME b TO \"\"", keyspace(), typeName));
+    }
+
     private String typeWithKs(String type1)
     {
         return keyspace() + '.' + type1;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/test/unit/org/apache/cassandra/cql3/validation/operations/AlterTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/AlterTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/AlterTest.java
index c48ffe5..b37462f 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/AlterTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/AlterTest.java
@@ -17,6 +17,9 @@
  */
 package org.apache.cassandra.cql3.validation.operations;
 
+import org.junit.Assert;
+import org.junit.Test;
+
 import org.apache.cassandra.cql3.CQLTester;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.Keyspace;
@@ -24,11 +27,6 @@ import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.exceptions.SyntaxException;
 import org.apache.cassandra.schema.SchemaKeyspace;
-import org.apache.cassandra.transport.Server;
-import org.apache.cassandra.utils.ByteBufferUtil;
-
-import org.junit.Assert;
-import org.junit.Test;
 
 import static java.lang.String.format;
 import static org.junit.Assert.assertEquals;
@@ -266,6 +264,13 @@ public class AlterTest extends CQLTester
         execute("alter table %s add v int");
     }
 
+    @Test(expected = SyntaxException.class)
+    public void renameToEmptyTest() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int, c1 int, v int, PRIMARY KEY (k, c1))");
+        execute("ALTER TABLE %s RENAME c1 TO \"\"");
+    }
+
     @Test
     // tests CASSANDRA-9565
     public void testDoubleWith() throws Throwable


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


[10/25] cassandra git commit: Merge branch 'cassandra-3.0' into cassandra-3.11

Posted by if...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d429cd0/src/antlr/Parser.g
----------------------------------------------------------------------
diff --cc src/antlr/Parser.g
index e5b7584,0000000..26074b8
mode 100644,000000..100644
--- a/src/antlr/Parser.g
+++ b/src/antlr/Parser.g
@@@ -1,1675 -1,0 +1,1688 @@@
 +/*
 + * 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.
 + */
 +
 +parser grammar Parser;
 +
 +options {
 +    language = Java;
 +}
 +
 +@members {
 +    private final List<ErrorListener> listeners = new ArrayList<ErrorListener>();
 +    protected final List<ColumnIdentifier> bindVariables = new ArrayList<ColumnIdentifier>();
 +
 +    public static final Set<String> reservedTypeNames = new HashSet<String>()
 +    {{
 +        add("byte");
 +        add("complex");
 +        add("enum");
 +        add("date");
 +        add("interval");
 +        add("macaddr");
 +        add("bitstring");
 +    }};
 +
 +    public AbstractMarker.Raw newBindVariables(ColumnIdentifier name)
 +    {
 +        AbstractMarker.Raw marker = new AbstractMarker.Raw(bindVariables.size());
 +        bindVariables.add(name);
 +        return marker;
 +    }
 +
 +    public AbstractMarker.INRaw newINBindVariables(ColumnIdentifier name)
 +    {
 +        AbstractMarker.INRaw marker = new AbstractMarker.INRaw(bindVariables.size());
 +        bindVariables.add(name);
 +        return marker;
 +    }
 +
 +    public Tuples.Raw newTupleBindVariables(ColumnIdentifier name)
 +    {
 +        Tuples.Raw marker = new Tuples.Raw(bindVariables.size());
 +        bindVariables.add(name);
 +        return marker;
 +    }
 +
 +    public Tuples.INRaw newTupleINBindVariables(ColumnIdentifier name)
 +    {
 +        Tuples.INRaw marker = new Tuples.INRaw(bindVariables.size());
 +        bindVariables.add(name);
 +        return marker;
 +    }
 +
 +    public Json.Marker newJsonBindVariables(ColumnIdentifier name)
 +    {
 +        Json.Marker marker = new Json.Marker(bindVariables.size());
 +        bindVariables.add(name);
 +        return marker;
 +    }
 +
 +    public void addErrorListener(ErrorListener listener)
 +    {
 +        this.listeners.add(listener);
 +    }
 +
 +    public void removeErrorListener(ErrorListener listener)
 +    {
 +        this.listeners.remove(listener);
 +    }
 +
 +    public void displayRecognitionError(String[] tokenNames, RecognitionException e)
 +    {
 +        for (int i = 0, m = listeners.size(); i < m; i++)
 +            listeners.get(i).syntaxError(this, tokenNames, e);
 +    }
 +
 +    protected void addRecognitionError(String msg)
 +    {
 +        for (int i = 0, m = listeners.size(); i < m; i++)
 +            listeners.get(i).syntaxError(this, msg);
 +    }
 +
 +    public Map<String, String> convertPropertyMap(Maps.Literal map)
 +    {
 +        if (map == null || map.entries == null || map.entries.isEmpty())
 +            return Collections.<String, String>emptyMap();
 +
 +        Map<String, String> res = new HashMap<>(map.entries.size());
 +
 +        for (Pair<Term.Raw, Term.Raw> entry : map.entries)
 +        {
 +            // Because the parser tries to be smart and recover on error (to
 +            // allow displaying more than one error I suppose), we have null
 +            // entries in there. Just skip those, a proper error will be thrown in the end.
 +            if (entry.left == null || entry.right == null)
 +                break;
 +
 +            if (!(entry.left instanceof Constants.Literal))
 +            {
 +                String msg = "Invalid property name: " + entry.left;
 +                if (entry.left instanceof AbstractMarker.Raw)
 +                    msg += " (bind variables are not supported in DDL queries)";
 +                addRecognitionError(msg);
 +                break;
 +            }
 +            if (!(entry.right instanceof Constants.Literal))
 +            {
 +                String msg = "Invalid property value: " + entry.right + " for property: " + entry.left;
 +                if (entry.right instanceof AbstractMarker.Raw)
 +                    msg += " (bind variables are not supported in DDL queries)";
 +                addRecognitionError(msg);
 +                break;
 +            }
 +
 +            if (res.put(((Constants.Literal)entry.left).getRawText(), ((Constants.Literal)entry.right).getRawText()) != null)
 +            {
 +                addRecognitionError(String.format("Multiple definition for property " + ((Constants.Literal)entry.left).getRawText()));
 +            }
 +        }
 +
 +        return res;
 +    }
 +
 +    public void addRawUpdate(List<Pair<ColumnDefinition.Raw, Operation.RawUpdate>> operations, ColumnDefinition.Raw key, Operation.RawUpdate update)
 +    {
 +        for (Pair<ColumnDefinition.Raw, Operation.RawUpdate> p : operations)
 +        {
 +            if (p.left.equals(key) && !p.right.isCompatibleWith(update))
 +                addRecognitionError("Multiple incompatible setting of column " + key);
 +        }
 +        operations.add(Pair.create(key, update));
 +    }
 +
 +    public Set<Permission> filterPermissions(Set<Permission> permissions, IResource resource)
 +    {
 +        if (resource == null)
 +            return Collections.emptySet();
 +        Set<Permission> filtered = new HashSet<>(permissions);
 +        filtered.retainAll(resource.applicablePermissions());
 +        if (filtered.isEmpty())
 +            addRecognitionError("Resource type " + resource.getClass().getSimpleName() +
 +                                    " does not support any of the requested permissions");
 +
 +        return filtered;
 +    }
 +
 +    public String canonicalizeObjectName(String s, boolean enforcePattern)
 +    {
 +        // these two conditions are here because technically they are valid
 +        // ObjectNames, but we want to restrict their use without adding unnecessary
 +        // work to JMXResource construction as that also happens on hotter code paths
 +        if ("".equals(s))
 +            addRecognitionError("Empty JMX object name supplied");
 +
 +        if ("*:*".equals(s))
 +            addRecognitionError("Please use ALL MBEANS instead of wildcard pattern");
 +
 +        try
 +        {
 +            javax.management.ObjectName objectName = javax.management.ObjectName.getInstance(s);
 +            if (enforcePattern && !objectName.isPattern())
 +                addRecognitionError("Plural form used, but non-pattern JMX object name specified (" + s + ")");
 +            return objectName.getCanonicalName();
 +        }
 +        catch (javax.management.MalformedObjectNameException e)
 +        {
 +          addRecognitionError(s + " is not a valid JMX object name");
 +          return s;
 +        }
 +    }
 +
 +    ////////////////////////////////////////////////////////////////////////////////////////////////////////////////
 +    // Recovery methods are overridden to avoid wasting work on recovering from errors when the result will be
 +    // ignored anyway.
 +    ////////////////////////////////////////////////////////////////////////////////////////////////////////////////
 +
 +    @Override
 +    protected Object recoverFromMismatchedToken(IntStream input, int ttype, BitSet follow) throws RecognitionException
 +    {
 +        throw new MismatchedTokenException(ttype, input);
 +    }
 +
 +    @Override
 +    public void recover(IntStream input, RecognitionException re)
 +    {
 +        // Do nothing.
 +    }
 +}
 +
 +/** STATEMENTS **/
 +
 +cqlStatement returns [ParsedStatement stmt]
 +    @after{ if (stmt != null) stmt.setBoundVariables(bindVariables); }
 +    : st1= selectStatement                 { $stmt = st1; }
 +    | st2= insertStatement                 { $stmt = st2; }
 +    | st3= updateStatement                 { $stmt = st3; }
 +    | st4= batchStatement                  { $stmt = st4; }
 +    | st5= deleteStatement                 { $stmt = st5; }
 +    | st6= useStatement                    { $stmt = st6; }
 +    | st7= truncateStatement               { $stmt = st7; }
 +    | st8= createKeyspaceStatement         { $stmt = st8; }
 +    | st9= createTableStatement            { $stmt = st9; }
 +    | st10=createIndexStatement            { $stmt = st10; }
 +    | st11=dropKeyspaceStatement           { $stmt = st11; }
 +    | st12=dropTableStatement              { $stmt = st12; }
 +    | st13=dropIndexStatement              { $stmt = st13; }
 +    | st14=alterTableStatement             { $stmt = st14; }
 +    | st15=alterKeyspaceStatement          { $stmt = st15; }
 +    | st16=grantPermissionsStatement       { $stmt = st16; }
 +    | st17=revokePermissionsStatement      { $stmt = st17; }
 +    | st18=listPermissionsStatement        { $stmt = st18; }
 +    | st19=createUserStatement             { $stmt = st19; }
 +    | st20=alterUserStatement              { $stmt = st20; }
 +    | st21=dropUserStatement               { $stmt = st21; }
 +    | st22=listUsersStatement              { $stmt = st22; }
 +    | st23=createTriggerStatement          { $stmt = st23; }
 +    | st24=dropTriggerStatement            { $stmt = st24; }
 +    | st25=createTypeStatement             { $stmt = st25; }
 +    | st26=alterTypeStatement              { $stmt = st26; }
 +    | st27=dropTypeStatement               { $stmt = st27; }
 +    | st28=createFunctionStatement         { $stmt = st28; }
 +    | st29=dropFunctionStatement           { $stmt = st29; }
 +    | st30=createAggregateStatement        { $stmt = st30; }
 +    | st31=dropAggregateStatement          { $stmt = st31; }
 +    | st32=createRoleStatement             { $stmt = st32; }
 +    | st33=alterRoleStatement              { $stmt = st33; }
 +    | st34=dropRoleStatement               { $stmt = st34; }
 +    | st35=listRolesStatement              { $stmt = st35; }
 +    | st36=grantRoleStatement              { $stmt = st36; }
 +    | st37=revokeRoleStatement             { $stmt = st37; }
 +    | st38=createMaterializedViewStatement { $stmt = st38; }
 +    | st39=dropMaterializedViewStatement   { $stmt = st39; }
 +    | st40=alterMaterializedViewStatement  { $stmt = st40; }
 +    ;
 +
 +/*
 + * USE <KEYSPACE>;
 + */
 +useStatement returns [UseStatement stmt]
 +    : K_USE ks=keyspaceName { $stmt = new UseStatement(ks); }
 +    ;
 +
 +/**
 + * SELECT <expression>
 + * FROM <CF>
 + * WHERE KEY = "key1" AND COL > 1 AND COL < 100
 + * LIMIT <NUMBER>;
 + */
 +selectStatement returns [SelectStatement.RawStatement expr]
 +    @init {
 +        boolean isDistinct = false;
 +        Term.Raw limit = null;
 +        Term.Raw perPartitionLimit = null;
 +        Map<ColumnDefinition.Raw, Boolean> orderings = new LinkedHashMap<>();
 +        List<ColumnDefinition.Raw> groups = new ArrayList<>();
 +        boolean allowFiltering = false;
 +        boolean isJson = false;
 +    }
 +    : K_SELECT
 +      ( K_JSON { isJson = true; } )?
 +      ( ( K_DISTINCT { isDistinct = true; } )? sclause=selectClause )
 +      K_FROM cf=columnFamilyName
 +      ( K_WHERE wclause=whereClause )?
 +      ( K_GROUP K_BY groupByClause[groups] ( ',' groupByClause[groups] )* )?
 +      ( K_ORDER K_BY orderByClause[orderings] ( ',' orderByClause[orderings] )* )?
 +      ( K_PER K_PARTITION K_LIMIT rows=intValue { perPartitionLimit = rows; } )?
 +      ( K_LIMIT rows=intValue { limit = rows; } )?
 +      ( K_ALLOW K_FILTERING  { allowFiltering = true; } )?
 +      {
 +          SelectStatement.Parameters params = new SelectStatement.Parameters(orderings,
 +                                                                             groups,
 +                                                                             isDistinct,
 +                                                                             allowFiltering,
 +                                                                             isJson);
 +          WhereClause where = wclause == null ? WhereClause.empty() : wclause.build();
 +          $expr = new SelectStatement.RawStatement(cf, params, sclause, where, limit, perPartitionLimit);
 +      }
 +    ;
 +
 +selectClause returns [List<RawSelector> expr]
 +    : t1=selector { $expr = new ArrayList<RawSelector>(); $expr.add(t1); } (',' tN=selector { $expr.add(tN); })*
 +    | '\*' { $expr = Collections.<RawSelector>emptyList();}
 +    ;
 +
 +selector returns [RawSelector s]
 +    @init{ ColumnIdentifier alias = null; }
 +    : us=unaliasedSelector (K_AS c=noncol_ident { alias = c; })? { $s = new RawSelector(us, alias); }
 +    ;
 +
 +/*
 + * A single selection. The core of it is selecting a column, but we also allow any term and function, as well as
 + * sub-element selection for UDT.
 + */
 +unaliasedSelector returns [Selectable.Raw s]
 +    @init { Selectable.Raw tmp = null; }
 +    :  ( c=cident                                  { tmp = c; }
 +       | v=value                                   { tmp = new Selectable.WithTerm.Raw(v); }
 +       | '(' ct=comparatorType ')' v=value         { tmp = new Selectable.WithTerm.Raw(new TypeCast(ct, v)); }
 +       | K_COUNT '(' '\*' ')'                      { tmp = Selectable.WithFunction.Raw.newCountRowsFunction(); }
 +       | K_WRITETIME '(' c=cident ')'              { tmp = new Selectable.WritetimeOrTTL.Raw(c, true); }
 +       | K_TTL       '(' c=cident ')'              { tmp = new Selectable.WritetimeOrTTL.Raw(c, false); }
 +       | K_CAST      '(' sn=unaliasedSelector K_AS t=native_type ')' {tmp = new Selectable.WithCast.Raw(sn, t);}
 +       | f=functionName args=selectionFunctionArgs { tmp = new Selectable.WithFunction.Raw(f, args); }
 +       ) ( '.' fi=fident { tmp = new Selectable.WithFieldSelection.Raw(tmp, fi); } )* { $s = tmp; }
 +    ;
 +
 +selectionFunctionArgs returns [List<Selectable.Raw> a]
 +    : '(' ')' { $a = Collections.emptyList(); }
 +    | '(' s1=unaliasedSelector { List<Selectable.Raw> args = new ArrayList<Selectable.Raw>(); args.add(s1); }
 +          ( ',' sn=unaliasedSelector { args.add(sn); } )*
 +      ')' { $a = args; }
 +    ;
 +
 +whereClause returns [WhereClause.Builder clause]
 +    @init{ $clause = new WhereClause.Builder(); }
 +    : relationOrExpression[$clause] (K_AND relationOrExpression[$clause])*
 +    ;
 +
 +relationOrExpression [WhereClause.Builder clause]
 +    : relation[$clause]
 +    | customIndexExpression[$clause]
 +    ;
 +
 +customIndexExpression [WhereClause.Builder clause]
 +    @init{IndexName name = new IndexName();}
 +    : 'expr(' idxName[name] ',' t=term ')' { clause.add(new CustomIndexExpression(name, t));}
 +    ;
 +
 +orderByClause[Map<ColumnDefinition.Raw, Boolean> orderings]
 +    @init{
 +        boolean reversed = false;
 +    }
 +    : c=cident (K_ASC | K_DESC { reversed = true; })? { orderings.put(c, reversed); }
 +    ;
 +
 +groupByClause[List<ColumnDefinition.Raw> groups]
 +    : c=cident { groups.add(c); }
 +    ;
 +
 +/**
 + * INSERT INTO <CF> (<column>, <column>, <column>, ...)
 + * VALUES (<value>, <value>, <value>, ...)
 + * USING TIMESTAMP <long>;
 + *
 + */
 +insertStatement returns [ModificationStatement.Parsed expr]
 +    : K_INSERT K_INTO cf=columnFamilyName
 +        ( st1=normalInsertStatement[cf] { $expr = st1; }
 +        | K_JSON st2=jsonInsertStatement[cf] { $expr = st2; })
 +    ;
 +
 +normalInsertStatement [CFName cf] returns [UpdateStatement.ParsedInsert expr]
 +    @init {
 +        Attributes.Raw attrs = new Attributes.Raw();
 +        List<ColumnDefinition.Raw> columnNames  = new ArrayList<>();
 +        List<Term.Raw> values = new ArrayList<>();
 +        boolean ifNotExists = false;
 +    }
 +    : '(' c1=cident { columnNames.add(c1); }  ( ',' cn=cident { columnNames.add(cn); } )* ')'
 +      K_VALUES
 +      '(' v1=term { values.add(v1); } ( ',' vn=term { values.add(vn); } )* ')'
 +      ( K_IF K_NOT K_EXISTS { ifNotExists = true; } )?
 +      ( usingClause[attrs] )?
 +      {
 +          $expr = new UpdateStatement.ParsedInsert(cf, attrs, columnNames, values, ifNotExists);
 +      }
 +    ;
 +
 +jsonInsertStatement [CFName cf] returns [UpdateStatement.ParsedInsertJson expr]
 +    @init {
 +        Attributes.Raw attrs = new Attributes.Raw();
 +        boolean ifNotExists = false;
 +        boolean defaultUnset = false;
 +    }
 +    : val=jsonValue
 +      ( K_DEFAULT ( K_NULL | ( { defaultUnset = true; } K_UNSET) ) )?
 +      ( K_IF K_NOT K_EXISTS { ifNotExists = true; } )?
 +      ( usingClause[attrs] )?
 +      {
 +          $expr = new UpdateStatement.ParsedInsertJson(cf, attrs, val, defaultUnset, ifNotExists);
 +      }
 +    ;
 +
 +jsonValue returns [Json.Raw value]
 +    : s=STRING_LITERAL { $value = new Json.Literal($s.text); }
 +    | ':' id=noncol_ident     { $value = newJsonBindVariables(id); }
 +    | QMARK            { $value = newJsonBindVariables(null); }
 +    ;
 +
 +usingClause[Attributes.Raw attrs]
 +    : K_USING usingClauseObjective[attrs] ( K_AND usingClauseObjective[attrs] )*
 +    ;
 +
 +usingClauseObjective[Attributes.Raw attrs]
 +    : K_TIMESTAMP ts=intValue { attrs.timestamp = ts; }
 +    | K_TTL t=intValue { attrs.timeToLive = t; }
 +    ;
 +
 +/**
 + * UPDATE <CF>
 + * USING TIMESTAMP <long>
 + * SET name1 = value1, name2 = value2
 + * WHERE key = value;
 + * [IF (EXISTS | name = value, ...)];
 + */
 +updateStatement returns [UpdateStatement.ParsedUpdate expr]
 +    @init {
 +        Attributes.Raw attrs = new Attributes.Raw();
 +        List<Pair<ColumnDefinition.Raw, Operation.RawUpdate>> operations = new ArrayList<>();
 +        boolean ifExists = false;
 +    }
 +    : K_UPDATE cf=columnFamilyName
 +      ( usingClause[attrs] )?
 +      K_SET columnOperation[operations] (',' columnOperation[operations])*
 +      K_WHERE wclause=whereClause
 +      ( K_IF ( K_EXISTS { ifExists = true; } | conditions=updateConditions ))?
 +      {
 +          $expr = new UpdateStatement.ParsedUpdate(cf,
 +                                                   attrs,
 +                                                   operations,
 +                                                   wclause.build(),
 +                                                   conditions == null ? Collections.<Pair<ColumnDefinition.Raw, ColumnCondition.Raw>>emptyList() : conditions,
 +                                                   ifExists);
 +     }
 +    ;
 +
 +updateConditions returns [List<Pair<ColumnDefinition.Raw, ColumnCondition.Raw>> conditions]
 +    @init { conditions = new ArrayList<Pair<ColumnDefinition.Raw, ColumnCondition.Raw>>(); }
 +    : columnCondition[conditions] ( K_AND columnCondition[conditions] )*
 +    ;
 +
 +
 +/**
 + * DELETE name1, name2
 + * FROM <CF>
 + * USING TIMESTAMP <long>
 + * WHERE KEY = keyname
 +   [IF (EXISTS | name = value, ...)];
 + */
 +deleteStatement returns [DeleteStatement.Parsed expr]
 +    @init {
 +        Attributes.Raw attrs = new Attributes.Raw();
 +        List<Operation.RawDeletion> columnDeletions = Collections.emptyList();
 +        boolean ifExists = false;
 +    }
 +    : K_DELETE ( dels=deleteSelection { columnDeletions = dels; } )?
 +      K_FROM cf=columnFamilyName
 +      ( usingClauseDelete[attrs] )?
 +      K_WHERE wclause=whereClause
 +      ( K_IF ( K_EXISTS { ifExists = true; } | conditions=updateConditions ))?
 +      {
 +          $expr = new DeleteStatement.Parsed(cf,
 +                                             attrs,
 +                                             columnDeletions,
 +                                             wclause.build(),
 +                                             conditions == null ? Collections.<Pair<ColumnDefinition.Raw, ColumnCondition.Raw>>emptyList() : conditions,
 +                                             ifExists);
 +      }
 +    ;
 +
 +deleteSelection returns [List<Operation.RawDeletion> operations]
 +    : { $operations = new ArrayList<Operation.RawDeletion>(); }
 +          t1=deleteOp { $operations.add(t1); }
 +          (',' tN=deleteOp { $operations.add(tN); })*
 +    ;
 +
 +deleteOp returns [Operation.RawDeletion op]
 +    : c=cident                { $op = new Operation.ColumnDeletion(c); }
 +    | c=cident '[' t=term ']' { $op = new Operation.ElementDeletion(c, t); }
 +    | c=cident '.' field=fident { $op = new Operation.FieldDeletion(c, field); }
 +    ;
 +
 +usingClauseDelete[Attributes.Raw attrs]
 +    : K_USING K_TIMESTAMP ts=intValue { attrs.timestamp = ts; }
 +    ;
 +
 +/**
 + * BEGIN BATCH
 + *   UPDATE <CF> SET name1 = value1 WHERE KEY = keyname1;
 + *   UPDATE <CF> SET name2 = value2 WHERE KEY = keyname2;
 + *   UPDATE <CF> SET name3 = value3 WHERE KEY = keyname3;
 + *   ...
 + * APPLY BATCH
 + *
 + * OR
 + *
 + * BEGIN BATCH
 + *   INSERT INTO <CF> (KEY, <name>) VALUES ('<key>', '<value>');
 + *   INSERT INTO <CF> (KEY, <name>) VALUES ('<key>', '<value>');
 + *   ...
 + * APPLY BATCH
 + *
 + * OR
 + *
 + * BEGIN BATCH
 + *   DELETE name1, name2 FROM <CF> WHERE key = <key>
 + *   DELETE name3, name4 FROM <CF> WHERE key = <key>
 + *   ...
 + * APPLY BATCH
 + */
 +batchStatement returns [BatchStatement.Parsed expr]
 +    @init {
 +        BatchStatement.Type type = BatchStatement.Type.LOGGED;
 +        List<ModificationStatement.Parsed> statements = new ArrayList<ModificationStatement.Parsed>();
 +        Attributes.Raw attrs = new Attributes.Raw();
 +    }
 +    : K_BEGIN
 +      ( K_UNLOGGED { type = BatchStatement.Type.UNLOGGED; } | K_COUNTER { type = BatchStatement.Type.COUNTER; } )?
 +      K_BATCH ( usingClause[attrs] )?
 +          ( s=batchStatementObjective ';'? { statements.add(s); } )*
 +      K_APPLY K_BATCH
 +      {
 +          $expr = new BatchStatement.Parsed(type, attrs, statements);
 +      }
 +    ;
 +
 +batchStatementObjective returns [ModificationStatement.Parsed statement]
 +    : i=insertStatement  { $statement = i; }
 +    | u=updateStatement  { $statement = u; }
 +    | d=deleteStatement  { $statement = d; }
 +    ;
 +
 +createAggregateStatement returns [CreateAggregateStatement expr]
 +    @init {
 +        boolean orReplace = false;
 +        boolean ifNotExists = false;
 +
 +        List<CQL3Type.Raw> argsTypes = new ArrayList<>();
 +    }
 +    : K_CREATE (K_OR K_REPLACE { orReplace = true; })?
 +      K_AGGREGATE
 +      (K_IF K_NOT K_EXISTS { ifNotExists = true; })?
 +      fn=functionName
 +      '('
 +        (
 +          v=comparatorType { argsTypes.add(v); }
 +          ( ',' v=comparatorType { argsTypes.add(v); } )*
 +        )?
 +      ')'
 +      K_SFUNC sfunc = allowedFunctionName
 +      K_STYPE stype = comparatorType
 +      (
 +        K_FINALFUNC ffunc = allowedFunctionName
 +      )?
 +      (
 +        K_INITCOND ival = term
 +      )?
 +      { $expr = new CreateAggregateStatement(fn, argsTypes, sfunc, stype, ffunc, ival, orReplace, ifNotExists); }
 +    ;
 +
 +dropAggregateStatement returns [DropAggregateStatement expr]
 +    @init {
 +        boolean ifExists = false;
 +        List<CQL3Type.Raw> argsTypes = new ArrayList<>();
 +        boolean argsPresent = false;
 +    }
 +    : K_DROP K_AGGREGATE
 +      (K_IF K_EXISTS { ifExists = true; } )?
 +      fn=functionName
 +      (
 +        '('
 +          (
 +            v=comparatorType { argsTypes.add(v); }
 +            ( ',' v=comparatorType { argsTypes.add(v); } )*
 +          )?
 +        ')'
 +        { argsPresent = true; }
 +      )?
 +      { $expr = new DropAggregateStatement(fn, argsTypes, argsPresent, ifExists); }
 +    ;
 +
 +createFunctionStatement returns [CreateFunctionStatement expr]
 +    @init {
 +        boolean orReplace = false;
 +        boolean ifNotExists = false;
 +
 +        List<ColumnIdentifier> argsNames = new ArrayList<>();
 +        List<CQL3Type.Raw> argsTypes = new ArrayList<>();
 +        boolean calledOnNullInput = false;
 +    }
 +    : K_CREATE (K_OR K_REPLACE { orReplace = true; })?
 +      K_FUNCTION
 +      (K_IF K_NOT K_EXISTS { ifNotExists = true; })?
 +      fn=functionName
 +      '('
 +        (
 +          k=noncol_ident v=comparatorType { argsNames.add(k); argsTypes.add(v); }
 +          ( ',' k=noncol_ident v=comparatorType { argsNames.add(k); argsTypes.add(v); } )*
 +        )?
 +      ')'
 +      ( (K_RETURNS K_NULL) | (K_CALLED { calledOnNullInput=true; })) K_ON K_NULL K_INPUT
 +      K_RETURNS rt = comparatorType
 +      K_LANGUAGE language = IDENT
 +      K_AS body = STRING_LITERAL
 +      { $expr = new CreateFunctionStatement(fn, $language.text.toLowerCase(), $body.text,
 +                                            argsNames, argsTypes, rt, calledOnNullInput, orReplace, ifNotExists); }
 +    ;
 +
 +dropFunctionStatement returns [DropFunctionStatement expr]
 +    @init {
 +        boolean ifExists = false;
 +        List<CQL3Type.Raw> argsTypes = new ArrayList<>();
 +        boolean argsPresent = false;
 +    }
 +    : K_DROP K_FUNCTION
 +      (K_IF K_EXISTS { ifExists = true; } )?
 +      fn=functionName
 +      (
 +        '('
 +          (
 +            v=comparatorType { argsTypes.add(v); }
 +            ( ',' v=comparatorType { argsTypes.add(v); } )*
 +          )?
 +        ')'
 +        { argsPresent = true; }
 +      )?
 +      { $expr = new DropFunctionStatement(fn, argsTypes, argsPresent, ifExists); }
 +    ;
 +
 +/**
 + * CREATE KEYSPACE [IF NOT EXISTS] <KEYSPACE> WITH attr1 = value1 AND attr2 = value2;
 + */
 +createKeyspaceStatement returns [CreateKeyspaceStatement expr]
 +    @init {
 +        KeyspaceAttributes attrs = new KeyspaceAttributes();
 +        boolean ifNotExists = false;
 +    }
 +    : K_CREATE K_KEYSPACE (K_IF K_NOT K_EXISTS { ifNotExists = true; } )? ks=keyspaceName
 +      K_WITH properties[attrs] { $expr = new CreateKeyspaceStatement(ks, attrs, ifNotExists); }
 +    ;
 +
 +/**
 + * CREATE COLUMNFAMILY [IF NOT EXISTS] <CF> (
 + *     <name1> <type>,
 + *     <name2> <type>,
 + *     <name3> <type>
 + * ) WITH <property> = <value> AND ...;
 + */
 +createTableStatement returns [CreateTableStatement.RawStatement expr]
 +    @init { boolean ifNotExists = false; }
 +    : K_CREATE K_COLUMNFAMILY (K_IF K_NOT K_EXISTS { ifNotExists = true; } )?
 +      cf=columnFamilyName { $expr = new CreateTableStatement.RawStatement(cf, ifNotExists); }
 +      cfamDefinition[expr]
 +    ;
 +
 +cfamDefinition[CreateTableStatement.RawStatement expr]
 +    : '(' cfamColumns[expr] ( ',' cfamColumns[expr]? )* ')'
 +      ( K_WITH cfamProperty[expr.properties] ( K_AND cfamProperty[expr.properties] )*)?
 +    ;
 +
 +cfamColumns[CreateTableStatement.RawStatement expr]
 +    : k=ident v=comparatorType { boolean isStatic=false; } (K_STATIC {isStatic = true;})? { $expr.addDefinition(k, v, isStatic); }
 +        (K_PRIMARY K_KEY { $expr.addKeyAliases(Collections.singletonList(k)); })?
 +    | K_PRIMARY K_KEY '(' pkDef[expr] (',' c=ident { $expr.addColumnAlias(c); } )* ')'
 +    ;
 +
 +pkDef[CreateTableStatement.RawStatement expr]
 +    : k=ident { $expr.addKeyAliases(Collections.singletonList(k)); }
 +    | '(' { List<ColumnIdentifier> l = new ArrayList<ColumnIdentifier>(); } k1=ident { l.add(k1); } ( ',' kn=ident { l.add(kn); } )* ')' { $expr.addKeyAliases(l); }
 +    ;
 +
 +cfamProperty[CFProperties props]
 +    : property[props.properties]
 +    | K_COMPACT K_STORAGE { $props.setCompactStorage(); }
 +    | K_CLUSTERING K_ORDER K_BY '(' cfamOrdering[props] (',' cfamOrdering[props])* ')'
 +    ;
 +
 +cfamOrdering[CFProperties props]
 +    @init{ boolean reversed=false; }
 +    : k=ident (K_ASC | K_DESC { reversed=true;} ) { $props.setOrdering(k, reversed); }
 +    ;
 +
 +
 +/**
 + * CREATE TYPE foo (
 + *    <name1> <type1>,
 + *    <name2> <type2>,
 + *    ....
 + * )
 + */
 +createTypeStatement returns [CreateTypeStatement expr]
 +    @init { boolean ifNotExists = false; }
 +    : K_CREATE K_TYPE (K_IF K_NOT K_EXISTS { ifNotExists = true; } )?
 +         tn=userTypeName { $expr = new CreateTypeStatement(tn, ifNotExists); }
 +         '(' typeColumns[expr] ( ',' typeColumns[expr]? )* ')'
 +    ;
 +
 +typeColumns[CreateTypeStatement expr]
 +    : k=fident v=comparatorType { $expr.addDefinition(k, v); }
 +    ;
 +
 +
 +/**
 + * CREATE INDEX [IF NOT EXISTS] [indexName] ON <columnFamily> (<columnName>);
 + * CREATE CUSTOM INDEX [IF NOT EXISTS] [indexName] ON <columnFamily> (<columnName>) USING <indexClass>;
 + */
 +createIndexStatement returns [CreateIndexStatement expr]
 +    @init {
 +        IndexPropDefs props = new IndexPropDefs();
 +        boolean ifNotExists = false;
 +        IndexName name = new IndexName();
 +        List<IndexTarget.Raw> targets = new ArrayList<>();
 +    }
 +    : K_CREATE (K_CUSTOM { props.isCustom = true; })? K_INDEX (K_IF K_NOT K_EXISTS { ifNotExists = true; } )?
 +        (idxName[name])? K_ON cf=columnFamilyName '(' (indexIdent[targets] (',' indexIdent[targets])*)? ')'
 +        (K_USING cls=STRING_LITERAL { props.customClass = $cls.text; })?
 +        (K_WITH properties[props])?
 +      { $expr = new CreateIndexStatement(cf, name, targets, props, ifNotExists); }
 +    ;
 +
 +indexIdent [List<IndexTarget.Raw> targets]
 +    : c=cident                   { $targets.add(IndexTarget.Raw.simpleIndexOn(c)); }
 +    | K_VALUES '(' c=cident ')'  { $targets.add(IndexTarget.Raw.valuesOf(c)); }
 +    | K_KEYS '(' c=cident ')'    { $targets.add(IndexTarget.Raw.keysOf(c)); }
 +    | K_ENTRIES '(' c=cident ')' { $targets.add(IndexTarget.Raw.keysAndValuesOf(c)); }
 +    | K_FULL '(' c=cident ')'    { $targets.add(IndexTarget.Raw.fullCollection(c)); }
 +    ;
 +
 +/**
 + * CREATE MATERIALIZED VIEW <viewName> AS
 + *  SELECT <columns>
 + *  FROM <CF>
 + *  WHERE <pkColumns> IS NOT NULL
 + *  PRIMARY KEY (<pkColumns>)
 + *  WITH <property> = <value> AND ...;
 + */
 +createMaterializedViewStatement returns [CreateViewStatement expr]
 +    @init {
 +        boolean ifNotExists = false;
 +        List<ColumnDefinition.Raw> partitionKeys = new ArrayList<>();
 +        List<ColumnDefinition.Raw> compositeKeys = new ArrayList<>();
 +    }
 +    : K_CREATE K_MATERIALIZED K_VIEW (K_IF K_NOT K_EXISTS { ifNotExists = true; })? cf=columnFamilyName K_AS
 +        K_SELECT sclause=selectClause K_FROM basecf=columnFamilyName
 +        (K_WHERE wclause=whereClause)?
 +        K_PRIMARY K_KEY (
 +        '(' '(' k1=cident { partitionKeys.add(k1); } ( ',' kn=cident { partitionKeys.add(kn); } )* ')' ( ',' c1=cident { compositeKeys.add(c1); } )* ')'
 +    |   '(' k1=cident { partitionKeys.add(k1); } ( ',' cn=cident { compositeKeys.add(cn); } )* ')'
 +        )
 +        {
 +             WhereClause where = wclause == null ? WhereClause.empty() : wclause.build();
 +             $expr = new CreateViewStatement(cf, basecf, sclause, where, partitionKeys, compositeKeys, ifNotExists);
 +        }
 +        ( K_WITH cfamProperty[expr.properties] ( K_AND cfamProperty[expr.properties] )*)?
 +    ;
 +
 +/**
 + * CREATE TRIGGER triggerName ON columnFamily USING 'triggerClass';
 + */
 +createTriggerStatement returns [CreateTriggerStatement expr]
 +    @init {
 +        boolean ifNotExists = false;
 +    }
-     : K_CREATE K_TRIGGER (K_IF K_NOT K_EXISTS { ifNotExists = true; } )? (name=cident)
++    : K_CREATE K_TRIGGER (K_IF K_NOT K_EXISTS { ifNotExists = true; } )? (name=ident)
 +        K_ON cf=columnFamilyName K_USING cls=STRING_LITERAL
-       { $expr = new CreateTriggerStatement(cf, name.rawText(), $cls.text, ifNotExists); }
++      { $expr = new CreateTriggerStatement(cf, name.toString(), $cls.text, ifNotExists); }
 +    ;
 +
 +/**
 + * DROP TRIGGER [IF EXISTS] triggerName ON columnFamily;
 + */
 +dropTriggerStatement returns [DropTriggerStatement expr]
 +     @init { boolean ifExists = false; }
-     : K_DROP K_TRIGGER (K_IF K_EXISTS { ifExists = true; } )? (name=cident) K_ON cf=columnFamilyName
-       { $expr = new DropTriggerStatement(cf, name.rawText(), ifExists); }
++    : K_DROP K_TRIGGER (K_IF K_EXISTS { ifExists = true; } )? (name=ident) K_ON cf=columnFamilyName
++      { $expr = new DropTriggerStatement(cf, name.toString(), ifExists); }
 +    ;
 +
 +/**
 + * ALTER KEYSPACE <KS> WITH <property> = <value>;
 + */
 +alterKeyspaceStatement returns [AlterKeyspaceStatement expr]
 +    @init { KeyspaceAttributes attrs = new KeyspaceAttributes(); }
 +    : K_ALTER K_KEYSPACE ks=keyspaceName
 +        K_WITH properties[attrs] { $expr = new AlterKeyspaceStatement(ks, attrs); }
 +    ;
 +
 +/**
 + * ALTER COLUMN FAMILY <CF> ALTER <column> TYPE <newtype>;
 + * ALTER COLUMN FAMILY <CF> ADD <column> <newtype>; | ALTER COLUMN FAMILY <CF> ADD (<column> <newtype>,<column1> <newtype1>..... <column n> <newtype n>)
 + * ALTER COLUMN FAMILY <CF> DROP <column>; | ALTER COLUMN FAMILY <CF> DROP ( <column>,<column1>.....<column n>)
 + * ALTER COLUMN FAMILY <CF> WITH <property> = <value>;
 + * ALTER COLUMN FAMILY <CF> RENAME <column> TO <column>;
 + */
 +alterTableStatement returns [AlterTableStatement expr]
 +    @init {
 +        AlterTableStatement.Type type = null;
 +        TableAttributes attrs = new TableAttributes();
 +        Map<ColumnDefinition.Raw, ColumnDefinition.Raw> renames = new HashMap<ColumnDefinition.Raw, ColumnDefinition.Raw>();
 +        List<AlterTableStatementColumn> colNameList = new ArrayList<AlterTableStatementColumn>();
 +        Long deleteTimestamp = null;
 +    }
 +    : K_ALTER K_COLUMNFAMILY cf=columnFamilyName
-           ( K_ALTER id=cident  K_TYPE v=comparatorType  { type = AlterTableStatement.Type.ALTER; } { colNameList.add(new AlterTableStatementColumn(id,v)); }
-           | K_ADD  (        (id=cident   v=comparatorType   b1=cfisStatic { colNameList.add(new AlterTableStatementColumn(id,v,b1)); })
-                      | ('('  id1=cident  v1=comparatorType  b1=cfisStatic { colNameList.add(new AlterTableStatementColumn(id1,v1,b1)); }
-                        ( ',' idn=cident  vn=comparatorType  bn=cfisStatic { colNameList.add(new AlterTableStatementColumn(idn,vn,bn)); } )* ')' ) ) { type = AlterTableStatement.Type.ADD; }
-           | K_DROP ( (         id=cident  { colNameList.add(new AlterTableStatementColumn(id)); }
-                       | ('('  id1=cident { colNameList.add(new AlterTableStatementColumn(id1)); }
-                         ( ',' idn=cident { colNameList.add(new AlterTableStatementColumn(idn)); } )* ')') )
++          ( K_ALTER id=schema_cident  K_TYPE v=comparatorType  { type = AlterTableStatement.Type.ALTER; } { colNameList.add(new AlterTableStatementColumn(id,v)); }
++          | K_ADD  (        (aid=schema_cident  v=comparatorType   b1=cfisStatic { colNameList.add(new AlterTableStatementColumn(aid,v,b1)); })
++                     | ('('  id1=schema_cident  v1=comparatorType  b1=cfisStatic { colNameList.add(new AlterTableStatementColumn(id1,v1,b1)); }
++                       ( ',' idn=schema_cident  vn=comparatorType  bn=cfisStatic { colNameList.add(new AlterTableStatementColumn(idn,vn,bn)); } )* ')' ) ) { type = AlterTableStatement.Type.ADD; }
++          | K_DROP K_COMPACT K_STORAGE          { type = AlterTableStatement.Type.DROP_COMPACT_STORAGE; }        
++          | K_DROP ( (        id=schema_cident  { colNameList.add(new AlterTableStatementColumn(id)); }
++                      | ('('  id1=schema_cident { colNameList.add(new AlterTableStatementColumn(id1)); }
++                        ( ',' idn=schema_cident { colNameList.add(new AlterTableStatementColumn(idn)); } )* ')') )
 +                     ( K_USING K_TIMESTAMP t=INTEGER { deleteTimestamp = Long.parseLong(Constants.Literal.integer($t.text).getText()); })? ) { type = AlterTableStatement.Type.DROP; }
 +          | K_WITH  properties[attrs]                 { type = AlterTableStatement.Type.OPTS; }
 +          | K_RENAME                                  { type = AlterTableStatement.Type.RENAME; }
-                id1=cident K_TO toId1=cident { renames.put(id1, toId1); }
-                ( K_AND idn=cident K_TO toIdn=cident { renames.put(idn, toIdn); } )*
++               id1=schema_cident K_TO toId1=schema_cident { renames.put(id1, toId1); }
++               ( K_AND idn=schema_cident K_TO toIdn=schema_cident { renames.put(idn, toIdn); } )*
 +          )
 +    {
 +        $expr = new AlterTableStatement(cf, type, colNameList, attrs, renames, deleteTimestamp);
 +    }
 +    ;
 +
 +cfisStatic returns [boolean isStaticColumn]
 +    @init{
 +        boolean isStatic = false;
 +    }
 +    : (K_STATIC { isStatic=true; })? { $isStaticColumn = isStatic;
 +    }
 +    ;
 +
 +alterMaterializedViewStatement returns [AlterViewStatement expr]
 +    @init {
 +        TableAttributes attrs = new TableAttributes();
 +    }
 +    : K_ALTER K_MATERIALIZED K_VIEW name=columnFamilyName
 +          K_WITH properties[attrs]
 +    {
 +        $expr = new AlterViewStatement(name, attrs);
 +    }
 +    ;
 +
 +
 +/**
 + * ALTER TYPE <name> ALTER <field> TYPE <newtype>;
 + * ALTER TYPE <name> ADD <field> <newtype>;
 + * ALTER TYPE <name> RENAME <field> TO <newtype> AND ...;
 + */
 +alterTypeStatement returns [AlterTypeStatement expr]
 +    : K_ALTER K_TYPE name=userTypeName
 +          ( K_ALTER f=fident K_TYPE v=comparatorType { $expr = AlterTypeStatement.alter(name, f, v); }
 +          | K_ADD   f=fident v=comparatorType        { $expr = AlterTypeStatement.addition(name, f, v); }
 +          | K_RENAME
 +               { Map<FieldIdentifier, FieldIdentifier> renames = new HashMap<>(); }
 +                 id1=fident K_TO toId1=fident { renames.put(id1, toId1); }
 +                 ( K_AND idn=fident K_TO toIdn=fident { renames.put(idn, toIdn); } )*
 +               { $expr = AlterTypeStatement.renames(name, renames); }
 +          )
 +    ;
 +
 +
 +/**
 + * DROP KEYSPACE [IF EXISTS] <KSP>;
 + */
 +dropKeyspaceStatement returns [DropKeyspaceStatement ksp]
 +    @init { boolean ifExists = false; }
 +    : K_DROP K_KEYSPACE (K_IF K_EXISTS { ifExists = true; } )? ks=keyspaceName { $ksp = new DropKeyspaceStatement(ks, ifExists); }
 +    ;
 +
 +/**
 + * DROP COLUMNFAMILY [IF EXISTS] <CF>;
 + */
 +dropTableStatement returns [DropTableStatement stmt]
 +    @init { boolean ifExists = false; }
 +    : K_DROP K_COLUMNFAMILY (K_IF K_EXISTS { ifExists = true; } )? cf=columnFamilyName { $stmt = new DropTableStatement(cf, ifExists); }
 +    ;
 +
 +/**
 + * DROP TYPE <name>;
 + */
 +dropTypeStatement returns [DropTypeStatement stmt]
 +    @init { boolean ifExists = false; }
 +    : K_DROP K_TYPE (K_IF K_EXISTS { ifExists = true; } )? name=userTypeName { $stmt = new DropTypeStatement(name, ifExists); }
 +    ;
 +
 +/**
 + * DROP INDEX [IF EXISTS] <INDEX_NAME>
 + */
 +dropIndexStatement returns [DropIndexStatement expr]
 +    @init { boolean ifExists = false; }
 +    : K_DROP K_INDEX (K_IF K_EXISTS { ifExists = true; } )? index=indexName
 +      { $expr = new DropIndexStatement(index, ifExists); }
 +    ;
 +
 +/**
 + * DROP MATERIALIZED VIEW [IF EXISTS] <view_name>
 + */
 +dropMaterializedViewStatement returns [DropViewStatement expr]
 +    @init { boolean ifExists = false; }
 +    : K_DROP K_MATERIALIZED K_VIEW (K_IF K_EXISTS { ifExists = true; } )? cf=columnFamilyName
 +      { $expr = new DropViewStatement(cf, ifExists); }
 +    ;
 +
 +/**
 +  * TRUNCATE <CF>;
 +  */
 +truncateStatement returns [TruncateStatement stmt]
 +    : K_TRUNCATE (K_COLUMNFAMILY)? cf=columnFamilyName { $stmt = new TruncateStatement(cf); }
 +    ;
 +
 +/**
 + * GRANT <permission> ON <resource> TO <rolename>
 + */
 +grantPermissionsStatement returns [GrantPermissionsStatement stmt]
 +    : K_GRANT
 +          permissionOrAll
 +      K_ON
 +          resource
 +      K_TO
 +          grantee=userOrRoleName
 +      { $stmt = new GrantPermissionsStatement(filterPermissions($permissionOrAll.perms, $resource.res), $resource.res, grantee); }
 +    ;
 +
 +/**
 + * REVOKE <permission> ON <resource> FROM <rolename>
 + */
 +revokePermissionsStatement returns [RevokePermissionsStatement stmt]
 +    : K_REVOKE
 +          permissionOrAll
 +      K_ON
 +          resource
 +      K_FROM
 +          revokee=userOrRoleName
 +      { $stmt = new RevokePermissionsStatement(filterPermissions($permissionOrAll.perms, $resource.res), $resource.res, revokee); }
 +    ;
 +
 +/**
 + * GRANT ROLE <rolename> TO <grantee>
 + */
 +grantRoleStatement returns [GrantRoleStatement stmt]
 +    : K_GRANT
 +          role=userOrRoleName
 +      K_TO
 +          grantee=userOrRoleName
 +      { $stmt = new GrantRoleStatement(role, grantee); }
 +    ;
 +
 +/**
 + * REVOKE ROLE <rolename> FROM <revokee>
 + */
 +revokeRoleStatement returns [RevokeRoleStatement stmt]
 +    : K_REVOKE
 +          role=userOrRoleName
 +      K_FROM
 +          revokee=userOrRoleName
 +      { $stmt = new RevokeRoleStatement(role, revokee); }
 +    ;
 +
 +listPermissionsStatement returns [ListPermissionsStatement stmt]
 +    @init {
 +        IResource resource = null;
 +        boolean recursive = true;
 +        RoleName grantee = new RoleName();
 +    }
 +    : K_LIST
 +          permissionOrAll
 +      ( K_ON resource { resource = $resource.res; } )?
 +      ( K_OF roleName[grantee] )?
 +      ( K_NORECURSIVE { recursive = false; } )?
 +      { $stmt = new ListPermissionsStatement($permissionOrAll.perms, resource, grantee, recursive); }
 +    ;
 +
 +permission returns [Permission perm]
 +    : p=(K_CREATE | K_ALTER | K_DROP | K_SELECT | K_MODIFY | K_AUTHORIZE | K_DESCRIBE | K_EXECUTE)
 +    { $perm = Permission.valueOf($p.text.toUpperCase()); }
 +    ;
 +
 +permissionOrAll returns [Set<Permission> perms]
 +    : K_ALL ( K_PERMISSIONS )?       { $perms = Permission.ALL; }
 +    | p=permission ( K_PERMISSION )? { $perms = EnumSet.of($p.perm); }
 +    ;
 +
 +resource returns [IResource res]
 +    : d=dataResource { $res = $d.res; }
 +    | r=roleResource { $res = $r.res; }
 +    | f=functionResource { $res = $f.res; }
 +    | j=jmxResource { $res = $j.res; }
 +    ;
 +
 +dataResource returns [DataResource res]
 +    : K_ALL K_KEYSPACES { $res = DataResource.root(); }
 +    | K_KEYSPACE ks = keyspaceName { $res = DataResource.keyspace($ks.id); }
 +    | ( K_COLUMNFAMILY )? cf = columnFamilyName
 +      { $res = DataResource.table($cf.name.getKeyspace(), $cf.name.getColumnFamily()); }
 +    ;
 +
 +jmxResource returns [JMXResource res]
 +    : K_ALL K_MBEANS { $res = JMXResource.root(); }
 +    // when a bean name (or pattern) is supplied, validate that it's a legal ObjectName
 +    // also, just to be picky, if the "MBEANS" form is used, only allow a pattern style names
 +    | K_MBEAN mbean { $res = JMXResource.mbean(canonicalizeObjectName($mbean.text, false)); }
 +    | K_MBEANS mbean { $res = JMXResource.mbean(canonicalizeObjectName($mbean.text, true)); }
 +    ;
 +
 +roleResource returns [RoleResource res]
 +    : K_ALL K_ROLES { $res = RoleResource.root(); }
 +    | K_ROLE role = userOrRoleName { $res = RoleResource.role($role.name.getName()); }
 +    ;
 +
 +functionResource returns [FunctionResource res]
 +    @init {
 +        List<CQL3Type.Raw> argsTypes = new ArrayList<>();
 +    }
 +    : K_ALL K_FUNCTIONS { $res = FunctionResource.root(); }
 +    | K_ALL K_FUNCTIONS K_IN K_KEYSPACE ks = keyspaceName { $res = FunctionResource.keyspace($ks.id); }
 +    // Arg types are mandatory for DCL statements on Functions
 +    | K_FUNCTION fn=functionName
 +      (
 +        '('
 +          (
 +            v=comparatorType { argsTypes.add(v); }
 +            ( ',' v=comparatorType { argsTypes.add(v); } )*
 +          )?
 +        ')'
 +      )
 +      { $res = FunctionResource.functionFromCql($fn.s.keyspace, $fn.s.name, argsTypes); }
 +    ;
 +
 +/**
 + * CREATE USER [IF NOT EXISTS] <username> [WITH PASSWORD <password>] [SUPERUSER|NOSUPERUSER]
 + */
 +createUserStatement returns [CreateRoleStatement stmt]
 +    @init {
 +        RoleOptions opts = new RoleOptions();
 +        opts.setOption(IRoleManager.Option.LOGIN, true);
 +        boolean superuser = false;
 +        boolean ifNotExists = false;
 +        RoleName name = new RoleName();
 +    }
 +    : K_CREATE K_USER (K_IF K_NOT K_EXISTS { ifNotExists = true; })? u=username { name.setName($u.text, true); }
 +      ( K_WITH userPassword[opts] )?
 +      ( K_SUPERUSER { superuser = true; } | K_NOSUPERUSER { superuser = false; } )?
 +      { opts.setOption(IRoleManager.Option.SUPERUSER, superuser);
 +        $stmt = new CreateRoleStatement(name, opts, ifNotExists); }
 +    ;
 +
 +/**
 + * ALTER USER <username> [WITH PASSWORD <password>] [SUPERUSER|NOSUPERUSER]
 + */
 +alterUserStatement returns [AlterRoleStatement stmt]
 +    @init {
 +        RoleOptions opts = new RoleOptions();
 +        RoleName name = new RoleName();
 +    }
 +    : K_ALTER K_USER u=username { name.setName($u.text, true); }
 +      ( K_WITH userPassword[opts] )?
 +      ( K_SUPERUSER { opts.setOption(IRoleManager.Option.SUPERUSER, true); }
 +        | K_NOSUPERUSER { opts.setOption(IRoleManager.Option.SUPERUSER, false); } ) ?
 +      {  $stmt = new AlterRoleStatement(name, opts); }
 +    ;
 +
 +/**
 + * DROP USER [IF EXISTS] <username>
 + */
 +dropUserStatement returns [DropRoleStatement stmt]
 +    @init {
 +        boolean ifExists = false;
 +        RoleName name = new RoleName();
 +    }
 +    : K_DROP K_USER (K_IF K_EXISTS { ifExists = true; })? u=username { name.setName($u.text, true); $stmt = new DropRoleStatement(name, ifExists); }
 +    ;
 +
 +/**
 + * LIST USERS
 + */
 +listUsersStatement returns [ListRolesStatement stmt]
 +    : K_LIST K_USERS { $stmt = new ListUsersStatement(); }
 +    ;
 +
 +/**
 + * CREATE ROLE [IF NOT EXISTS] <rolename> [ [WITH] option [ [AND] option ]* ]
 + *
 + * where option can be:
 + *  PASSWORD = '<password>'
 + *  SUPERUSER = (true|false)
 + *  LOGIN = (true|false)
 + *  OPTIONS = { 'k1':'v1', 'k2':'v2'}
 + */
 +createRoleStatement returns [CreateRoleStatement stmt]
 +    @init {
 +        RoleOptions opts = new RoleOptions();
 +        boolean ifNotExists = false;
 +    }
 +    : K_CREATE K_ROLE (K_IF K_NOT K_EXISTS { ifNotExists = true; })? name=userOrRoleName
 +      ( K_WITH roleOptions[opts] )?
 +      {
 +        // set defaults if they weren't explictly supplied
 +        if (!opts.getLogin().isPresent())
 +        {
 +            opts.setOption(IRoleManager.Option.LOGIN, false);
 +        }
 +        if (!opts.getSuperuser().isPresent())
 +        {
 +            opts.setOption(IRoleManager.Option.SUPERUSER, false);
 +        }
 +        $stmt = new CreateRoleStatement(name, opts, ifNotExists);
 +      }
 +    ;
 +
 +/**
 + * ALTER ROLE <rolename> [ [WITH] option [ [AND] option ]* ]
 + *
 + * where option can be:
 + *  PASSWORD = '<password>'
 + *  SUPERUSER = (true|false)
 + *  LOGIN = (true|false)
 + *  OPTIONS = { 'k1':'v1', 'k2':'v2'}
 + */
 +alterRoleStatement returns [AlterRoleStatement stmt]
 +    @init {
 +        RoleOptions opts = new RoleOptions();
 +    }
 +    : K_ALTER K_ROLE name=userOrRoleName
 +      ( K_WITH roleOptions[opts] )?
 +      {  $stmt = new AlterRoleStatement(name, opts); }
 +    ;
 +
 +/**
 + * DROP ROLE [IF EXISTS] <rolename>
 + */
 +dropRoleStatement returns [DropRoleStatement stmt]
 +    @init {
 +        boolean ifExists = false;
 +    }
 +    : K_DROP K_ROLE (K_IF K_EXISTS { ifExists = true; })? name=userOrRoleName
 +      { $stmt = new DropRoleStatement(name, ifExists); }
 +    ;
 +
 +/**
 + * LIST ROLES [OF <rolename>] [NORECURSIVE]
 + */
 +listRolesStatement returns [ListRolesStatement stmt]
 +    @init {
 +        boolean recursive = true;
 +        RoleName grantee = new RoleName();
 +    }
 +    : K_LIST K_ROLES
 +      ( K_OF roleName[grantee])?
 +      ( K_NORECURSIVE { recursive = false; } )?
 +      { $stmt = new ListRolesStatement(grantee, recursive); }
 +    ;
 +
 +roleOptions[RoleOptions opts]
 +    : roleOption[opts] (K_AND roleOption[opts])*
 +    ;
 +
 +roleOption[RoleOptions opts]
 +    :  K_PASSWORD '=' v=STRING_LITERAL { opts.setOption(IRoleManager.Option.PASSWORD, $v.text); }
 +    |  K_OPTIONS '=' m=mapLiteral { opts.setOption(IRoleManager.Option.OPTIONS, convertPropertyMap(m)); }
 +    |  K_SUPERUSER '=' b=BOOLEAN { opts.setOption(IRoleManager.Option.SUPERUSER, Boolean.valueOf($b.text)); }
 +    |  K_LOGIN '=' b=BOOLEAN { opts.setOption(IRoleManager.Option.LOGIN, Boolean.valueOf($b.text)); }
 +    ;
 +
 +// for backwards compatibility in CREATE/ALTER USER, this has no '='
 +userPassword[RoleOptions opts]
 +    :  K_PASSWORD v=STRING_LITERAL { opts.setOption(IRoleManager.Option.PASSWORD, $v.text); }
 +    ;
 +
 +/** DEFINITIONS **/
 +
 +// Column Identifiers.  These need to be treated differently from other
 +// identifiers because the underlying comparator is not necessarily text. See
 +// CASSANDRA-8178 for details.
++// Also, we need to support the internal of the super column map (for backward
++// compatibility) which is empty (we only want to allow this is in data manipulation
++// queries, not in schema defition etc).
 +cident returns [ColumnDefinition.Raw id]
++    : EMPTY_QUOTED_NAME    { $id = ColumnDefinition.Raw.forQuoted(""); }
++    | t=IDENT              { $id = ColumnDefinition.Raw.forUnquoted($t.text); }
++    | t=QUOTED_NAME        { $id = ColumnDefinition.Raw.forQuoted($t.text); }
++    | k=unreserved_keyword { $id = ColumnDefinition.Raw.forUnquoted(k); }
++    ;
++
++schema_cident returns [ColumnDefinition.Raw id]
 +    : t=IDENT              { $id = ColumnDefinition.Raw.forUnquoted($t.text); }
 +    | t=QUOTED_NAME        { $id = ColumnDefinition.Raw.forQuoted($t.text); }
 +    | k=unreserved_keyword { $id = ColumnDefinition.Raw.forUnquoted(k); }
 +    ;
 +
 +// Column identifiers where the comparator is known to be text
 +ident returns [ColumnIdentifier id]
 +    : t=IDENT              { $id = ColumnIdentifier.getInterned($t.text, false); }
 +    | t=QUOTED_NAME        { $id = ColumnIdentifier.getInterned($t.text, true); }
 +    | k=unreserved_keyword { $id = ColumnIdentifier.getInterned(k, false); }
 +    ;
 +
 +fident returns [FieldIdentifier id]
 +    : t=IDENT              { $id = FieldIdentifier.forUnquoted($t.text); }
 +    | t=QUOTED_NAME        { $id = FieldIdentifier.forQuoted($t.text); }
 +    | k=unreserved_keyword { $id = FieldIdentifier.forUnquoted(k); }
 +    ;
 +
 +// Identifiers that do not refer to columns
 +noncol_ident returns [ColumnIdentifier id]
 +    : t=IDENT              { $id = new ColumnIdentifier($t.text, false); }
 +    | t=QUOTED_NAME        { $id = new ColumnIdentifier($t.text, true); }
 +    | k=unreserved_keyword { $id = new ColumnIdentifier(k, false); }
 +    ;
 +
 +// Keyspace & Column family names
 +keyspaceName returns [String id]
 +    @init { CFName name = new CFName(); }
 +    : ksName[name] { $id = name.getKeyspace(); }
 +    ;
 +
 +indexName returns [IndexName name]
 +    @init { $name = new IndexName(); }
 +    : (ksName[name] '.')? idxName[name]
 +    ;
 +
 +columnFamilyName returns [CFName name]
 +    @init { $name = new CFName(); }
 +    : (ksName[name] '.')? cfName[name]
 +    ;
 +
 +userTypeName returns [UTName name]
 +    : (ks=noncol_ident '.')? ut=non_type_ident { $name = new UTName(ks, ut); }
 +    ;
 +
 +userOrRoleName returns [RoleName name]
 +    @init { RoleName role = new RoleName(); }
 +    : roleName[role] {$name = role;}
 +    ;
 +
 +ksName[KeyspaceElementName name]
 +    : t=IDENT              { $name.setKeyspace($t.text, false);}
 +    | t=QUOTED_NAME        { $name.setKeyspace($t.text, true);}
 +    | k=unreserved_keyword { $name.setKeyspace(k, false);}
 +    | QMARK {addRecognitionError("Bind variables cannot be used for keyspace names");}
 +    ;
 +
 +cfName[CFName name]
 +    : t=IDENT              { $name.setColumnFamily($t.text, false); }
 +    | t=QUOTED_NAME        { $name.setColumnFamily($t.text, true); }
 +    | k=unreserved_keyword { $name.setColumnFamily(k, false); }
 +    | QMARK {addRecognitionError("Bind variables cannot be used for table names");}
 +    ;
 +
 +idxName[IndexName name]
 +    : t=IDENT              { $name.setIndex($t.text, false); }
 +    | t=QUOTED_NAME        { $name.setIndex($t.text, true);}
 +    | k=unreserved_keyword { $name.setIndex(k, false); }
 +    | QMARK {addRecognitionError("Bind variables cannot be used for index names");}
 +    ;
 +
 +roleName[RoleName name]
 +    : t=IDENT              { $name.setName($t.text, false); }
 +    | s=STRING_LITERAL     { $name.setName($s.text, true); }
 +    | t=QUOTED_NAME        { $name.setName($t.text, true); }
 +    | k=unreserved_keyword { $name.setName(k, false); }
 +    | QMARK {addRecognitionError("Bind variables cannot be used for role names");}
 +    ;
 +
 +constant returns [Constants.Literal constant]
 +    : t=STRING_LITERAL { $constant = Constants.Literal.string($t.text); }
 +    | t=INTEGER        { $constant = Constants.Literal.integer($t.text); }
 +    | t=FLOAT          { $constant = Constants.Literal.floatingPoint($t.text); }
 +    | t=BOOLEAN        { $constant = Constants.Literal.bool($t.text); }
 +    | t=DURATION       { $constant = Constants.Literal.duration($t.text);}
 +    | t=UUID           { $constant = Constants.Literal.uuid($t.text); }
 +    | t=HEXNUMBER      { $constant = Constants.Literal.hex($t.text); }
 +    | { String sign=""; } ('-' {sign = "-"; } )? t=(K_NAN | K_INFINITY) { $constant = Constants.Literal.floatingPoint(sign + $t.text); }
 +    ;
 +
 +mapLiteral returns [Maps.Literal map]
 +    : '{' { List<Pair<Term.Raw, Term.Raw>> m = new ArrayList<Pair<Term.Raw, Term.Raw>>(); }
 +          ( k1=term ':' v1=term { m.add(Pair.create(k1, v1)); } ( ',' kn=term ':' vn=term { m.add(Pair.create(kn, vn)); } )* )?
 +      '}' { $map = new Maps.Literal(m); }
 +    ;
 +
 +setOrMapLiteral[Term.Raw t] returns [Term.Raw value]
 +    : ':' v=term { List<Pair<Term.Raw, Term.Raw>> m = new ArrayList<Pair<Term.Raw, Term.Raw>>(); m.add(Pair.create(t, v)); }
 +          ( ',' kn=term ':' vn=term { m.add(Pair.create(kn, vn)); } )*
 +      { $value = new Maps.Literal(m); }
 +    | { List<Term.Raw> s = new ArrayList<Term.Raw>(); s.add(t); }
 +          ( ',' tn=term { s.add(tn); } )*
 +      { $value = new Sets.Literal(s); }
 +    ;
 +
 +collectionLiteral returns [Term.Raw value]
 +    : '[' { List<Term.Raw> l = new ArrayList<Term.Raw>(); }
 +          ( t1=term { l.add(t1); } ( ',' tn=term { l.add(tn); } )* )?
 +      ']' { $value = new Lists.Literal(l); }
 +    | '{' t=term v=setOrMapLiteral[t] { $value = v; } '}'
 +    // Note that we have an ambiguity between maps and set for "{}". So we force it to a set literal,
 +    // and deal with it later based on the type of the column (SetLiteral.java).
 +    | '{' '}' { $value = new Sets.Literal(Collections.<Term.Raw>emptyList()); }
 +    ;
 +
 +usertypeLiteral returns [UserTypes.Literal ut]
 +    @init{ Map<FieldIdentifier, Term.Raw> m = new HashMap<>(); }
 +    @after{ $ut = new UserTypes.Literal(m); }
 +    // We don't allow empty literals because that conflicts with sets/maps and is currently useless since we don't allow empty user types
 +    : '{' k1=fident ':' v1=term { m.put(k1, v1); } ( ',' kn=fident ':' vn=term { m.put(kn, vn); } )* '}'
 +    ;
 +
 +tupleLiteral returns [Tuples.Literal tt]
 +    @init{ List<Term.Raw> l = new ArrayList<Term.Raw>(); }
 +    @after{ $tt = new Tuples.Literal(l); }
 +    : '(' t1=term { l.add(t1); } ( ',' tn=term { l.add(tn); } )* ')'
 +    ;
 +
 +value returns [Term.Raw value]
 +    : c=constant           { $value = c; }
 +    | l=collectionLiteral  { $value = l; }
 +    | u=usertypeLiteral    { $value = u; }
 +    | t=tupleLiteral       { $value = t; }
 +    | K_NULL               { $value = Constants.NULL_LITERAL; }
 +    | ':' id=noncol_ident  { $value = newBindVariables(id); }
 +    | QMARK                { $value = newBindVariables(null); }
 +    ;
 +
 +intValue returns [Term.Raw value]
 +    : t=INTEGER     { $value = Constants.Literal.integer($t.text); }
 +    | ':' id=noncol_ident  { $value = newBindVariables(id); }
 +    | QMARK         { $value = newBindVariables(null); }
 +    ;
 +
 +functionName returns [FunctionName s]
-     : (ks=keyspaceName '.')? f=allowedFunctionName   { $s = new FunctionName(ks, f); }
++     // antlr might try to recover and give a null for f. It will still error out in the end, but FunctionName
++     // wouldn't be happy with that so we should bypass this for now or we'll have a weird user-facing error
++    : (ks=keyspaceName '.')? f=allowedFunctionName   { $s = f == null ? null : new FunctionName(ks, f); }
 +    ;
 +
 +allowedFunctionName returns [String s]
 +    : f=IDENT                       { $s = $f.text.toLowerCase(); }
 +    | f=QUOTED_NAME                 { $s = $f.text; }
 +    | u=unreserved_function_keyword { $s = u; }
 +    | K_TOKEN                       { $s = "token"; }
 +    | K_COUNT                       { $s = "count"; }
 +    ;
 +
 +function returns [Term.Raw t]
 +    : f=functionName '(' ')'                   { $t = new FunctionCall.Raw(f, Collections.<Term.Raw>emptyList()); }
 +    | f=functionName '(' args=functionArgs ')' { $t = new FunctionCall.Raw(f, args); }
 +    ;
 +
 +functionArgs returns [List<Term.Raw> args]
 +    @init{ $args = new ArrayList<Term.Raw>(); }
 +    : t1=term {args.add(t1); } ( ',' tn=term { args.add(tn); } )*
 +    ;
 +
 +term returns [Term.Raw term]
 +    : v=value                          { $term = v; }
 +    | f=function                       { $term = f; }
 +    | '(' c=comparatorType ')' t=term  { $term = new TypeCast(c, t); }
 +    ;
 +
 +columnOperation[List<Pair<ColumnDefinition.Raw, Operation.RawUpdate>> operations]
 +    : key=cident columnOperationDifferentiator[operations, key]
 +    ;
 +
 +columnOperationDifferentiator[List<Pair<ColumnDefinition.Raw, Operation.RawUpdate>> operations, ColumnDefinition.Raw key]
 +    : '=' normalColumnOperation[operations, key]
 +    | shorthandColumnOperation[operations, key]
 +    | '[' k=term ']' collectionColumnOperation[operations, key, k]
 +    | '.' field=fident udtColumnOperation[operations, key, field]
 +    ;
 +
 +normalColumnOperation[List<Pair<ColumnDefinition.Raw, Operation.RawUpdate>> operations, ColumnDefinition.Raw key]
 +    : t=term ('+' c=cident )?
 +      {
 +          if (c == null)
 +          {
 +              addRawUpdate(operations, key, new Operation.SetValue(t));
 +          }
 +          else
 +          {
 +              if (!key.equals(c))
 +                  addRecognitionError("Only expressions of the form X = <value> + X are supported.");
 +              addRawUpdate(operations, key, new Operation.Prepend(t));
 +          }
 +      }
 +    | c=cident sig=('+' | '-') t=term
 +      {
 +          if (!key.equals(c))
 +              addRecognitionError("Only expressions of the form X = X " + $sig.text + "<value> are supported.");
 +          addRawUpdate(operations, key, $sig.text.equals("+") ? new Operation.Addition(t) : new Operation.Substraction(t));
 +      }
 +    | c=cident i=INTEGER
 +      {
 +          // Note that this production *is* necessary because X = X - 3 will in fact be lexed as [ X, '=', X, INTEGER].
 +          if (!key.equals(c))
 +              // We don't yet allow a '+' in front of an integer, but we could in the future really, so let's be future-proof in our error message
 +              addRecognitionError("Only expressions of the form X = X " + ($i.text.charAt(0) == '-' ? '-' : '+') + " <value> are supported.");
 +          addRawUpdate(operations, key, new Operation.Addition(Constants.Literal.integer($i.text)));
 +      }
 +    ;
 +
 +shorthandColumnOperation[List<Pair<ColumnDefinition.Raw, Operation.RawUpdate>> operations, ColumnDefinition.Raw key]
 +    : sig=('+=' | '-=') t=term
 +      {
 +          addRawUpdate(operations, key, $sig.text.equals("+=") ? new Operation.Addition(t) : new Operation.Substraction(t));
 +      }
 +    ;
 +
 +collectionColumnOperation[List<Pair<ColumnDefinition.Raw, Operation.RawUpdate>> operations, ColumnDefinition.Raw key, Term.Raw k]
 +    : '=' t=term
 +      {
 +          addRawUpdate(operations, key, new Operation.SetElement(k, t));
 +      }
 +    ;
 +
 +udtColumnOperation[List<Pair<ColumnDefinition.Raw, Operation.RawUpdate>> operations, ColumnDefinition.Raw key, FieldIdentifier field]
 +    : '=' t=term
 +      {
 +          addRawUpdate(operations, key, new Operation.SetField(field, t));
 +      }
 +    ;
 +
 +columnCondition[List<Pair<ColumnDefinition.Raw, ColumnCondition.Raw>> conditions]
 +    // Note: we'll reject duplicates later
 +    : key=cident
 +        ( op=relationType t=term { conditions.add(Pair.create(key, ColumnCondition.Raw.simpleCondition(t, op))); }
 +        | K_IN
 +            ( values=singleColumnInValues { conditions.add(Pair.create(key, ColumnCondition.Raw.simpleInCondition(values))); }
 +            | marker=inMarker { conditions.add(Pair.create(key, ColumnCondition.Raw.simpleInCondition(marker))); }
 +            )
 +        | '[' element=term ']'
 +            ( op=relationType t=term { conditions.add(Pair.create(key, ColumnCondition.Raw.collectionCondition(t, element, op))); }
 +            | K_IN
 +                ( values=singleColumnInValues { conditions.add(Pair.create(key, ColumnCondition.Raw.collectionInCondition(element, values))); }
 +                | marker=inMarker { conditions.add(Pair.create(key, ColumnCondition.Raw.collectionInCondition(element, marker))); }
 +                )
 +            )
 +        | '.' field=fident
 +            ( op=relationType t=term { conditions.add(Pair.create(key, ColumnCondition.Raw.udtFieldCondition(t, field, op))); }
 +            | K_IN
 +                ( values=singleColumnInValues { conditions.add(Pair.create(key, ColumnCondition.Raw.udtFieldInCondition(field, values))); }
 +                | marker=inMarker { conditions.add(Pair.create(key, ColumnCondition.Raw.udtFieldInCondition(field, marker))); }
 +                )
 +            )
 +        )
 +    ;
 +
 +properties[PropertyDefinitions props]
 +    : property[props] (K_AND property[props])*
 +    ;
 +
 +property[PropertyDefinitions props]
 +    : k=noncol_ident '=' simple=propertyValue { try { $props.addProperty(k.toString(), simple); } catch (SyntaxException e) { addRecognitionError(e.getMessage()); } }
 +    | k=noncol_ident '=' map=mapLiteral { try { $props.addProperty(k.toString(), convertPropertyMap(map)); } catch (SyntaxException e) { addRecognitionError(e.getMessage()); } }
 +    ;
 +
 +propertyValue returns [String str]
 +    : c=constant           { $str = c.getRawText(); }
 +    | u=unreserved_keyword { $str = u; }
 +    ;
 +
 +relationType returns [Operator op]
 +    : '='  { $op = Operator.EQ; }
 +    | '<'  { $op = Operator.LT; }
 +    | '<=' { $op = Operator.LTE; }
 +    | '>'  { $op = Operator.GT; }
 +    | '>=' { $op = Operator.GTE; }
 +    | '!=' { $op = Operator.NEQ; }
 +    ;
 +
 +relation[WhereClause.Builder clauses]
 +    : name=cident type=relationType t=term { $clauses.add(new SingleColumnRelation(name, type, t)); }
 +    | name=cident K_LIKE t=term { $clauses.add(new SingleColumnRelation(name, Operator.LIKE, t)); }
 +    | name=cident K_IS K_NOT K_NULL { $clauses.add(new SingleColumnRelation(name, Operator.IS_NOT, Constants.NULL_LITERAL)); }
 +    | K_TOKEN l=tupleOfIdentifiers type=relationType t=term
 +        { $clauses.add(new TokenRelation(l, type, t)); }
 +    | name=cident K_IN marker=inMarker
 +        { $clauses.add(new SingleColumnRelation(name, Operator.IN, marker)); }
 +    | name=cident K_IN inValues=singleColumnInValues
 +        { $clauses.add(SingleColumnRelation.createInRelation($name.id, inValues)); }
 +    | name=cident K_CONTAINS { Operator rt = Operator.CONTAINS; } (K_KEY { rt = Operator.CONTAINS_KEY; })?
 +        t=term { $clauses.add(new SingleColumnRelation(name, rt, t)); }
 +    | name=cident '[' key=term ']' type=relationType t=term { $clauses.add(new SingleColumnRelation(name, key, type, t)); }
 +    | ids=tupleOfIdentifiers
 +      ( K_IN
 +          ( '(' ')'
 +              { $clauses.add(MultiColumnRelation.createInRelation(ids, new ArrayList<Tuples.Literal>())); }
 +          | tupleInMarker=inMarkerForTuple /* (a, b, c) IN ? */
 +              { $clauses.add(MultiColumnRelation.createSingleMarkerInRelation(ids, tupleInMarker)); }
 +          | literals=tupleOfTupleLiterals /* (a, b, c) IN ((1, 2, 3), (4, 5, 6), ...) */
 +              {
 +                  $clauses.add(MultiColumnRelation.createInRelation(ids, literals));
 +              }
 +          | markers=tupleOfMarkersForTuples /* (a, b, c) IN (?, ?, ...) */
 +              { $clauses.add(MultiColumnRelation.createInRelation(ids, markers)); }
 +          )
 +      | type=relationType literal=tupleLiteral /* (a, b, c) > (1, 2, 3) or (a, b, c) > (?, ?, ?) */
 +          {
 +              $clauses.add(MultiColumnRelation.createNonInRelation(ids, type, literal));
 +          }
 +      | type=relationType tupleMarker=markerForTuple /* (a, b, c) >= ? */
 +          { $clauses.add(MultiColumnRelation.createNonInRelation(ids, type, tupleMarker)); }
 +      )
 +    | '(' relation[$clauses] ')'
 +    ;
 +
 +inMarker returns [AbstractMarker.INRaw marker]
 +    : QMARK { $marker = newINBindVariables(null); }
 +    | ':' name=noncol_ident { $marker = newINBindVariables(name); }
 +    ;
 +
 +tupleOfIdentifiers returns [List<ColumnDefinition.Raw> ids]
 +    @init { $ids = new ArrayList<ColumnDefinition.Raw>(); }
 +    : '(' n1=cident { $ids.add(n1); } (',' ni=cident { $ids.add(ni); })* ')'
 +    ;
 +
 +singleColumnInValues returns [List<Term.Raw> terms]
 +    @init { $terms = new ArrayList<Term.Raw>(); }
 +    : '(' ( t1 = term { $terms.add(t1); } (',' ti=term { $terms.add(ti); })* )? ')'
 +    ;
 +
 +tupleOfTupleLiterals returns [List<Tuples.Literal> literals]
 +    @init { $literals = new ArrayList<>(); }
 +    : '(' t1=tupleLiteral { $literals.add(t1); } (',' ti=tupleLiteral { $literals.add(ti); })* ')'
 +    ;
 +
 +markerForTuple returns [Tuples.Raw marker]
 +    : QMARK { $marker = newTupleBindVariables(null); }
 +    | ':' name=noncol_ident { $marker = newTupleBindVariables(name); }
 +    ;
 +
 +tupleOfMarkersForTuples returns [List<Tuples.Raw> markers]
 +    @init { $markers = new ArrayList<Tuples.Raw>(); }
 +    : '(' m1=markerForTuple { $markers.add(m1); } (',' mi=markerForTuple { $markers.add(mi); })* ')'
 +    ;
 +
 +inMarkerForTuple returns [Tuples.INRaw marker]
 +    : QMARK { $marker = newTupleINBindVariables(null); }
 +    | ':' name=noncol_ident { $marker = newTupleINBindVariables(name); }
 +    ;
 +
 +comparatorType returns [CQL3Type.Raw t]
 +    : n=native_type     { $t = CQL3Type.Raw.from(n); }
 +    | c=collection_type { $t = c; }
 +    | tt=tuple_type     { $t = tt; }
 +    | id=userTypeName   { $t = CQL3Type.Raw.userType(id); }
 +    | K_FROZEN '<' f=comparatorType '>'
 +      {
 +        try {
 +            $t = CQL3Type.Raw.frozen(f);
 +        } catch (InvalidRequestException e) {
 +            addRecognitionError(e.getMessage());
 +        }
 +      }
 +    | s=STRING_LITERAL
 +      {
 +        try {
 +            $t = CQL3Type.Raw.from(new CQL3Type.Custom($s.text));
 +        } catch (SyntaxException e) {
 +            addRecognitionError("Cannot parse type " + $s.text + ": " + e.getMessage());
 +        } catch (ConfigurationException e) {
 +            addRecognitionError("Error setting type " + $s.text + ": " + e.getMessage());
 +        }
 +      }
 +    ;
 +
 +native_type returns [CQL3Type t]
 +    : K_ASCII     { $t = CQL3Type.Native.ASCII; }
 +    | K_BIGINT    { $t = CQL3Type.Native.BIGINT; }
 +    | K_BLOB      { $t = CQL3Type.Native.BLOB; }
 +    | K_BOOLEAN   { $t = CQL3Type.Native.BOOLEAN; }
 +    | K_COUNTER   { $t = CQL3Type.Native.COUNTER; }
 +    | K_DECIMAL   { $t = CQL3Type.Native.DECIMAL; }
 +    | K_DOUBLE    { $t = CQL3Type.Native.DOUBLE; }
 +    | K_DURATION    { $t = CQL3Type.Native.DURATION; }
 +    | K_FLOAT     { $t = CQL3Type.Native.FLOAT; }
 +    | K_INET      { $t = CQL3Type.Native.INET;}
 +    | K_INT       { $t = CQL3Type.Native.INT; }
 +    | K_SMALLINT  { $t = CQL3Type.Native.SMALLINT; }
 +    | K_TEXT      { $t = CQL3Type.Native.TEXT; }
 +    | K_TIMESTAMP { $t = CQL3Type.Native.TIMESTAMP; }
 +    | K_TINYINT   { $t = CQL3Type.Native.TINYINT; }
 +    | K_UUID      { $t = CQL3Type.Native.UUID; }
 +    | K_VARCHAR   { $t = CQL3Type.Native.VARCHAR; }
 +    | K_VARINT    { $t = CQL3Type.Native.VARINT; }
 +    | K_TIMEUUID  { $t = CQL3Type.Native.TIMEUUID; }
 +    | K_DATE      { $t = CQL3Type.Native.DATE; }
 +    | K_TIME      { $t = CQL3Type.Native.TIME; }
 +    ;
 +
 +collection_type returns [CQL3Type.Raw pt]
 +    : K_MAP  '<' t1=comparatorType ',' t2=comparatorType '>'
 +        {
 +            // if we can't parse either t1 or t2, antlr will "recover" and we may have t1 or t2 null.
 +            if (t1 != null && t2 != null)
 +                $pt = CQL3Type.Raw.map(t1, t2);
 +        }
 +    | K_LIST '<' t=comparatorType '>'
 +        { if (t != null) $pt = CQL3Type.Raw.list(t); }
 +    | K_SET  '<' t=comparatorType '>'
 +        { if (t != null) $pt = CQL3Type.Raw.set(t); }
 +    ;
 +
 +tuple_type returns [CQL3Type.Raw t]
 +    : K_TUPLE '<' { List<CQL3Type.Raw> types = new ArrayList<>(); }
 +         t1=comparatorType { types.add(t1); } (',' tn=comparatorType { types.add(tn); })*
 +      '>' { $t = CQL3Type.Raw.tuple(types); }
 +    ;
 +
 +username
 +    : IDENT
 +    | STRING_LITERAL
 +    | QUOTED_NAME { addRecognitionError("Quoted strings are are not supported for user names and USER is deprecated, please use ROLE");}
 +    ;
 +
 +mbean
 +    : STRING_LITERAL
 +    ;
 +
 +// Basically the same as cident, but we need to exlude existing CQL3 types
 +// (which for some reason are not reserved otherwise)
 +non_type_ident returns [ColumnIdentifier id]
 +    : t=IDENT                    { if (reservedTypeNames.contains($t.text)) addRecognitionError("Invalid (reserved) user type name " + $t.text); $id = new ColumnIdentifier($t.text, false); }
 +    | t=QUOTED_NAME              { $id = new ColumnIdentifier($t.text, true); }
 +    | k=basic_unreserved_keyword { $id = new ColumnIdentifier(k, false); }
 +    | kk=K_KEY                   { $id = new ColumnIdentifier($kk.text, false); }
 +    ;
 +
 +unreserved_keyword returns [String str]
 +    : u=unreserved_function_keyword     { $str = u; }
 +    | k=(K_TTL | K_COUNT | K_WRITETIME | K_KEY | K_CAST | K_JSON | K_DISTINCT) { $str = $k.text; }
 +    ;
 +
 +unreserved_function_keyword returns [String str]
 +    : u=basic_unreserved_keyword { $str = u; }
 +    | t=native_type              { $str = t.toString(); }
 +    ;
 +
 +basic_unreserved_keyword returns [String str]
 +    : k=( K_KEYS
 +        | K_AS
 +        | K_CLUSTERING
 +        | K_COMPACT
 +        | K_STORAGE
 +        | K_TYPE
 +        | K_VALUES
 +        | K_MAP
 +        | K_LIST
 +        | K_FILTERING
 +        | K_PERMISSION
 +        | K_PERMISSIONS
 +        | K_KEYSPACES
 +        | K_ALL
 +        | K_USER
 +        | K_USERS
 +        | K_ROLE
 +        | K_ROLES
 +        | K_SUPERUSER
 +        | K_NOSUPERUSER
 +        | K_LOGIN
 +        | K_NOLOGIN
 +        | K_OPTIONS
 +        | K_PASSWORD
 +        | K_EXISTS
 +        | K_CUSTOM
 +        | K_TRIGGER
 +        | K_CONTAINS
 +        | K_STATIC
 +        | K_FROZEN
 +        | K_TUPLE
 +        | K_FUNCTION
 +        | K_FUNCTIONS
 +        | K_AGGREGATE
 +        | K_SFUNC
 +        | K_STYPE
 +        | K_FINALFUNC
 +        | K_INITCOND
 +        | K_RETURNS
 +        | K_LANGUAGE
 +        | K_CALLED
 +        | K_INPUT
 +        | K_LIKE
 +        | K_PER
 +        | K_PARTITION
 +        | K_GROUP
 +        ) { $str = $k.text; }
 +    ;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d429cd0/src/java/org/apache/cassandra/auth/CassandraRoleManager.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d429cd0/src/java/org/apache/cassandra/config/CFMetaData.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d429cd0/src/java/org/apache/cassandra/cql3/QueryProcessor.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/QueryProcessor.java
index 0e0ba3c,ddee6c7..ef0ff12
--- a/src/java/org/apache/cassandra/cql3/QueryProcessor.java
+++ b/src/java/org/apache/cassandra/cql3/QueryProcessor.java
@@@ -561,25 -517,9 +561,25 @@@ public class QueryProcessor implements 
              ((CFStatement)statement).prepareKeyspace(clientState);
  
          Tracing.trace("Preparing statement");
-         return statement.prepare();
+         return statement.prepare(clientState);
      }
  
 +    public static <T extends ParsedStatement> T parseStatement(String queryStr, Class<T> klass, String type) throws SyntaxException
 +    {
 +        try
 +        {
 +            ParsedStatement stmt = parseStatement(queryStr);
 +
 +            if (!klass.isAssignableFrom(stmt.getClass()))
 +                throw new IllegalArgumentException("Invalid query, must be a " + type + " statement but was: " + stmt.getClass());
 +
 +            return klass.cast(stmt);
 +        }
 +        catch (RequestValidationException e)
 +        {
 +            throw new IllegalArgumentException(e.getMessage(), e);
 +        }
 +    }
      public static ParsedStatement parseStatement(String queryStr) throws SyntaxException
      {
          try


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


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

Posted by if...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/07fbd8ee/test/unit/org/apache/cassandra/cql3/validation/operations/SelectGroupByTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/SelectGroupByTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/SelectGroupByTest.java
index b41b81f..c941070 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/SelectGroupByTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/SelectGroupByTest.java
@@ -26,487 +26,475 @@ public class SelectGroupByTest extends CQLTester
     @Test
     public void testGroupByWithoutPaging() throws Throwable
     {
-        for (String compactOption : new String[] { "", " WITH COMPACT STORAGE" })
-        {
-            createTable("CREATE TABLE %s (a int, b int, c int, d int, e int, primary key (a, b, c, d))"
-                    + compactOption);
-
-            execute("INSERT INTO %s (a, b, c, d, e) VALUES (1, 2, 1, 3, 6)");
-            execute("INSERT INTO %s (a, b, c, d, e) VALUES (1, 2, 2, 6, 12)");
-            execute("INSERT INTO %s (a, b, c, d, e) VALUES (1, 3, 2, 12, 24)");
-            execute("INSERT INTO %s (a, b, c, d, e) VALUES (1, 4, 2, 12, 24)");
-            execute("INSERT INTO %s (a, b, c, d, e) VALUES (1, 4, 2, 6, 12)");
-            execute("INSERT INTO %s (a, b, c, d, e) VALUES (2, 2, 3, 3, 6)");
-            execute("INSERT INTO %s (a, b, c, d, e) VALUES (2, 4, 3, 6, 12)");
-            execute("INSERT INTO %s (a, b, c, d, e) VALUES (3, 3, 2, 12, 24)");
-            execute("INSERT INTO %s (a, b, c, d, e) VALUES (4, 8, 2, 12, 24)");
-
-            // Makes sure that we have some tombstones
-            execute("DELETE FROM %s WHERE a = 1 AND b = 3 AND c = 2 AND d = 12");
-            execute("DELETE FROM %s WHERE a = 3");
+        createTable("CREATE TABLE %s (a int, b int, c int, d int, e int, primary key (a, b, c, d))");
+
+        execute("INSERT INTO %s (a, b, c, d, e) VALUES (1, 2, 1, 3, 6)");
+        execute("INSERT INTO %s (a, b, c, d, e) VALUES (1, 2, 2, 6, 12)");
+        execute("INSERT INTO %s (a, b, c, d, e) VALUES (1, 3, 2, 12, 24)");
+        execute("INSERT INTO %s (a, b, c, d, e) VALUES (1, 4, 2, 12, 24)");
+        execute("INSERT INTO %s (a, b, c, d, e) VALUES (1, 4, 2, 6, 12)");
+        execute("INSERT INTO %s (a, b, c, d, e) VALUES (2, 2, 3, 3, 6)");
+        execute("INSERT INTO %s (a, b, c, d, e) VALUES (2, 4, 3, 6, 12)");
+        execute("INSERT INTO %s (a, b, c, d, e) VALUES (3, 3, 2, 12, 24)");
+        execute("INSERT INTO %s (a, b, c, d, e) VALUES (4, 8, 2, 12, 24)");
 
-            // Range queries
-            assertRows(execute("SELECT a, b, e, count(b), max(e) FROM %s GROUP BY a"),
-                       row(1, 2, 6, 4L, 24),
-                       row(2, 2, 6, 2L, 12),
-                       row(4, 8, 24, 1L, 24));
+        // Makes sure that we have some tombstones
+        execute("DELETE FROM %s WHERE a = 1 AND b = 3 AND c = 2 AND d = 12");
+        execute("DELETE FROM %s WHERE a = 3");
 
-            assertRows(execute("SELECT a, b, e, count(b), max(e) FROM %s GROUP BY a, b"),
-                       row(1, 2, 6, 2L, 12),
-                       row(1, 4, 12, 2L, 24),
-                       row(2, 2, 6, 1L, 6),
-                       row(2, 4, 12, 1L, 12),
-                       row(4, 8, 24, 1L, 24));
+        // Range queries
+        assertRows(execute("SELECT a, b, e, count(b), max(e) FROM %s GROUP BY a"),
+                   row(1, 2, 6, 4L, 24),
+                   row(2, 2, 6, 2L, 12),
+                   row(4, 8, 24, 1L, 24));
 
-            assertRows(execute("SELECT a, b, e, count(b), max(e) FROM %s WHERE b = 2 GROUP BY a, b ALLOW FILTERING"),
-                       row(1, 2, 6, 2L, 12),
-                       row(2, 2, 6, 1L, 6));
+        assertRows(execute("SELECT a, b, e, count(b), max(e) FROM %s GROUP BY a, b"),
+                   row(1, 2, 6, 2L, 12),
+                   row(1, 4, 12, 2L, 24),
+                   row(2, 2, 6, 1L, 6),
+                   row(2, 4, 12, 1L, 12),
+                   row(4, 8, 24, 1L, 24));
 
-            assertEmpty(execute("SELECT a, b, e, count(b), max(e) FROM %s WHERE b IN () GROUP BY a, b ALLOW FILTERING"));
+        assertRows(execute("SELECT a, b, e, count(b), max(e) FROM %s WHERE b = 2 GROUP BY a, b ALLOW FILTERING"),
+                   row(1, 2, 6, 2L, 12),
+                   row(2, 2, 6, 1L, 6));
 
-            // Range queries without aggregates
-            assertRows(execute("SELECT a, b, c, d FROM %s GROUP BY a, b, c"),
-                       row(1, 2, 1, 3),
-                       row(1, 2, 2, 6),
-                       row(1, 4, 2, 6),
-                       row(2, 2, 3, 3),
-                       row(2, 4, 3, 6),
-                       row(4, 8, 2, 12));
-
-            assertRows(execute("SELECT a, b, c, d FROM %s GROUP BY a, b"),
-                       row(1, 2, 1, 3),
-                       row(1, 4, 2, 6),
-                       row(2, 2, 3, 3),
-                       row(2, 4, 3, 6),
-                       row(4, 8, 2, 12));
+        assertEmpty(execute("SELECT a, b, e, count(b), max(e) FROM %s WHERE b IN () GROUP BY a, b ALLOW FILTERING"));
 
-            // Range query with LIMIT
-            assertRows(execute("SELECT a, b, e, count(b), max(e) FROM %s GROUP BY a, b LIMIT 2"),
-                       row(1, 2, 6, 2L, 12),
-                       row(1, 4, 12, 2L, 24));
+        // Range queries without aggregates
+        assertRows(execute("SELECT a, b, c, d FROM %s GROUP BY a, b, c"),
+                   row(1, 2, 1, 3),
+                   row(1, 2, 2, 6),
+                   row(1, 4, 2, 6),
+                   row(2, 2, 3, 3),
+                   row(2, 4, 3, 6),
+                   row(4, 8, 2, 12));
+
+        assertRows(execute("SELECT a, b, c, d FROM %s GROUP BY a, b"),
+                   row(1, 2, 1, 3),
+                   row(1, 4, 2, 6),
+                   row(2, 2, 3, 3),
+                   row(2, 4, 3, 6),
+                   row(4, 8, 2, 12));
 
-            // Range queries with PER PARTITION LIMIT
-            assertRows(execute("SELECT a, b, e, count(b), max(e) FROM %s GROUP BY a, b PER PARTITION LIMIT 1"),
-                       row(1, 2, 6, 2L, 12),
-                       row(2, 2, 6, 1L, 6),
-                       row(4, 8, 24, 1L, 24));
+        // Range query with LIMIT
+        assertRows(execute("SELECT a, b, e, count(b), max(e) FROM %s GROUP BY a, b LIMIT 2"),
+                   row(1, 2, 6, 2L, 12),
+                   row(1, 4, 12, 2L, 24));
 
-            assertRows(execute("SELECT a, b, e, count(b), max(e) FROM %s GROUP BY a PER PARTITION LIMIT 2"),
-                       row(1, 2, 6, 4L, 24),
-                       row(2, 2, 6, 2L, 12),
-                       row(4, 8, 24, 1L, 24));
+        // Range queries with PER PARTITION LIMIT
+        assertRows(execute("SELECT a, b, e, count(b), max(e) FROM %s GROUP BY a, b PER PARTITION LIMIT 1"),
+                   row(1, 2, 6, 2L, 12),
+                   row(2, 2, 6, 1L, 6),
+                   row(4, 8, 24, 1L, 24));
 
-            // Range query with PER PARTITION LIMIT and LIMIT
-            assertRows(execute("SELECT a, b, e, count(b), max(e) FROM %s GROUP BY a, b PER PARTITION LIMIT 1 LIMIT 2"),
-                       row(1, 2, 6, 2L, 12),
-                       row(2, 2, 6, 1L, 6));
+        assertRows(execute("SELECT a, b, e, count(b), max(e) FROM %s GROUP BY a PER PARTITION LIMIT 2"),
+                   row(1, 2, 6, 4L, 24),
+                   row(2, 2, 6, 2L, 12),
+                   row(4, 8, 24, 1L, 24));
 
-            assertRows(execute("SELECT a, b, e, count(b), max(e) FROM %s GROUP BY a PER PARTITION LIMIT 2"),
-                       row(1, 2, 6, 4L, 24),
-                       row(2, 2, 6, 2L, 12),
-                       row(4, 8, 24, 1L, 24));
+        // Range query with PER PARTITION LIMIT and LIMIT
+        assertRows(execute("SELECT a, b, e, count(b), max(e) FROM %s GROUP BY a, b PER PARTITION LIMIT 1 LIMIT 2"),
+                   row(1, 2, 6, 2L, 12),
+                   row(2, 2, 6, 1L, 6));
 
-            // Range queries without aggregates and with LIMIT
-            assertRows(execute("SELECT a, b, c, d FROM %s GROUP BY a, b, c LIMIT 3"),
-                       row(1, 2, 1, 3),
-                       row(1, 2, 2, 6),
-                       row(1, 4, 2, 6));
-
-            assertRows(execute("SELECT a, b, c, d FROM %s GROUP BY a, b LIMIT 3"),
-                       row(1, 2, 1, 3),
-                       row(1, 4, 2, 6),
-                       row(2, 2, 3, 3));
-
-            // Range queries without aggregates and with PER PARTITION LIMIT
-            assertRows(execute("SELECT a, b, c, d FROM %s GROUP BY a, b, c PER PARTITION LIMIT 2"),
-                       row(1, 2, 1, 3),
-                       row(1, 2, 2, 6),
-                       row(2, 2, 3, 3),
-                       row(2, 4, 3, 6),
-                       row(4, 8, 2, 12));
-
-            assertRows(execute("SELECT a, b, c, d FROM %s GROUP BY a, b PER PARTITION LIMIT 1"),
-                       row(1, 2, 1, 3),
-                       row(2, 2, 3, 3),
-                       row(4, 8, 2, 12));
-
-            // Range queries without aggregates, with PER PARTITION LIMIT and LIMIT
-            assertRows(execute("SELECT a, b, c, d FROM %s GROUP BY a, b, c PER PARTITION LIMIT 2 LIMIT 3"),
-                       row(1, 2, 1, 3),
-                       row(1, 2, 2, 6),
-                       row(2, 2, 3, 3));
+        assertRows(execute("SELECT a, b, e, count(b), max(e) FROM %s GROUP BY a PER PARTITION LIMIT 2"),
+                   row(1, 2, 6, 4L, 24),
+                   row(2, 2, 6, 2L, 12),
+                   row(4, 8, 24, 1L, 24));
 
-            // Range query with DISTINCT
-            assertRows(execute("SELECT DISTINCT a, count(a)FROM %s GROUP BY a"),
-                       row(1, 1L),
-                       row(2, 1L),
-                       row(4, 1L));
+        // Range queries without aggregates and with LIMIT
+        assertRows(execute("SELECT a, b, c, d FROM %s GROUP BY a, b, c LIMIT 3"),
+                   row(1, 2, 1, 3),
+                   row(1, 2, 2, 6),
+                   row(1, 4, 2, 6));
 
-            assertInvalidMessage("Grouping on clustering columns is not allowed for SELECT DISTINCT queries",
-                                 "SELECT DISTINCT a, count(a)FROM %s GROUP BY a, b");
+        assertRows(execute("SELECT a, b, c, d FROM %s GROUP BY a, b LIMIT 3"),
+                   row(1, 2, 1, 3),
+                   row(1, 4, 2, 6),
+                   row(2, 2, 3, 3));
 
-            // Range query with DISTINCT and LIMIT
-            assertRows(execute("SELECT DISTINCT a, count(a)FROM %s GROUP BY a LIMIT 2"),
-                       row(1, 1L),
-                       row(2, 1L));
+        // Range queries without aggregates and with PER PARTITION LIMIT
+        assertRows(execute("SELECT a, b, c, d FROM %s GROUP BY a, b, c PER PARTITION LIMIT 2"),
+                   row(1, 2, 1, 3),
+                   row(1, 2, 2, 6),
+                   row(2, 2, 3, 3),
+                   row(2, 4, 3, 6),
+                   row(4, 8, 2, 12));
+
+        assertRows(execute("SELECT a, b, c, d FROM %s GROUP BY a, b PER PARTITION LIMIT 1"),
+                   row(1, 2, 1, 3),
+                   row(2, 2, 3, 3),
+                   row(4, 8, 2, 12));
+
+        // Range queries without aggregates, with PER PARTITION LIMIT and LIMIT
+        assertRows(execute("SELECT a, b, c, d FROM %s GROUP BY a, b, c PER PARTITION LIMIT 2 LIMIT 3"),
+                   row(1, 2, 1, 3),
+                   row(1, 2, 2, 6),
+                   row(2, 2, 3, 3));
 
-            assertInvalidMessage("Grouping on clustering columns is not allowed for SELECT DISTINCT queries",
-                                 "SELECT DISTINCT a, count(a)FROM %s GROUP BY a, b LIMIT 2");
+        // Range query with DISTINCT
+        assertRows(execute("SELECT DISTINCT a, count(a)FROM %s GROUP BY a"),
+                   row(1, 1L),
+                   row(2, 1L),
+                   row(4, 1L));
 
-            // Range query with ORDER BY
-            assertInvalidMessage("ORDER BY is only supported when the partition key is restricted by an EQ or an IN",
-                                 "SELECT a, b, c, count(b), max(e) FROM %s GROUP BY a, b ORDER BY b DESC, c DESC");
+        assertInvalidMessage("Grouping on clustering columns is not allowed for SELECT DISTINCT queries",
+                             "SELECT DISTINCT a, count(a)FROM %s GROUP BY a, b");
 
-            // Single partition queries
-            assertRows(execute("SELECT a, b, e, count(b), max(e) FROM %s WHERE a = 1 GROUP BY a, b, c"),
-                       row(1, 2, 6, 1L, 6),
-                       row(1, 2, 12, 1L, 12),
-                       row(1, 4, 12, 2L, 24));
+        // Range query with DISTINCT and LIMIT
+        assertRows(execute("SELECT DISTINCT a, count(a)FROM %s GROUP BY a LIMIT 2"),
+                   row(1, 1L),
+                   row(2, 1L));
 
-            assertRows(execute("SELECT a, b, e, count(b), max(e) FROM %s WHERE a = 1 GROUP BY b, c"),
-                       row(1, 2, 6, 1L, 6),
-                       row(1, 2, 12, 1L, 12),
-                       row(1, 4, 12, 2L, 24));
+        assertInvalidMessage("Grouping on clustering columns is not allowed for SELECT DISTINCT queries",
+                             "SELECT DISTINCT a, count(a)FROM %s GROUP BY a, b LIMIT 2");
 
-            assertRows(execute("SELECT a, b, e, count(b), max(e) FROM %s WHERE a = 1 AND b = 2 GROUP BY a, b, c"),
-                       row(1, 2, 6, 1L, 6),
-                       row(1, 2, 12, 1L, 12));
+        // Range query with ORDER BY
+        assertInvalidMessage("ORDER BY is only supported when the partition key is restricted by an EQ or an IN",
+                             "SELECT a, b, c, count(b), max(e) FROM %s GROUP BY a, b ORDER BY b DESC, c DESC");
 
-            assertRows(execute("SELECT a, b, e, count(b), max(e) FROM %s WHERE a = 1 AND b = 2 GROUP BY a, c"),
-                       row(1, 2, 6, 1L, 6),
-                       row(1, 2, 12, 1L, 12));
+        // Single partition queries
+        assertRows(execute("SELECT a, b, e, count(b), max(e) FROM %s WHERE a = 1 GROUP BY a, b, c"),
+                   row(1, 2, 6, 1L, 6),
+                   row(1, 2, 12, 1L, 12),
+                   row(1, 4, 12, 2L, 24));
 
-            assertRows(execute("SELECT a, b, e, count(b), max(e) FROM %s WHERE a = 1 AND b = 2 GROUP BY c"),
-                       row(1, 2, 6, 1L, 6),
-                       row(1, 2, 12, 1L, 12));
+        assertRows(execute("SELECT a, b, e, count(b), max(e) FROM %s WHERE a = 1 GROUP BY b, c"),
+                   row(1, 2, 6, 1L, 6),
+                   row(1, 2, 12, 1L, 12),
+                   row(1, 4, 12, 2L, 24));
 
-            // Single partition queries without aggregates
-            assertRows(execute("SELECT a, b, c, d FROM %s WHERE a = 1 GROUP BY a, b"),
-                       row(1, 2, 1, 3),
-                       row(1, 4, 2, 6));
+        assertRows(execute("SELECT a, b, e, count(b), max(e) FROM %s WHERE a = 1 AND b = 2 GROUP BY a, b, c"),
+                   row(1, 2, 6, 1L, 6),
+                   row(1, 2, 12, 1L, 12));
 
-            assertRows(execute("SELECT a, b, c, d FROM %s WHERE a = 1 GROUP BY a, b, c"),
-                       row(1, 2, 1, 3),
-                       row(1, 2, 2, 6),
-                       row(1, 4, 2, 6));
+        assertRows(execute("SELECT a, b, e, count(b), max(e) FROM %s WHERE a = 1 AND b = 2 GROUP BY a, c"),
+                   row(1, 2, 6, 1L, 6),
+                   row(1, 2, 12, 1L, 12));
 
-            assertRows(execute("SELECT a, b, c, d FROM %s WHERE a = 1 GROUP BY b, c"),
-                       row(1, 2, 1, 3),
-                       row(1, 2, 2, 6),
-                       row(1, 4, 2, 6));
+        assertRows(execute("SELECT a, b, e, count(b), max(e) FROM %s WHERE a = 1 AND b = 2 GROUP BY c"),
+                   row(1, 2, 6, 1L, 6),
+                   row(1, 2, 12, 1L, 12));
 
-            assertRows(execute("SELECT a, b, c, d FROM %s WHERE a = 1 and token(a) = token(1) GROUP BY b, c"),
-                       row(1, 2, 1, 3),
-                       row(1, 2, 2, 6),
-                       row(1, 4, 2, 6));
+        // Single partition queries without aggregates
+        assertRows(execute("SELECT a, b, c, d FROM %s WHERE a = 1 GROUP BY a, b"),
+                   row(1, 2, 1, 3),
+                   row(1, 4, 2, 6));
 
-            // Single partition queries with DISTINCT
-            assertRows(execute("SELECT DISTINCT a, count(a)FROM %s WHERE a = 1 GROUP BY a"),
-                       row(1, 1L));
+        assertRows(execute("SELECT a, b, c, d FROM %s WHERE a = 1 GROUP BY a, b, c"),
+                   row(1, 2, 1, 3),
+                   row(1, 2, 2, 6),
+                   row(1, 4, 2, 6));
 
-            assertInvalidMessage("Grouping on clustering columns is not allowed for SELECT DISTINCT queries",
-                                 "SELECT DISTINCT a, count(a)FROM %s WHERE a = 1 GROUP BY a, b");
+        assertRows(execute("SELECT a, b, c, d FROM %s WHERE a = 1 GROUP BY b, c"),
+                   row(1, 2, 1, 3),
+                   row(1, 2, 2, 6),
+                   row(1, 4, 2, 6));
 
-            // Single partition queries with LIMIT
-            assertRows(execute("SELECT a, b, e, count(b), max(e) FROM %s WHERE a = 1 GROUP BY a, b, c LIMIT 10"),
-                       row(1, 2, 6, 1L, 6),
-                       row(1, 2, 12, 1L, 12),
-                       row(1, 4, 12, 2L, 24));
+        assertRows(execute("SELECT a, b, c, d FROM %s WHERE a = 1 and token(a) = token(1) GROUP BY b, c"),
+                   row(1, 2, 1, 3),
+                   row(1, 2, 2, 6),
+                   row(1, 4, 2, 6));
 
-            assertRows(execute("SELECT a, b, e, count(b), max(e) FROM %s WHERE a = 1 GROUP BY a, b, c LIMIT 2"),
-                       row(1, 2, 6, 1L, 6),
-                       row(1, 2, 12, 1L, 12));
+        // Single partition queries with DISTINCT
+        assertRows(execute("SELECT DISTINCT a, count(a)FROM %s WHERE a = 1 GROUP BY a"),
+                   row(1, 1L));
 
-            assertRows(execute("SELECT count(b), max(e) FROM %s WHERE a = 1 GROUP BY a, b, c LIMIT 1"),
-                       row(1L, 6));
+        assertInvalidMessage("Grouping on clustering columns is not allowed for SELECT DISTINCT queries",
+                             "SELECT DISTINCT a, count(a)FROM %s WHERE a = 1 GROUP BY a, b");
 
-            // Single partition queries with PER PARTITION LIMIT
-            assertRows(execute("SELECT a, b, e, count(b), max(e) FROM %s WHERE a = 1 GROUP BY a, b, c PER PARTITION LIMIT 10"),
-                       row(1, 2, 6, 1L, 6),
-                       row(1, 2, 12, 1L, 12),
-                       row(1, 4, 12, 2L, 24));
+        // Single partition queries with LIMIT
+        assertRows(execute("SELECT a, b, e, count(b), max(e) FROM %s WHERE a = 1 GROUP BY a, b, c LIMIT 10"),
+                   row(1, 2, 6, 1L, 6),
+                   row(1, 2, 12, 1L, 12),
+                   row(1, 4, 12, 2L, 24));
 
-            assertRows(execute("SELECT a, b, e, count(b), max(e) FROM %s WHERE a = 1 GROUP BY a, b, c PER PARTITION LIMIT 2"),
-                       row(1, 2, 6, 1L, 6),
-                       row(1, 2, 12, 1L, 12));
+        assertRows(execute("SELECT a, b, e, count(b), max(e) FROM %s WHERE a = 1 GROUP BY a, b, c LIMIT 2"),
+                   row(1, 2, 6, 1L, 6),
+                   row(1, 2, 12, 1L, 12));
 
-            assertRows(execute("SELECT count(b), max(e) FROM %s WHERE a = 1 GROUP BY a, b, c PER PARTITION LIMIT 1"),
-                       row(1L, 6));
+        assertRows(execute("SELECT count(b), max(e) FROM %s WHERE a = 1 GROUP BY a, b, c LIMIT 1"),
+                   row(1L, 6));
 
-            // Single partition queries without aggregates and with LIMIT
-            assertRows(execute("SELECT a, b, c, d FROM %s WHERE a = 1 GROUP BY a, b LIMIT 2"),
-                       row(1, 2, 1, 3),
-                       row(1, 4, 2, 6));
+        // Single partition queries with PER PARTITION LIMIT
+        assertRows(execute("SELECT a, b, e, count(b), max(e) FROM %s WHERE a = 1 GROUP BY a, b, c PER PARTITION LIMIT 10"),
+                   row(1, 2, 6, 1L, 6),
+                   row(1, 2, 12, 1L, 12),
+                   row(1, 4, 12, 2L, 24));
 
-            assertRows(execute("SELECT a, b, c, d FROM %s WHERE a = 1 GROUP BY a, b LIMIT 1"),
-                       row(1, 2, 1, 3));
+        assertRows(execute("SELECT a, b, e, count(b), max(e) FROM %s WHERE a = 1 GROUP BY a, b, c PER PARTITION LIMIT 2"),
+                   row(1, 2, 6, 1L, 6),
+                   row(1, 2, 12, 1L, 12));
 
-            assertRows(execute("SELECT a, b, c, d FROM %s WHERE a = 1 GROUP BY a, b, c LIMIT 2"),
-                       row(1, 2, 1, 3),
-                       row(1, 2, 2, 6));
+        assertRows(execute("SELECT count(b), max(e) FROM %s WHERE a = 1 GROUP BY a, b, c PER PARTITION LIMIT 1"),
+                   row(1L, 6));
 
-            // Single partition queries without aggregates and with PER PARTITION LIMIT
-            assertRows(execute("SELECT a, b, c, d FROM %s WHERE a = 1 GROUP BY a, b PER PARTITION LIMIT 2"),
-                       row(1, 2, 1, 3),
-                       row(1, 4, 2, 6));
+        // Single partition queries without aggregates and with LIMIT
+        assertRows(execute("SELECT a, b, c, d FROM %s WHERE a = 1 GROUP BY a, b LIMIT 2"),
+                   row(1, 2, 1, 3),
+                   row(1, 4, 2, 6));
 
-            assertRows(execute("SELECT a, b, c, d FROM %s WHERE a = 1 GROUP BY a, b PER PARTITION LIMIT 1"),
-                       row(1, 2, 1, 3));
+        assertRows(execute("SELECT a, b, c, d FROM %s WHERE a = 1 GROUP BY a, b LIMIT 1"),
+                   row(1, 2, 1, 3));
 
-            assertRows(execute("SELECT a, b, c, d FROM %s WHERE a = 1 GROUP BY a, b, c PER PARTITION LIMIT 2"),
-                       row(1, 2, 1, 3),
-                       row(1, 2, 2, 6));
+        assertRows(execute("SELECT a, b, c, d FROM %s WHERE a = 1 GROUP BY a, b, c LIMIT 2"),
+                   row(1, 2, 1, 3),
+                   row(1, 2, 2, 6));
 
-            // Single partition queries with ORDER BY
-            assertRows(execute("SELECT a, b, e, count(b), max(e) FROM %s WHERE a = 1 GROUP BY a, b, c ORDER BY b DESC, c DESC"),
-                       row(1, 4, 24, 2L, 24),
-                       row(1, 2, 12, 1L, 12),
-                       row(1, 2, 6, 1L, 6));
+        // Single partition queries without aggregates and with PER PARTITION LIMIT
+        assertRows(execute("SELECT a, b, c, d FROM %s WHERE a = 1 GROUP BY a, b PER PARTITION LIMIT 2"),
+                   row(1, 2, 1, 3),
+                   row(1, 4, 2, 6));
 
-            // Single partition queries with ORDER BY and PER PARTITION LIMIT
-            assertRows(execute("SELECT a, b, e, count(b), max(e) FROM %s WHERE a = 1 GROUP BY a, b, c ORDER BY b DESC, c DESC PER PARTITION LIMIT 1"),
-                       row(1, 4, 24, 2L, 24));
+        assertRows(execute("SELECT a, b, c, d FROM %s WHERE a = 1 GROUP BY a, b PER PARTITION LIMIT 1"),
+                   row(1, 2, 1, 3));
 
-            // Single partition queries with ORDER BY and LIMIT
-            assertRows(execute("SELECT a, b, e, count(b), max(e) FROM %s WHERE a = 1 GROUP BY a, b, c ORDER BY b DESC, c DESC LIMIT 2"),
-                       row(1, 4, 24, 2L, 24),
-                       row(1, 2, 12, 1L, 12));
+        assertRows(execute("SELECT a, b, c, d FROM %s WHERE a = 1 GROUP BY a, b, c PER PARTITION LIMIT 2"),
+                   row(1, 2, 1, 3),
+                   row(1, 2, 2, 6));
 
-            // Multi-partitions queries
-            assertRows(execute("SELECT a, b, e, count(b), max(e) FROM %s WHERE a IN (1, 2, 4) GROUP BY a, b, c"),
-                       row(1, 2, 6, 1L, 6),
-                       row(1, 2, 12, 1L, 12),
-                       row(1, 4, 12, 2L, 24),
-                       row(2, 2, 6, 1L, 6),
-                       row(2, 4, 12, 1L, 12),
-                       row(4, 8, 24, 1L, 24));
-
-            assertRows(execute("SELECT a, b, e, count(b), max(e) FROM %s WHERE a IN (1, 2, 4) AND b = 2 GROUP BY a, b, c"),
-                       row(1, 2, 6, 1L, 6),
-                       row(1, 2, 12, 1L, 12),
-                       row(2, 2, 6, 1L, 6));
+        // Single partition queries with ORDER BY
+        assertRows(execute("SELECT a, b, e, count(b), max(e) FROM %s WHERE a = 1 GROUP BY a, b, c ORDER BY b DESC, c DESC"),
+                   row(1, 4, 24, 2L, 24),
+                   row(1, 2, 12, 1L, 12),
+                   row(1, 2, 6, 1L, 6));
 
-            // Multi-partitions queries without aggregates
-            assertRows(execute("SELECT a, b, c, d FROM %s WHERE a IN (1, 2, 4) GROUP BY a, b"),
-                       row(1, 2, 1, 3),
-                       row(1, 4, 2, 6),
-                       row(2, 2, 3, 3),
-                       row(2, 4, 3, 6),
-                       row(4, 8, 2, 12));
-
-            assertRows(execute("SELECT a, b, c, d FROM %s WHERE a IN (1, 2, 4) GROUP BY a, b, c"),
-                       row(1, 2, 1, 3),
-                       row(1, 2, 2, 6),
-                       row(1, 4, 2, 6),
-                       row(2, 2, 3, 3),
-                       row(2, 4, 3, 6),
-                       row(4, 8, 2, 12));
-
-            // Multi-partitions query with DISTINCT
-            assertRows(execute("SELECT DISTINCT a, count(a)FROM %s WHERE a IN (1, 2, 4) GROUP BY a"),
-                       row(1, 1L),
-                       row(2, 1L),
-                       row(4, 1L));
-
-            assertInvalidMessage("Grouping on clustering columns is not allowed for SELECT DISTINCT queries",
-                                 "SELECT DISTINCT a, count(a)FROM %s WHERE a IN (1, 2, 4) GROUP BY a, b");
+        // Single partition queries with ORDER BY and PER PARTITION LIMIT
+        assertRows(execute("SELECT a, b, e, count(b), max(e) FROM %s WHERE a = 1 GROUP BY a, b, c ORDER BY b DESC, c DESC PER PARTITION LIMIT 1"),
+                   row(1, 4, 24, 2L, 24));
 
-            // Multi-partitions query with DISTINCT and LIMIT
-            assertRows(execute("SELECT DISTINCT a, count(a)FROM %s WHERE a IN (1, 2, 4) GROUP BY a LIMIT 2"),
-                       row(1, 1L),
-                       row(2, 1L));
+        // Single partition queries with ORDER BY and LIMIT
+        assertRows(execute("SELECT a, b, e, count(b), max(e) FROM %s WHERE a = 1 GROUP BY a, b, c ORDER BY b DESC, c DESC LIMIT 2"),
+                   row(1, 4, 24, 2L, 24),
+                   row(1, 2, 12, 1L, 12));
 
-            // Multi-partitions queries with PER PARTITION LIMIT
-            assertRows(execute("SELECT a, b, e, count(b), max(e) FROM %s WHERE a IN (1, 2, 4) GROUP BY a, b, c PER PARTITION LIMIT 1"),
-                       row(1, 2, 6, 1L, 6),
-                       row(2, 2, 6, 1L, 6),
-                       row(4, 8, 24, 1L, 24));
-
-            assertRows(execute("SELECT a, b, e, count(b), max(e) FROM %s WHERE a IN (1, 2, 4) GROUP BY a, b, c PER PARTITION LIMIT 2"),
-                       row(1, 2, 6, 1L, 6),
-                       row(1, 2, 12, 1L, 12),
-                       row(2, 2, 6, 1L, 6),
-                       row(2, 4, 12, 1L, 12),
-                       row(4, 8, 24, 1L, 24));
-
-            // Multi-partitions queries with ORDER BY
-            assertRows(execute("SELECT a, b, c, count(b), max(e) FROM %s WHERE a IN (1, 2, 4) GROUP BY a, b ORDER BY b DESC, c DESC"),
-                       row(4, 8, 2, 1L, 24),
-                       row(2, 4, 3, 1L, 12),
-                       row(1, 4, 2, 2L, 24),
-                       row(2, 2, 3, 1L, 6),
-                       row(1, 2, 2, 2L, 12));
-
-            assertRows(execute("SELECT a, b, c, d FROM %s WHERE a IN (1, 2, 4) GROUP BY a, b, c ORDER BY b DESC, c DESC"),
-                       row(4, 8, 2, 12),
-                       row(2, 4, 3, 6),
-                       row(1, 4, 2, 12),
-                       row(2, 2, 3, 3),
-                       row(1, 2, 2, 6),
-                       row(1, 2, 1, 3));
-
-            // Multi-partitions queries with ORDER BY and LIMIT
-            assertRows(execute("SELECT a, b, c, d FROM %s WHERE a IN (1, 2, 4) GROUP BY a, b ORDER BY b DESC, c DESC LIMIT 3"),
-                       row(4, 8, 2, 12),
-                       row(2, 4, 3, 6),
-                       row(1, 4, 2, 12));
-
-            // Invalid queries
-            assertInvalidMessage("Group by is currently only supported on the columns of the PRIMARY KEY, got e",
-                                 "SELECT a, b, d, count(b), max(c) FROM %s WHERE a = 1 GROUP BY a, e");
-
-            assertInvalidMessage("Group by currently only support groups of columns following their declared order in the PRIMARY KEY",
-                                 "SELECT a, b, d, count(b), max(c) FROM %s WHERE a = 1 GROUP BY c");
-
-            assertInvalidMessage("Group by currently only support groups of columns following their declared order in the PRIMARY KEY",
-                                 "SELECT a, b, d, count(b), max(c) FROM %s WHERE a = 1 GROUP BY a, c, b");
-
-            assertInvalidMessage("Group by currently only support groups of columns following their declared order in the PRIMARY KEY",
-                                 "SELECT a, b, d, count(b), max(c) FROM %s WHERE a = 1 GROUP BY a, a");
-
-            assertInvalidMessage("Group by currently only support groups of columns following their declared order in the PRIMARY KEY",
-                                 "SELECT a, b, c, d FROM %s WHERE token(a) = token(1) GROUP BY b, c");
-
-            assertInvalidMessage("Undefined column name clustering1",
-                                 "SELECT a, b as clustering1, max(c) FROM %s WHERE a = 1 GROUP BY a, clustering1");
-
-            assertInvalidMessage("Undefined column name z",
-                                 "SELECT a, b, max(c) FROM %s WHERE a = 1 GROUP BY a, b, z");
-
-            // Test with composite partition key
-            createTable("CREATE TABLE %s (a int, b int, c int, d int, e int, primary key ((a, b), c, d))" + compactOption);
-
-            execute("INSERT INTO %s (a, b, c, d, e) VALUES (1, 1, 1, 3, 6)");
-            execute("INSERT INTO %s (a, b, c, d, e) VALUES (1, 1, 2, 6, 12)");
-            execute("INSERT INTO %s (a, b, c, d, e) VALUES (1, 1, 3, 12, 24)");
-            execute("INSERT INTO %s (a, b, c, d, e) VALUES (1, 2, 1, 12, 24)");
-            execute("INSERT INTO %s (a, b, c, d, e) VALUES (1, 2, 2, 6, 12)");
-
-            assertInvalidMessage("Group by is not supported on only a part of the partition key",
-                                 "SELECT a, b, max(d) FROM %s GROUP BY a");
-
-            assertRows(execute("SELECT a, b, max(d) FROM %s GROUP BY a, b"),
-                       row(1, 2, 12),
-                       row(1, 1, 12));
-
-            assertRows(execute("SELECT a, b, max(d) FROM %s WHERE a = 1 AND b = 1 GROUP BY b"),
-                       row(1, 1, 12));
-
-            // Test with table without clustering key
-            createTable("CREATE TABLE %s (a int primary key, b int, c int)" + compactOption);
-
-            execute("INSERT INTO %s (a, b, c) VALUES (1, 3, 6)");
-            execute("INSERT INTO %s (a, b, c) VALUES (2, 6, 12)");
-            execute("INSERT INTO %s (a, b, c) VALUES (3, 12, 24)");
-
-            assertInvalidMessage("Group by currently only support groups of columns following their declared order in the PRIMARY KEY",
-                    "SELECT a, max(c) FROM %s WHERE a = 1 GROUP BY a, a");
-        }
+        // Multi-partitions queries
+        assertRows(execute("SELECT a, b, e, count(b), max(e) FROM %s WHERE a IN (1, 2, 4) GROUP BY a, b, c"),
+                   row(1, 2, 6, 1L, 6),
+                   row(1, 2, 12, 1L, 12),
+                   row(1, 4, 12, 2L, 24),
+                   row(2, 2, 6, 1L, 6),
+                   row(2, 4, 12, 1L, 12),
+                   row(4, 8, 24, 1L, 24));
+
+        assertRows(execute("SELECT a, b, e, count(b), max(e) FROM %s WHERE a IN (1, 2, 4) AND b = 2 GROUP BY a, b, c"),
+                   row(1, 2, 6, 1L, 6),
+                   row(1, 2, 12, 1L, 12),
+                   row(2, 2, 6, 1L, 6));
+
+        // Multi-partitions queries without aggregates
+        assertRows(execute("SELECT a, b, c, d FROM %s WHERE a IN (1, 2, 4) GROUP BY a, b"),
+                   row(1, 2, 1, 3),
+                   row(1, 4, 2, 6),
+                   row(2, 2, 3, 3),
+                   row(2, 4, 3, 6),
+                   row(4, 8, 2, 12));
+
+        assertRows(execute("SELECT a, b, c, d FROM %s WHERE a IN (1, 2, 4) GROUP BY a, b, c"),
+                   row(1, 2, 1, 3),
+                   row(1, 2, 2, 6),
+                   row(1, 4, 2, 6),
+                   row(2, 2, 3, 3),
+                   row(2, 4, 3, 6),
+                   row(4, 8, 2, 12));
+
+        // Multi-partitions query with DISTINCT
+        assertRows(execute("SELECT DISTINCT a, count(a)FROM %s WHERE a IN (1, 2, 4) GROUP BY a"),
+                   row(1, 1L),
+                   row(2, 1L),
+                   row(4, 1L));
+
+        assertInvalidMessage("Grouping on clustering columns is not allowed for SELECT DISTINCT queries",
+                             "SELECT DISTINCT a, count(a)FROM %s WHERE a IN (1, 2, 4) GROUP BY a, b");
+
+        // Multi-partitions query with DISTINCT and LIMIT
+        assertRows(execute("SELECT DISTINCT a, count(a)FROM %s WHERE a IN (1, 2, 4) GROUP BY a LIMIT 2"),
+                   row(1, 1L),
+                   row(2, 1L));
+
+        // Multi-partitions queries with PER PARTITION LIMIT
+        assertRows(execute("SELECT a, b, e, count(b), max(e) FROM %s WHERE a IN (1, 2, 4) GROUP BY a, b, c PER PARTITION LIMIT 1"),
+                   row(1, 2, 6, 1L, 6),
+                   row(2, 2, 6, 1L, 6),
+                   row(4, 8, 24, 1L, 24));
+
+        assertRows(execute("SELECT a, b, e, count(b), max(e) FROM %s WHERE a IN (1, 2, 4) GROUP BY a, b, c PER PARTITION LIMIT 2"),
+                   row(1, 2, 6, 1L, 6),
+                   row(1, 2, 12, 1L, 12),
+                   row(2, 2, 6, 1L, 6),
+                   row(2, 4, 12, 1L, 12),
+                   row(4, 8, 24, 1L, 24));
+
+        // Multi-partitions queries with ORDER BY
+        assertRows(execute("SELECT a, b, c, count(b), max(e) FROM %s WHERE a IN (1, 2, 4) GROUP BY a, b ORDER BY b DESC, c DESC"),
+                   row(4, 8, 2, 1L, 24),
+                   row(2, 4, 3, 1L, 12),
+                   row(1, 4, 2, 2L, 24),
+                   row(2, 2, 3, 1L, 6),
+                   row(1, 2, 2, 2L, 12));
+
+        assertRows(execute("SELECT a, b, c, d FROM %s WHERE a IN (1, 2, 4) GROUP BY a, b, c ORDER BY b DESC, c DESC"),
+                   row(4, 8, 2, 12),
+                   row(2, 4, 3, 6),
+                   row(1, 4, 2, 12),
+                   row(2, 2, 3, 3),
+                   row(1, 2, 2, 6),
+                   row(1, 2, 1, 3));
+
+        // Multi-partitions queries with ORDER BY and LIMIT
+        assertRows(execute("SELECT a, b, c, d FROM %s WHERE a IN (1, 2, 4) GROUP BY a, b ORDER BY b DESC, c DESC LIMIT 3"),
+                   row(4, 8, 2, 12),
+                   row(2, 4, 3, 6),
+                   row(1, 4, 2, 12));
+
+        // Invalid queries
+        assertInvalidMessage("Group by is currently only supported on the columns of the PRIMARY KEY, got e",
+                             "SELECT a, b, d, count(b), max(c) FROM %s WHERE a = 1 GROUP BY a, e");
+
+        assertInvalidMessage("Group by currently only support groups of columns following their declared order in the PRIMARY KEY",
+                             "SELECT a, b, d, count(b), max(c) FROM %s WHERE a = 1 GROUP BY c");
+
+        assertInvalidMessage("Group by currently only support groups of columns following their declared order in the PRIMARY KEY",
+                             "SELECT a, b, d, count(b), max(c) FROM %s WHERE a = 1 GROUP BY a, c, b");
+
+        assertInvalidMessage("Group by currently only support groups of columns following their declared order in the PRIMARY KEY",
+                             "SELECT a, b, d, count(b), max(c) FROM %s WHERE a = 1 GROUP BY a, a");
+
+        assertInvalidMessage("Group by currently only support groups of columns following their declared order in the PRIMARY KEY",
+                             "SELECT a, b, c, d FROM %s WHERE token(a) = token(1) GROUP BY b, c");
+
+        assertInvalidMessage("Undefined column name clustering1",
+                             "SELECT a, b as clustering1, max(c) FROM %s WHERE a = 1 GROUP BY a, clustering1");
+
+        assertInvalidMessage("Undefined column name z",
+                             "SELECT a, b, max(c) FROM %s WHERE a = 1 GROUP BY a, b, z");
+
+        // Test with composite partition key
+        createTable("CREATE TABLE %s (a int, b int, c int, d int, e int, primary key ((a, b), c, d))");
+
+        execute("INSERT INTO %s (a, b, c, d, e) VALUES (1, 1, 1, 3, 6)");
+        execute("INSERT INTO %s (a, b, c, d, e) VALUES (1, 1, 2, 6, 12)");
+        execute("INSERT INTO %s (a, b, c, d, e) VALUES (1, 1, 3, 12, 24)");
+        execute("INSERT INTO %s (a, b, c, d, e) VALUES (1, 2, 1, 12, 24)");
+        execute("INSERT INTO %s (a, b, c, d, e) VALUES (1, 2, 2, 6, 12)");
+
+        assertInvalidMessage("Group by is not supported on only a part of the partition key",
+                             "SELECT a, b, max(d) FROM %s GROUP BY a");
+
+        assertRows(execute("SELECT a, b, max(d) FROM %s GROUP BY a, b"),
+                   row(1, 2, 12),
+                   row(1, 1, 12));
+
+        assertRows(execute("SELECT a, b, max(d) FROM %s WHERE a = 1 AND b = 1 GROUP BY b"),
+                   row(1, 1, 12));
+
+        // Test with table without clustering key
+        createTable("CREATE TABLE %s (a int primary key, b int, c int)");
+
+        execute("INSERT INTO %s (a, b, c) VALUES (1, 3, 6)");
+        execute("INSERT INTO %s (a, b, c) VALUES (2, 6, 12)");
+        execute("INSERT INTO %s (a, b, c) VALUES (3, 12, 24)");
+
+        assertInvalidMessage("Group by currently only support groups of columns following their declared order in the PRIMARY KEY",
+                             "SELECT a, max(c) FROM %s WHERE a = 1 GROUP BY a, a");
     }
 
     @Test
     public void testGroupByWithoutPagingWithDeletions() throws Throwable
     {
-        for (String compactOption : new String[] { "", " WITH COMPACT STORAGE" })
-        {
-            createTable("CREATE TABLE %s (a int, b int, c int, d int, e int, primary key (a, b, c, d))"
-                    + compactOption);
-
-            execute("INSERT INTO %s (a, b, c, d, e) VALUES (1, 2, 1, 3, 6)");
-            execute("INSERT INTO %s (a, b, c, d, e) VALUES (1, 2, 1, 6, 12)");
-            execute("INSERT INTO %s (a, b, c, d, e) VALUES (1, 2, 1, 9, 18)");
-            execute("INSERT INTO %s (a, b, c, d, e) VALUES (1, 2, 1, 12, 24)");
-            execute("INSERT INTO %s (a, b, c, d, e) VALUES (1, 2, 2, 3, 6)");
-            execute("INSERT INTO %s (a, b, c, d, e) VALUES (1, 2, 2, 6, 12)");
-            execute("INSERT INTO %s (a, b, c, d, e) VALUES (1, 2, 2, 9, 18)");
-            execute("INSERT INTO %s (a, b, c, d, e) VALUES (1, 2, 2, 12, 24)");
-            execute("INSERT INTO %s (a, b, c, d, e) VALUES (1, 2, 3, 3, 6)");
-            execute("INSERT INTO %s (a, b, c, d, e) VALUES (1, 2, 3, 6, 12)");
-            execute("INSERT INTO %s (a, b, c, d, e) VALUES (1, 2, 3, 9, 18)");
-            execute("INSERT INTO %s (a, b, c, d, e) VALUES (1, 2, 3, 12, 24)");
-
-            execute("DELETE FROM %s WHERE a = 1 AND b = 2 AND c = 1 AND d = 12");
-            execute("DELETE FROM %s WHERE a = 1 AND b = 2 AND c = 2 AND d = 9");
-
-            assertRows(execute("SELECT a, b, c, count(b), max(d) FROM %s GROUP BY a, b, c"),
-                       row(1, 2, 1, 3L, 9),
-                       row(1, 2, 2, 3L, 12),
-                       row(1, 2, 3, 4L, 12));
-        }
+        createTable("CREATE TABLE %s (a int, b int, c int, d int, e int, primary key (a, b, c, d))");
+
+        execute("INSERT INTO %s (a, b, c, d, e) VALUES (1, 2, 1, 3, 6)");
+        execute("INSERT INTO %s (a, b, c, d, e) VALUES (1, 2, 1, 6, 12)");
+        execute("INSERT INTO %s (a, b, c, d, e) VALUES (1, 2, 1, 9, 18)");
+        execute("INSERT INTO %s (a, b, c, d, e) VALUES (1, 2, 1, 12, 24)");
+        execute("INSERT INTO %s (a, b, c, d, e) VALUES (1, 2, 2, 3, 6)");
+        execute("INSERT INTO %s (a, b, c, d, e) VALUES (1, 2, 2, 6, 12)");
+        execute("INSERT INTO %s (a, b, c, d, e) VALUES (1, 2, 2, 9, 18)");
+        execute("INSERT INTO %s (a, b, c, d, e) VALUES (1, 2, 2, 12, 24)");
+        execute("INSERT INTO %s (a, b, c, d, e) VALUES (1, 2, 3, 3, 6)");
+        execute("INSERT INTO %s (a, b, c, d, e) VALUES (1, 2, 3, 6, 12)");
+        execute("INSERT INTO %s (a, b, c, d, e) VALUES (1, 2, 3, 9, 18)");
+        execute("INSERT INTO %s (a, b, c, d, e) VALUES (1, 2, 3, 12, 24)");
+
+        execute("DELETE FROM %s WHERE a = 1 AND b = 2 AND c = 1 AND d = 12");
+        execute("DELETE FROM %s WHERE a = 1 AND b = 2 AND c = 2 AND d = 9");
+
+        assertRows(execute("SELECT a, b, c, count(b), max(d) FROM %s GROUP BY a, b, c"),
+                   row(1, 2, 1, 3L, 9),
+                   row(1, 2, 2, 3L, 12),
+                   row(1, 2, 3, 4L, 12));
     }
 
     @Test
     public void testGroupByWithRangeNamesQueryWithoutPaging() throws Throwable
     {
-        for (String compactOption : new String[] { "", " WITH COMPACT STORAGE" })
-        {
-            createTable("CREATE TABLE %s (a int, b int, c int, d int, primary key (a, b, c))"
-                    + compactOption);
-
-            for (int i = 1; i < 5; i++)
-                for (int j = 1; j < 5; j++)
-                    for (int k = 1; k < 5; k++)
-                        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", i, j, k, i + j);
-
-            // Makes sure that we have some tombstones
-            execute("DELETE FROM %s WHERE a = 3");
-
-            // Range queries
-            assertRows(execute("SELECT a, b, d, count(b), max(d) FROM %s WHERE b = 1 and c IN (1, 2) GROUP BY a ALLOW FILTERING"),
-                       row(1, 1, 2, 2L, 2),
-                       row(2, 1, 3, 2L, 3),
-                       row(4, 1, 5, 2L, 5));
-
-            assertRows(execute("SELECT a, b, d, count(b), max(d) FROM %s WHERE b = 1 and c IN (1, 2) GROUP BY a, b ALLOW FILTERING"),
-                       row(1, 1, 2, 2L, 2),
-                       row(2, 1, 3, 2L, 3),
-                       row(4, 1, 5, 2L, 5));
-
-            assertRows(execute("SELECT a, b, d, count(b), max(d) FROM %s WHERE b IN (1, 2) and c IN (1, 2) GROUP BY a, b ALLOW FILTERING"),
-                       row(1, 1, 2, 2L, 2),
-                       row(1, 2, 3, 2L, 3),
-                       row(2, 1, 3, 2L, 3),
-                       row(2, 2, 4, 2L, 4),
-                       row(4, 1, 5, 2L, 5),
-                       row(4, 2, 6, 2L, 6));
-
-            // Range queries with LIMIT
-            assertRows(execute("SELECT a, b, d, count(b), max(d) FROM %s WHERE b = 1 and c IN (1, 2) GROUP BY a LIMIT 5 ALLOW FILTERING"),
-                       row(1, 1, 2, 2L, 2),
-                       row(2, 1, 3, 2L, 3),
-                       row(4, 1, 5, 2L, 5));
+        createTable("CREATE TABLE %s (a int, b int, c int, d int, primary key (a, b, c))");
 
-            assertRows(execute("SELECT a, b, d, count(b), max(d) FROM %s WHERE b = 1 and c IN (1, 2) GROUP BY a, b LIMIT 3 ALLOW FILTERING"),
-                       row(1, 1, 2, 2L, 2),
-                       row(2, 1, 3, 2L, 3),
-                       row(4, 1, 5, 2L, 5));
+        for (int i = 1; i < 5; i++)
+            for (int j = 1; j < 5; j++)
+                for (int k = 1; k < 5; k++)
+                    execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", i, j, k, i + j);
 
-            assertRows(execute("SELECT a, b, d, count(b), max(d) FROM %s WHERE b IN (1, 2) and c IN (1, 2) GROUP BY a, b LIMIT 3 ALLOW FILTERING"),
-                       row(1, 1, 2, 2L, 2),
-                       row(1, 2, 3, 2L, 3),
-                       row(2, 1, 3, 2L, 3));
-
-            // Range queries with PER PARTITION LIMIT
-            assertRows(execute("SELECT a, b, d, count(b), max(d) FROM %s WHERE b = 1 and c IN (1, 2) GROUP BY a, b PER PARTITION LIMIT 2 ALLOW FILTERING"),
-                       row(1, 1, 2, 2L, 2),
-                       row(2, 1, 3, 2L, 3),
-                       row(4, 1, 5, 2L, 5));
+        // Makes sure that we have some tombstones
+        execute("DELETE FROM %s WHERE a = 3");
 
-            assertRows(execute("SELECT a, b, d, count(b), max(d) FROM %s WHERE b IN (1, 2) and c IN (1, 2) GROUP BY a, b PER PARTITION LIMIT 1 ALLOW FILTERING"),
-                       row(1, 1, 2, 2L, 2),
-                       row(2, 1, 3, 2L, 3),
-                       row(4, 1, 5, 2L, 5));
+        // Range queries
+        assertRows(execute("SELECT a, b, d, count(b), max(d) FROM %s WHERE b = 1 and c IN (1, 2) GROUP BY a ALLOW FILTERING"),
+                   row(1, 1, 2, 2L, 2),
+                   row(2, 1, 3, 2L, 3),
+                   row(4, 1, 5, 2L, 5));
+
+        assertRows(execute("SELECT a, b, d, count(b), max(d) FROM %s WHERE b = 1 and c IN (1, 2) GROUP BY a, b ALLOW FILTERING"),
+                   row(1, 1, 2, 2L, 2),
+                   row(2, 1, 3, 2L, 3),
+                   row(4, 1, 5, 2L, 5));
+
+        assertRows(execute("SELECT a, b, d, count(b), max(d) FROM %s WHERE b IN (1, 2) and c IN (1, 2) GROUP BY a, b ALLOW FILTERING"),
+                   row(1, 1, 2, 2L, 2),
+                   row(1, 2, 3, 2L, 3),
+                   row(2, 1, 3, 2L, 3),
+                   row(2, 2, 4, 2L, 4),
+                   row(4, 1, 5, 2L, 5),
+                   row(4, 2, 6, 2L, 6));
+
+        // Range queries with LIMIT
+        assertRows(execute("SELECT a, b, d, count(b), max(d) FROM %s WHERE b = 1 and c IN (1, 2) GROUP BY a LIMIT 5 ALLOW FILTERING"),
+                   row(1, 1, 2, 2L, 2),
+                   row(2, 1, 3, 2L, 3),
+                   row(4, 1, 5, 2L, 5));
+
+        assertRows(execute("SELECT a, b, d, count(b), max(d) FROM %s WHERE b = 1 and c IN (1, 2) GROUP BY a, b LIMIT 3 ALLOW FILTERING"),
+                   row(1, 1, 2, 2L, 2),
+                   row(2, 1, 3, 2L, 3),
+                   row(4, 1, 5, 2L, 5));
+
+        assertRows(execute("SELECT a, b, d, count(b), max(d) FROM %s WHERE b IN (1, 2) and c IN (1, 2) GROUP BY a, b LIMIT 3 ALLOW FILTERING"),
+                   row(1, 1, 2, 2L, 2),
+                   row(1, 2, 3, 2L, 3),
+                   row(2, 1, 3, 2L, 3));
 
-            // Range queries with PER PARTITION LIMIT and LIMIT
-            assertRows(execute("SELECT a, b, d, count(b), max(d) FROM %s WHERE b = 1 and c IN (1, 2) GROUP BY a, b PER PARTITION LIMIT 2 LIMIT 5 ALLOW FILTERING"),
-                       row(1, 1, 2, 2L, 2),
-                       row(2, 1, 3, 2L, 3),
-                       row(4, 1, 5, 2L, 5));
-
-            assertRows(execute("SELECT a, b, d, count(b), max(d) FROM %s WHERE b IN (1, 2) and c IN (1, 2) GROUP BY a, b PER PARTITION LIMIT 1 LIMIT 2 ALLOW FILTERING"),
-                       row(1, 1, 2, 2L, 2),
-                       row(2, 1, 3, 2L, 3));
-        }
+        // Range queries with PER PARTITION LIMIT
+        assertRows(execute("SELECT a, b, d, count(b), max(d) FROM %s WHERE b = 1 and c IN (1, 2) GROUP BY a, b PER PARTITION LIMIT 2 ALLOW FILTERING"),
+                   row(1, 1, 2, 2L, 2),
+                   row(2, 1, 3, 2L, 3),
+                   row(4, 1, 5, 2L, 5));
+
+        assertRows(execute("SELECT a, b, d, count(b), max(d) FROM %s WHERE b IN (1, 2) and c IN (1, 2) GROUP BY a, b PER PARTITION LIMIT 1 ALLOW FILTERING"),
+                   row(1, 1, 2, 2L, 2),
+                   row(2, 1, 3, 2L, 3),
+                   row(4, 1, 5, 2L, 5));
+
+        // Range queries with PER PARTITION LIMIT and LIMIT
+        assertRows(execute("SELECT a, b, d, count(b), max(d) FROM %s WHERE b = 1 and c IN (1, 2) GROUP BY a, b PER PARTITION LIMIT 2 LIMIT 5 ALLOW FILTERING"),
+                   row(1, 1, 2, 2L, 2),
+                   row(2, 1, 3, 2L, 3),
+                   row(4, 1, 5, 2L, 5));
+
+        assertRows(execute("SELECT a, b, d, count(b), max(d) FROM %s WHERE b IN (1, 2) and c IN (1, 2) GROUP BY a, b PER PARTITION LIMIT 1 LIMIT 2 ALLOW FILTERING"),
+                   row(1, 1, 2, 2L, 2),
+                   row(2, 1, 3, 2L, 3));
     }
 
     @Test
@@ -895,403 +883,395 @@ public class SelectGroupByTest extends CQLTester
     @Test
     public void testGroupByWithPaging() throws Throwable
     {
-        for (String compactOption : new String[] { "", " WITH COMPACT STORAGE" })
+        createTable("CREATE TABLE %s (a int, b int, c int, d int, e int, primary key (a, b, c, d))");
+
+        execute("INSERT INTO %s (a, b, c, d, e) VALUES (1, 2, 1, 3, 6)");
+        execute("INSERT INTO %s (a, b, c, d, e) VALUES (1, 2, 2, 6, 12)");
+        execute("INSERT INTO %s (a, b, c, d, e) VALUES (1, 3, 2, 12, 24)");
+        execute("INSERT INTO %s (a, b, c, d, e) VALUES (1, 4, 2, 12, 24)");
+        execute("INSERT INTO %s (a, b, c, d, e) VALUES (1, 4, 2, 6, 12)");
+        execute("INSERT INTO %s (a, b, c, d, e) VALUES (2, 2, 3, 3, 6)");
+        execute("INSERT INTO %s (a, b, c, d, e) VALUES (2, 4, 3, 6, 12)");
+        execute("INSERT INTO %s (a, b, c, d, e) VALUES (3, 3, 2, 12, 24)");
+        execute("INSERT INTO %s (a, b, c, d, e) VALUES (4, 8, 2, 12, 24)");
+
+        // Makes sure that we have some tombstones
+        execute("DELETE FROM %s WHERE a = 1 AND b = 3 AND c = 2 AND d = 12");
+        execute("DELETE FROM %s WHERE a = 3");
+
+        for (int pageSize = 1; pageSize < 10; pageSize++)
         {
-            createTable("CREATE TABLE %s (a int, b int, c int, d int, e int, primary key (a, b, c, d))"
-                    + compactOption);
-
-            execute("INSERT INTO %s (a, b, c, d, e) VALUES (1, 2, 1, 3, 6)");
-            execute("INSERT INTO %s (a, b, c, d, e) VALUES (1, 2, 2, 6, 12)");
-            execute("INSERT INTO %s (a, b, c, d, e) VALUES (1, 3, 2, 12, 24)");
-            execute("INSERT INTO %s (a, b, c, d, e) VALUES (1, 4, 2, 12, 24)");
-            execute("INSERT INTO %s (a, b, c, d, e) VALUES (1, 4, 2, 6, 12)");
-            execute("INSERT INTO %s (a, b, c, d, e) VALUES (2, 2, 3, 3, 6)");
-            execute("INSERT INTO %s (a, b, c, d, e) VALUES (2, 4, 3, 6, 12)");
-            execute("INSERT INTO %s (a, b, c, d, e) VALUES (3, 3, 2, 12, 24)");
-            execute("INSERT INTO %s (a, b, c, d, e) VALUES (4, 8, 2, 12, 24)");
-
-            // Makes sure that we have some tombstones
-            execute("DELETE FROM %s WHERE a = 1 AND b = 3 AND c = 2 AND d = 12");
-            execute("DELETE FROM %s WHERE a = 3");
-
-            for (int pageSize = 1; pageSize < 10; pageSize++)
-            {
-                // Range queries
-                assertRowsNet(executeNetWithPaging("SELECT a, b, e, count(b), max(e) FROM %s GROUP BY a", pageSize),
-                              row(1, 2, 6, 4L, 24),
-                              row(2, 2, 6, 2L, 12),
-                              row(4, 8, 24, 1L, 24));
-
-                assertRowsNet(executeNetWithPaging("SELECT a, b, e, count(b), max(e) FROM %s GROUP BY a, b", pageSize),
-                              row(1, 2, 6, 2L, 12),
-                              row(1, 4, 12, 2L, 24),
-                              row(2, 2, 6, 1L, 6),
-                              row(2, 4, 12, 1L, 12),
-                              row(4, 8, 24, 1L, 24));
-
-                assertRowsNet(executeNetWithPaging("SELECT a, b, e, count(b), max(e) FROM %s", pageSize),
-                              row(1, 2, 6, 7L, 24));
-
-                assertRowsNet(executeNetWithPaging("SELECT a, b, e, count(b), max(e) FROM %s WHERE b = 2 GROUP BY a, b ALLOW FILTERING",
-                                                   pageSize),
-                              row(1, 2, 6, 2L, 12),
-                              row(2, 2, 6, 1L, 6));
-
-                assertRowsNet(executeNetWithPaging("SELECT a, b, e, count(b), max(e) FROM %s WHERE b = 2 ALLOW FILTERING",
-                                                   pageSize),
-                              row(1, 2, 6, 3L, 12));
-
-                // Range queries without aggregates
-                assertRowsNet(executeNetWithPaging("SELECT a, b, c, d FROM %s GROUP BY a, b, c", pageSize),
-                              row(1, 2, 1, 3),
-                              row(1, 2, 2, 6),
-                              row(1, 4, 2, 6),
-                              row(2, 2, 3, 3),
-                              row(2, 4, 3, 6),
-                              row(4, 8, 2, 12));
-
-                assertRowsNet(executeNetWithPaging("SELECT a, b, c, d FROM %s GROUP BY a, b", pageSize),
-                              row(1, 2, 1, 3),
-                              row(1, 4, 2, 6),
-                              row(2, 2, 3, 3),
-                              row(2, 4, 3, 6),
-                              row(4, 8, 2, 12));
-
-                // Range query with LIMIT
-                assertRowsNet(executeNetWithPaging("SELECT a, b, e, count(b), max(e) FROM %s GROUP BY a, b LIMIT 2",
-                                                   pageSize),
-                              row(1, 2, 6, 2L, 12),
-                              row(1, 4, 12, 2L, 24));
-
-                assertRowsNet(executeNetWithPaging("SELECT a, b, e, count(b), max(e) FROM %s LIMIT 2",
-                                                   pageSize),
-                              row(1, 2, 6, 7L, 24));
-
-                // Range queries with PER PARTITION LIMIT
-                assertRowsNet(executeNetWithPaging("SELECT a, b, e, count(b), max(e) FROM %s GROUP BY a, b PER PARTITION LIMIT 3", pageSize),
-                              row(1, 2, 6, 2L, 12),
-                              row(1, 4, 12, 2L, 24),
-                              row(2, 2, 6, 1L, 6),
-                              row(2, 4, 12, 1L, 12),
-                              row(4, 8, 24, 1L, 24));
-
-                assertRowsNet(executeNetWithPaging("SELECT a, b, e, count(b), max(e) FROM %s GROUP BY a, b PER PARTITION LIMIT 1", pageSize),
-                              row(1, 2, 6, 2L, 12),
-                              row(2, 2, 6, 1L, 6),
-                              row(4, 8, 24, 1L, 24));
-
-                // Range query with PER PARTITION LIMIT
-                assertRowsNet(executeNetWithPaging("SELECT a, b, e, count(b), max(e) FROM %s GROUP BY a, b PER PARTITION LIMIT 1 LIMIT 2", pageSize),
-                              row(1, 2, 6, 2L, 12),
-                              row(2, 2, 6, 1L, 6));
-
-                // Range query without aggregates and with PER PARTITION LIMIT
-                assertRowsNet(executeNetWithPaging("SELECT a, b, c, d FROM %s GROUP BY a, b, c PER PARTITION LIMIT 2", pageSize),
-                              row(1, 2, 1, 3),
-                              row(1, 2, 2, 6),
-                              row(2, 2, 3, 3),
-                              row(2, 4, 3, 6),
-                              row(4, 8, 2, 12));
-
-                // Range queries without aggregates and with LIMIT
-                assertRowsNet(executeNetWithPaging("SELECT a, b, c, d FROM %s GROUP BY a, b, c LIMIT 3", pageSize),
-                              row(1, 2, 1, 3),
-                              row(1, 2, 2, 6),
-                              row(1, 4, 2, 6));
-
-                assertRowsNet(executeNetWithPaging("SELECT a, b, c, d FROM %s GROUP BY a, b LIMIT 3", pageSize),
-                              row(1, 2, 1, 3),
-                              row(1, 4, 2, 6),
-                              row(2, 2, 3, 3));
-
-                // Range query without aggregates, with PER PARTITION LIMIT and with LIMIT
-                assertRowsNet(executeNetWithPaging("SELECT a, b, c, d FROM %s GROUP BY a, b, c PER PARTITION LIMIT 2 LIMIT 3", pageSize),
-                              row(1, 2, 1, 3),
-                              row(1, 2, 2, 6),
-                              row(2, 2, 3, 3));
-
-                // Range query with DISTINCT
-                assertRowsNet(executeNetWithPaging("SELECT DISTINCT a, count(a)FROM %s GROUP BY a", pageSize),
-                              row(1, 1L),
-                              row(2, 1L),
-                              row(4, 1L));
-
-                assertRowsNet(executeNetWithPaging("SELECT DISTINCT a, count(a)FROM %s", pageSize),
-                              row(1, 3L));
-
-                // Range query with DISTINCT and LIMIT
-                assertRowsNet(executeNetWithPaging("SELECT DISTINCT a, count(a)FROM %s GROUP BY a LIMIT 2", pageSize),
-                              row(1, 1L),
-                              row(2, 1L));
-
-                assertRowsNet(executeNetWithPaging("SELECT DISTINCT a, count(a)FROM %s LIMIT 2", pageSize),
-                              row(1, 3L));
-
-                // Range query with ORDER BY
-                assertInvalidMessage("ORDER BY is only supported when the partition key is restricted by an EQ or an IN",
-                                     "SELECT a, b, c, count(b), max(e) FROM %s GROUP BY a, b ORDER BY b DESC, c DESC");
-
-                // Single partition queries
-                assertRowsNet(executeNetWithPaging("SELECT a, b, e, count(b), max(e) FROM %s WHERE a = 1 GROUP BY a, b, c",
-                                                   pageSize),
-                              row(1, 2, 6, 1L, 6),
-                              row(1, 2, 12, 1L, 12),
-                              row(1, 4, 12, 2L, 24));
-
-                assertRowsNet(executeNetWithPaging("SELECT a, b, e, count(b), max(e) FROM %s WHERE a = 1", pageSize),
-                              row(1, 2, 6, 4L, 24));
-
-                assertRowsNet(executeNetWithPaging("SELECT a, b, e, count(b), max(e) FROM %s WHERE a = 1 AND b = 2 GROUP BY a, b, c",
-                                                   pageSize),
-                              row(1, 2, 6, 1L, 6),
-                              row(1, 2, 12, 1L, 12));
-
-                assertRowsNet(executeNetWithPaging("SELECT a, b, e, count(b), max(e) FROM %s WHERE a = 1 AND b = 2",
-                                                   pageSize),
-                              row(1, 2, 6, 2L, 12));
-
-                // Single partition queries without aggregates
-                assertRowsNet(executeNetWithPaging("SELECT a, b, c, d FROM %s WHERE a = 1 GROUP BY a, b", pageSize),
-                              row(1, 2, 1, 3),
-                              row(1, 4, 2, 6));
-
-                assertRowsNet(executeNetWithPaging("SELECT a, b, c, d FROM %s WHERE a = 1 GROUP BY a, b, c", pageSize),
-                              row(1, 2, 1, 3),
-                              row(1, 2, 2, 6),
-                              row(1, 4, 2, 6));
-
-                // Single partition query with DISTINCT
-                assertRowsNet(executeNetWithPaging("SELECT DISTINCT a, count(a)FROM %s WHERE a = 1 GROUP BY a",
-                                                   pageSize),
-                              row(1, 1L));
-
-                assertRowsNet(executeNetWithPaging("SELECT DISTINCT a, count(a)FROM %s WHERE a = 1 GROUP BY a",
-                                                   pageSize),
-                              row(1, 1L));
-
-                // Single partition queries with LIMIT
-                assertRowsNet(executeNetWithPaging("SELECT a, b, e, count(b), max(e) FROM %s WHERE a = 1 GROUP BY a, b, c LIMIT 10",
-                                                   pageSize),
-                              row(1, 2, 6, 1L, 6),
-                              row(1, 2, 12, 1L, 12),
-                              row(1, 4, 12, 2L, 24));
-
-                assertRowsNet(executeNetWithPaging("SELECT a, b, e, count(b), max(e) FROM %s WHERE a = 1 GROUP BY a, b, c LIMIT 2",
-                                                   pageSize),
-                              row(1, 2, 6, 1L, 6),
-                              row(1, 2, 12, 1L, 12));
-
-                assertRowsNet(executeNetWithPaging("SELECT a, b, e, count(b), max(e) FROM %s WHERE a = 1 LIMIT 2",
-                                                   pageSize),
-                              row(1, 2, 6, 4L, 24));
-
-                assertRowsNet(executeNetWithPaging("SELECT count(b), max(e) FROM %s WHERE a = 1 GROUP BY a, b, c LIMIT 1",
-                                                   pageSize),
-                              row(1L, 6));
-
-                // Single partition query with PER PARTITION LIMIT
-                assertRowsNet(executeNetWithPaging("SELECT a, b, e, count(b), max(e) FROM %s WHERE a = 1 GROUP BY a, b, c PER PARTITION LIMIT 2",
-                                                   pageSize),
-                              row(1, 2, 6, 1L, 6),
-                              row(1, 2, 12, 1L, 12));
-
-                // Single partition queries without aggregates and with LIMIT
-                assertRowsNet(executeNetWithPaging("SELECT a, b, c, d FROM %s WHERE a = 1 GROUP BY a, b LIMIT 2",
-                                                   pageSize),
-                              row(1, 2, 1, 3),
-                              row(1, 4, 2, 6));
-
-                assertRowsNet(executeNetWithPaging("SELECT a, b, c, d FROM %s WHERE a = 1 GROUP BY a, b LIMIT 1",
-                                                   pageSize),
-                              row(1, 2, 1, 3));
-
-                assertRowsNet(executeNetWithPaging("SELECT a, b, c, d FROM %s WHERE a = 1 GROUP BY a, b, c LIMIT 2",
-                                                   pageSize),
-                              row(1, 2, 1, 3),
-                              row(1, 2, 2, 6));
-
-                // Single partition queries with ORDER BY
-                assertRowsNet(executeNetWithPaging("SELECT a, b, e, count(b), max(e) FROM %s WHERE a = 1 GROUP BY a, b, c ORDER BY b DESC, c DESC",
-                                                   pageSize),
-                              row(1, 4, 24, 2L, 24),
-                              row(1, 2, 12, 1L, 12),
-                              row(1, 2, 6, 1L, 6));
-
-                assertRowsNet(executeNetWithPaging("SELECT a, b, e, count(b), max(e) FROM %s WHERE a = 1 ORDER BY b DESC, c DESC",
-                                                   pageSize),
-                              row(1, 4, 24, 4L, 24));
-
-                // Single partition queries with ORDER BY and LIMIT
-                assertRowsNet(executeNetWithPaging("SELECT a, b, e, count(b), max(e) FROM %s WHERE a = 1 GROUP BY a, b, c ORDER BY b DESC, c DESC LIMIT 2",
-                                                   pageSize),
-                              row(1, 4, 24, 2L, 24),
-                              row(1, 2, 12, 1L, 12));
-
-                assertRowsNet(executeNetWithPaging("SELECT a, b, e, count(b), max(e) FROM %s WHERE a = 1 ORDER BY b DESC, c DESC LIMIT 2",
-                                                   pageSize),
-                              row(1, 4, 24, 4L, 24));
-
-                // Single partition queries with ORDER BY and PER PARTITION LIMIT
-                assertRowsNet(executeNetWithPaging("SELECT a, b, e, count(b), max(e) FROM %s WHERE a = 1 GROUP BY a, b, c ORDER BY b DESC, c DESC PER PARTITION LIMIT 2",
-                                                   pageSize),
-                              row(1, 4, 24, 2L, 24),
-                              row(1, 2, 12, 1L, 12));
-
-                // Multi-partitions queries
-                assertRowsNet(executeNetWithPaging("SELECT a, b, e, count(b), max(e) FROM %s WHERE a IN (1, 2, 4) GROUP BY a, b, c",
-                                                   pageSize),
-                              row(1, 2, 6, 1L, 6),
-                              row(1, 2, 12, 1L, 12),
-                              row(1, 4, 12, 2L, 24),
-                              row(2, 2, 6, 1L, 6),
-                              row(2, 4, 12, 1L, 12),
-                              row(4, 8, 24, 1L, 24));
-
-                assertRowsNet(executeNetWithPaging("SELECT a, b, e, count(b), max(e) FROM %s WHERE a IN (1, 2, 4)",
-                                                   pageSize),
-                              row(1, 2, 6, 7L, 24));
-
-                assertRowsNet(executeNetWithPaging("SELECT a, b, e, count(b), max(e) FROM %s WHERE a IN (1, 2, 4) AND b = 2 GROUP BY a, b, c",
-                                                   pageSize),
-                              row(1, 2, 6, 1L, 6),
-                              row(1, 2, 12, 1L, 12),
-                              row(2, 2, 6, 1L, 6));
-
-                assertRowsNet(executeNetWithPaging("SELECT a, b, e, count(b), max(e) FROM %s WHERE a IN (1, 2, 4) AND b = 2",
-                                                   pageSize),
-                              row(1, 2, 6, 3L, 12));
-
-                // Multi-partitions queries with PER PARTITION LIMIT
-                assertRowsNet(executeNetWithPaging("SELECT a, b, e, count(b), max(e) FROM %s WHERE a IN (1, 2, 4) GROUP BY a, b, c PER PARTITION LIMIT 2",
-                                                   pageSize),
-                              row(1, 2, 6, 1L, 6),
-                              row(1, 2, 12, 1L, 12),
-                              row(2, 2, 6, 1L, 6),
-                              row(2, 4, 12, 1L, 12),
-                              row(4, 8, 24, 1L, 24));
-
-                assertRowsNet(executeNetWithPaging("SELECT a, b, e, count(b), max(e) FROM %s WHERE a IN (1, 2, 4) GROUP BY a, b, c PER PARTITION LIMIT 1",
-                                                   pageSize),
-                              row(1, 2, 6, 1L, 6),
-                              row(2, 2, 6, 1L, 6),
-                              row(4, 8, 24, 1L, 24));
-
-                // Multi-partitions queries without aggregates
-                assertRowsNet(executeNetWithPaging("SELECT a, b, c, d FROM %s WHERE a IN (1, 2, 4) GROUP BY a, b",
-                                                   pageSize),
-                              row(1, 2, 1, 3),
-                              row(1, 4, 2, 6),
-                              row(2, 2, 3, 3),
-                              row(2, 4, 3, 6),
-                              row(4, 8, 2, 12));
-
-                assertRowsNet(executeNetWithPaging("SELECT a, b, c, d FROM %s WHERE a IN (1, 2, 4) GROUP BY a, b, c",
-                                                   pageSize),
-                              row(1, 2, 1, 3),
-                              row(1, 2, 2, 6),
-                              row(1, 4, 2, 6),
-                              row(2, 2, 3, 3),
-                              row(2, 4, 3, 6),
-                              row(4, 8, 2, 12));
-
-                // Multi-partitions queries with DISTINCT
-                assertRowsNet(executeNetWithPaging("SELECT DISTINCT a, count(a)FROM %s WHERE a IN (1, 2, 4) GROUP BY a",
-                                                   pageSize),
-                              row(1, 1L),
-                              row(2, 1L),
-                              row(4, 1L));
-
-                assertRowsNet(executeNetWithPaging("SELECT DISTINCT a, count(a)FROM %s WHERE a IN (1, 2, 4)",
-                                                   pageSize),
-                              row(1, 3L));
-
-                // Multi-partitions query with DISTINCT and LIMIT
-                assertRowsNet(executeNetWithPaging("SELECT DISTINCT a, count(a)FROM %s WHERE a IN (1, 2, 4) GROUP BY a LIMIT 2",
-                                                   pageSize),
-                              row(1, 1L),
-                              row(2, 1L));
-
-                assertRowsNet(executeNetWithPaging("SELECT DISTINCT a, count(a)FROM %s WHERE a IN (1, 2, 4) LIMIT 2",
-                                                   pageSize),
-                              row(1, 3L));
-            }
+            // Range queries
+            assertRowsNet(executeNetWithPaging("SELECT a, b, e, count(b), max(e) FROM %s GROUP BY a", pageSize),
+                          row(1, 2, 6, 4L, 24),
+                          row(2, 2, 6, 2L, 12),
+                          row(4, 8, 24, 1L, 24));
+
+            assertRowsNet(executeNetWithPaging("SELECT a, b, e, count(b), max(e) FROM %s GROUP BY a, b", pageSize),
+                          row(1, 2, 6, 2L, 12),
+                          row(1, 4, 12, 2L, 24),
+                          row(2, 2, 6, 1L, 6),
+                          row(2, 4, 12, 1L, 12),
+                          row(4, 8, 24, 1L, 24));
+
+            assertRowsNet(executeNetWithPaging("SELECT a, b, e, count(b), max(e) FROM %s", pageSize),
+                          row(1, 2, 6, 7L, 24));
+
+            assertRowsNet(executeNetWithPaging("SELECT a, b, e, count(b), max(e) FROM %s WHERE b = 2 GROUP BY a, b ALLOW FILTERING",
+                                               pageSize),
+                          row(1, 2, 6, 2L, 12),
+                          row(2, 2, 6, 1L, 6));
+
+            assertRowsNet(executeNetWithPaging("SELECT a, b, e, count(b), max(e) FROM %s WHERE b = 2 ALLOW FILTERING",
+                                               pageSize),
+                          row(1, 2, 6, 3L, 12));
+
+            // Range queries without aggregates
+            assertRowsNet(executeNetWithPaging("SELECT a, b, c, d FROM %s GROUP BY a, b, c", pageSize),
+                          row(1, 2, 1, 3),
+                          row(1, 2, 2, 6),
+                          row(1, 4, 2, 6),
+                          row(2, 2, 3, 3),
+                          row(2, 4, 3, 6),
+                          row(4, 8, 2, 12));
+
+            assertRowsNet(executeNetWithPaging("SELECT a, b, c, d FROM %s GROUP BY a, b", pageSize),
+                          row(1, 2, 1, 3),
+                          row(1, 4, 2, 6),
+                          row(2, 2, 3, 3),
+                          row(2, 4, 3, 6),
+                          row(4, 8, 2, 12));
+
+            // Range query with LIMIT
+            assertRowsNet(executeNetWithPaging("SELECT a, b, e, count(b), max(e) FROM %s GROUP BY a, b LIMIT 2",
+                                               pageSize),
+                          row(1, 2, 6, 2L, 12),
+                          row(1, 4, 12, 2L, 24));
+
+            assertRowsNet(executeNetWithPaging("SELECT a, b, e, count(b), max(e) FROM %s LIMIT 2",
+                                               pageSize),
+                          row(1, 2, 6, 7L, 24));
+
+            // Range queries with PER PARTITION LIMIT
+            assertRowsNet(executeNetWithPaging("SELECT a, b, e, count(b), max(e) FROM %s GROUP BY a, b PER PARTITION LIMIT 3", pageSize),
+                          row(1, 2, 6, 2L, 12),
+                          row(1, 4, 12, 2L, 24),
+                          row(2, 2, 6, 1L, 6),
+                          row(2, 4, 12, 1L, 12),
+                          row(4, 8, 24, 1L, 24));
+
+            assertRowsNet(executeNetWithPaging("SELECT a, b, e, count(b), max(e) FROM %s GROUP BY a, b PER PARTITION LIMIT 1", pageSize),
+                          row(1, 2, 6, 2L, 12),
+                          row(2, 2, 6, 1L, 6),
+                          row(4, 8, 24, 1L, 24));
+
+            // Range query with PER PARTITION LIMIT
+            assertRowsNet(executeNetWithPaging("SELECT a, b, e, count(b), max(e) FROM %s GROUP BY a, b PER PARTITION LIMIT 1 LIMIT 2", pageSize),
+                          row(1, 2, 6, 2L, 12),
+                          row(2, 2, 6, 1L, 6));
+
+            // Range query without aggregates and with PER PARTITION LIMIT
+            assertRowsNet(executeNetWithPaging("SELECT a, b, c, d FROM %s GROUP BY a, b, c PER PARTITION LIMIT 2", pageSize),
+                          row(1, 2, 1, 3),
+                          row(1, 2, 2, 6),
+                          row(2, 2, 3, 3),
+                          row(2, 4, 3, 6),
+                          row(4, 8, 2, 12));
+
+            // Range queries without aggregates and with LIMIT
+            assertRowsNet(executeNetWithPaging("SELECT a, b, c, d FROM %s GROUP BY a, b, c LIMIT 3", pageSize),
+                          row(1, 2, 1, 3),
+                          row(1, 2, 2, 6),
+                          row(1, 4, 2, 6));
+
+            assertRowsNet(executeNetWithPaging("SELECT a, b, c, d FROM %s GROUP BY a, b LIMIT 3", pageSize),
+                          row(1, 2, 1, 3),
+                          row(1, 4, 2, 6),
+                          row(2, 2, 3, 3));
+
+            // Range query without aggregates, with PER PARTITION LIMIT and with LIMIT
+            assertRowsNet(executeNetWithPaging("SELECT a, b, c, d FROM %s GROUP BY a, b, c PER PARTITION LIMIT 2 LIMIT 3", pageSize),
+                          row(1, 2, 1, 3),
+                          row(1, 2, 2, 6),
+                          row(2, 2, 3, 3));
+
+            // Range query with DISTINCT
+            assertRowsNet(executeNetWithPaging("SELECT DISTINCT a, count(a)FROM %s GROUP BY a", pageSize),
+                          row(1, 1L),
+                          row(2, 1L),
+                          row(4, 1L));
+
+            assertRowsNet(executeNetWithPaging("SELECT DISTINCT a, count(a)FROM %s", pageSize),
+                          row(1, 3L));
+
+            // Range query with DISTINCT and LIMIT
+            assertRowsNet(executeNetWithPaging("SELECT DISTINCT a, count(a)FROM %s GROUP BY a LIMIT 2", pageSize),
+                          row(1, 1L),
+                          row(2, 1L));
+
+            assertRowsNet(executeNetWithPaging("SELECT DISTINCT a, count(a)FROM %s LIMIT 2", pageSize),
+                          row(1, 3L));
+
+            // Range query with ORDER BY
+            assertInvalidMessage("ORDER BY is only supported when the partition key is restricted by an EQ or an IN",
+                                 "SELECT a, b, c, count(b), max(e) FROM %s GROUP BY a, b ORDER BY b DESC, c DESC");
+
+            // Single partition queries
+            assertRowsNet(executeNetWithPaging("SELECT a, b, e, count(b), max(e) FROM %s WHERE a = 1 GROUP BY a, b, c",
+                                               pageSize),
+                          row(1, 2, 6, 1L, 6),
+                          row(1, 2, 12, 1L, 12),
+                          row(1, 4, 12, 2L, 24));
+
+            assertRowsNet(executeNetWithPaging("SELECT a, b, e, count(b), max(e) FROM %s WHERE a = 1", pageSize),
+                          row(1, 2, 6, 4L, 24));
+
+            assertRowsNet(executeNetWithPaging("SELECT a, b, e, count(b), max(e) FROM %s WHERE a = 1 AND b = 2 GROUP BY a, b, c",
+                                               pageSize),
+                          row(1, 2, 6, 1L, 6),
+                          row(1, 2, 12, 1L, 12));
+
+            assertRowsNet(executeNetWithPaging("SELECT a, b, e, count(b), max(e) FROM %s WHERE a = 1 AND b = 2",
+                                               pageSize),
+                          row(1, 2, 6, 2L, 12));
+
+            // Single partition queries without aggregates
+            assertRowsNet(executeNetWithPaging("SELECT a, b, c, d FROM %s WHERE a = 1 GROUP BY a, b", pageSize),
+                          row(1, 2, 1, 3),
+                          row(1, 4, 2, 6));
+
+            assertRowsNet(executeNetWithPaging("SELECT a, b, c, d FROM %s WHERE a = 1 GROUP BY a, b, c", pageSize),
+                          row(1, 2, 1, 3),
+                          row(1, 2, 2, 6),
+                          row(1, 4, 2, 6));
+
+            // Single partition query with DISTINCT
+            assertRowsNet(executeNetWithPaging("SELECT DISTINCT a, count(a)FROM %s WHERE a = 1 GROUP BY a",
+                                               pageSize),
+                          row(1, 1L));
+
+            assertRowsNet(executeNetWithPaging("SELECT DISTINCT a, count(a)FROM %s WHERE a = 1 GROUP BY a",
+                                               pageSize),
+                          row(1, 1L));
+
+            // Single partition queries with LIMIT
+            assertRowsNet(executeNetWithPaging("SELECT a, b, e, count(b), max(e) FROM %s WHERE a = 1 GROUP BY a, b, c LIMIT 10",
+                                               pageSize),
+                          row(1, 2, 6, 1L, 6),
+                          row(1, 2, 12, 1L, 12),
+                          row(1, 4, 12, 2L, 24));
+
+            assertRowsNet(executeNetWithPaging("SELECT a, b, e, count(b), max(e) FROM %s WHERE a = 1 GROUP BY a, b, c LIMIT 2",
+                                               pageSize),
+                          row(1, 2, 6, 1L, 6),
+                          row(1, 2, 12, 1L, 12));
+
+            assertRowsNet(executeNetWithPaging("SELECT a, b, e, count(b), max(e) FROM %s WHERE a = 1 LIMIT 2",
+                                               pageSize),
+                          row(1, 2, 6, 4L, 24));
+
+            assertRowsNet(executeNetWithPaging("SELECT count(b), max(e) FROM %s WHERE a = 1 GROUP BY a, b, c LIMIT 1",
+                                               pageSize),
+                          row(1L, 6));
+
+            // Single partition query with PER PARTITION LIMIT
+            assertRowsNet(executeNetWithPaging("SELECT a, b, e, count(b), max(e) FROM %s WHERE a = 1 GROUP BY a, b, c PER PARTITION LIMIT 2",
+                                               pageSize),
+                          row(1, 2, 6, 1L, 6),
+                          row(1, 2, 12, 1L, 12));
+
+            // Single partition queries without aggregates and with LIMIT
+            assertRowsNet(executeNetWithPaging("SELECT a, b, c, d FROM %s WHERE a = 1 GROUP BY a, b LIMIT 2",
+                                               pageSize),
+                          row(1, 2, 1, 3),
+                          row(1, 4, 2, 6));
+
+            assertRowsNet(executeNetWithPaging("SELECT a, b, c, d FROM %s WHERE a = 1 GROUP BY a, b LIMIT 1",
+                                               pageSize),
+                          row(1, 2, 1, 3));
+
+            assertRowsNet(executeNetWithPaging("SELECT a, b, c, d FROM %s WHERE a = 1 GROUP BY a, b, c LIMIT 2",
+                                               pageSize),
+                          row(1, 2, 1, 3),
+                          row(1, 2, 2, 6));
+
+            // Single partition queries with ORDER BY
+            assertRowsNet(executeNetWithPaging("SELECT a, b, e, count(b), max(e) FROM %s WHERE a = 1 GROUP BY a, b, c ORDER BY b DESC, c DESC",
+                                               pageSize),
+                          row(1, 4, 24, 2L, 24),
+                          row(1, 2, 12, 1L, 12),
+                          row(1, 2, 6, 1L, 6));
+
+            assertRowsNet(executeNetWithPaging("SELECT a, b, e, count(b), max(e) FROM %s WHERE a = 1 ORDER BY b DESC, c DESC",
+                                               pageSize),
+                          row(1, 4, 24, 4L, 24));
+
+            // Single partition queries with ORDER BY and LIMIT
+            assertRowsNet(executeNetWithPaging("SELECT a, b, e, count(b), max(e) FROM %s WHERE a = 1 GROUP BY a, b, c ORDER BY b DESC, c DESC LIMIT 2",
+                                               pageSize),
+                          row(1, 4, 24, 2L, 24),
+                          row(1, 2, 12, 1L, 12));
+
+            assertRowsNet(executeNetWithPaging("SELECT a, b, e, count(b), max(e) FROM %s WHERE a = 1 ORDER BY b DESC, c DESC LIMIT 2",
+                                               pageSize),
+                          row(1, 4, 24, 4L, 24));
+
+            // Single partition queries with ORDER BY and PER PARTITION LIMIT
+            assertRowsNet(executeNetWithPaging("SELECT a, b, e, count(b), max(e) FROM %s WHERE a = 1 GROUP BY a, b, c ORDER BY b DESC, c DESC PER PARTITION LIMIT 2",
+                                               pageSize),
+                          row(1, 4, 24, 2L, 24),
+                          row(1, 2, 12, 1L, 12));
+
+            // Multi-partitions queries
+            assertRowsNet(executeNetWithPaging("SELECT a, b, e, count(b), max(e) FROM %s WHERE a IN (1, 2, 4) GROUP BY a, b, c",
+                                               pageSize),
+                          row(1, 2, 6, 1L, 6),
+                          row(1, 2, 12, 1L, 12),
+                          row(1, 4, 12, 2L, 24),
+                          row(2, 2, 6, 1L, 6),
+                          row(2, 4, 12, 1L, 12),
+                          row(4, 8, 24, 1L, 24));
+
+            assertRowsNet(executeNetWithPaging("SELECT a, b, e, count(b), max(e) FROM %s WHERE a IN (1, 2, 4)",
+                                               pageSize),
+                          row(1, 2, 6, 7L, 24));
+
+            assertRowsNet(executeNetWithPaging("SELECT a, b, e, count(b), max(e) FROM %s WHERE a IN (1, 2, 4) AND b = 2 GROUP BY a, b, c",
+                                               pageSize),
+                          row(1, 2, 6, 1L, 6),
+                          row(1, 2, 12, 1L, 12),
+                          row(2, 2, 6, 1L, 6));
+
+            assertRowsNet(executeNetWithPaging("SELECT a, b, e, count(b), max(e) FROM %s WHERE a IN (1, 2, 4) AND b = 2",
+                                               pageSize),
+                          row(1, 2, 6, 3L, 12));
+
+            // Multi-partitions queries with PER PARTITION LIMIT
+            assertRowsNet(executeNetWithPaging("SELECT a, b, e, count(b), max(e) FROM %s WHERE a IN (1, 2, 4) GROUP BY a, b, c PER PARTITION LIMIT 2",
+                                               pageSize),
+                          row(1, 2, 6, 1L, 6),
+                          row(1, 2, 12, 1L, 12),
+                          row(2, 2, 6, 1L, 6),
+                          row(2, 4, 12, 1L, 12),
+                          row(4, 8, 24, 1L, 24));
+
+            assertRowsNet(executeNetWithPaging("SELECT a, b, e, count(b), max(e) FROM %s WHERE a IN (1, 2, 4) GROUP BY a, b, c PER PARTITION LIMIT 1",
+                                               pageSize),
+                          row(1, 2, 6, 1L, 6),
+                          row(2, 2, 6, 1L, 6),
+                          row(4, 8, 24, 1L, 24));
+
+            // Multi-partitions queries without aggregates
+            assertRowsNet(executeNetWithPaging("SELECT a, b, c, d FROM %s WHERE a IN (1, 2, 4) GROUP BY a, b",
+                                               pageSize),
+                          row(1, 2, 1, 3),
+                          row(1, 4, 2, 6),
+                          row(2, 2, 3, 3),
+                          row(2, 4, 3, 6),
+                          row(4, 8, 2, 12));
+
+            assertRowsNet(executeNetWithPaging("SELECT a, b, c, d FROM %s WHERE a IN (1, 2, 4) GROUP BY a, b, c",
+                                               pageSize),
+                          row(1, 2, 1, 3),
+                          row(1, 2, 2, 6),
+                          row(1, 4, 2, 6),
+                          row(2, 2, 3, 3),
+                          row(2, 4, 3, 6),
+                          row(4, 8, 2, 12));
+
+            // Multi-partitions queries with DISTINCT
+            assertRowsNet(executeNetWithPaging("SELECT DISTINCT a, count(a)FROM %s WHERE a IN (1, 2, 4) GROUP BY a",
+                                               pageSize),
+                          row(1, 1L),
+                          row(2, 1L),
+                          row(4, 1L));
+
+            assertRowsNet(executeNetWithPaging("SELECT DISTINCT a, count(a)FROM %s WHERE a IN (1, 2, 4)",
+                                               pageSize),
+                          row(1, 3L));
+
+            // Multi-partitions query with DISTINCT and LIMIT
+            assertRowsNet(executeNetWithPaging("SELECT DISTINCT a, count(a)FROM %s WHERE a IN (1, 2, 4) GROUP BY a LIMIT 2",
+                                               pageSize),
+                          row(1, 1L),
+                          row(2, 1L));
+
+            assertRowsNet(executeNetWithPaging("SELECT DISTINCT a, count(a)FROM %s WHERE a IN (1, 2, 4) LIMIT 2",
+                                               pageSize),
+                          row(1, 3L));
         }
     }
 
     @Test
     public void testGroupByWithRangeNamesQueryWithPaging() throws Throwable
     {
-        for (String compactOption : new String[] { "", " WITH COMPACT STORAGE" })
+        createTable("CREATE TABLE %s (a int, b int, c int, d int, primary key (a, b, c))");
+
+        for (int i = 1; i < 5; i++)
+            for (int j = 1; j < 5; j++)
+                for (int k = 1; k < 5; k++)
+                    execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", i, j, k, i + j);
+
+        // Makes sure that we have some tombstones
+        execute("DELETE FROM %s WHERE a = 3");
+
+        for (int pageSize = 1; pageSize < 2; pageSize++)
         {
-            createTable("CREATE TABLE %s (a int, b int, c int, d int, primary key (a, b, c))"
-                    + compactOption);
-
-            for (int i = 1; i < 5; i++)
-                for (int j = 1; j < 5; j++)
-                    for (int k = 1; k < 5; k++)
-                        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", i, j, k, i + j);
-
-            // Makes sure that we have some tombstones
-            execute("DELETE FROM %s WHERE a = 3");
-
-            for (int pageSize = 1; pageSize < 2; pageSize++)
-            {
-                // Range queries
-                assertRowsNet(executeNetWithPaging("SELECT a, b, d, count(b), max(d) FROM %s WHERE b = 1 and c IN (1, 2) GROUP BY a ALLOW FILTERING", pageSize),
-                              row(1, 1, 2, 2L, 2),
-                              row(2, 1, 3, 2L, 3),
-                              row(4, 1, 5, 2L, 5));
-
-                assertRowsNet(executeNetWithPaging("SELECT a, b, d, count(b), max(d) FROM %s WHERE b = 1 and c IN (1, 2) GROUP BY a, b ALLOW FILTERING", pageSize),
-                              row(1, 1, 2, 2L, 2),
-                              row(2, 1, 3, 2L, 3),
-                              row(4, 1, 5, 2L, 5));
-
-                assertRowsNet(executeNetWithPaging("SELECT a, b, d, count(b), max(d) FROM %s WHERE b IN (1, 2) and c IN (1, 2) GROUP BY a, b ALLOW FILTERING", pageSize),
-                              row(1, 1, 2, 2L, 2),
-                              row(1, 2, 3, 2L, 3),
-                              row(2, 1, 3, 2L, 3),
-                              row(2, 2, 4, 2L, 4),
-                              row(4, 1, 5, 2L, 5),
-                              row(4, 2, 6, 2L, 6));
-
-                // Range queries with LIMIT
-                assertRowsNet(executeNetWithPaging("SELECT a, b, d, count(b), max(d) FROM %s WHERE b = 1 and c IN (1, 2) GROUP BY a LIMIT 5 ALLOW FILTERING", pageSize),
-                              row(1, 1, 2, 2L, 2),
-                              row(2, 1, 3, 2L, 3),
-                              row(4, 1, 5, 2L, 5));
-
-                assertRowsNet(executeNetWithPaging("SELECT a, b, d, count(b), max(d) FROM %s WHERE b = 1 and c IN (1, 2) GROUP BY a, b LIMIT 3 ALLOW FILTERING", pageSize),
-                              row(1, 1, 2, 2L, 2),
-                              row(2, 1, 3, 2L, 3),
-                              row(4, 1, 5, 2L, 5));
-
-                assertRowsNet(executeNetWithPaging("SELECT a, b, d, count(b), max(d) FROM %s WHERE b IN (1, 2) and c IN (1, 2) GROUP BY a, b LIMIT 3 ALLOW FILTERING", pageSize),
-                              row(1, 1, 2, 2L, 2),
-                              row(1, 2, 3, 2L, 3),
-                              row(2, 1, 3, 2L, 3));
-
-                // Range queries with PER PARTITION LIMIT
-                assertRowsNet(executeNetWithPaging("SELECT a, b, d, count(b), max(d) FROM %s WHERE b = 1 and c IN (1, 2) GROUP BY a, b PER PARTITION LIMIT 2 ALLOW FILTERING", pageSize),
-                              row(1, 1, 2, 2L, 2),
-                              row(2, 1, 3, 2L, 3),
-                              row(4, 1, 5, 2L, 5));
-
-                assertRowsNet(executeNetWithPaging("SELECT a, b, d, count(b), max(d) FROM %s WHERE b IN (1, 2) and c IN (1, 2) GROUP BY a, b PER PARTITION LIMIT 1 ALLOW FILTERING", pageSize),
-                              row(1, 1, 2, 2L, 2),
-                              row(2,

<TRUNCATED>

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


[03/25] cassandra git commit: Allow dropping COMPACT STORAGE flag

Posted by if...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/test/unit/org/apache/cassandra/cql3/validation/operations/DropCompactStorageThriftTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/DropCompactStorageThriftTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/DropCompactStorageThriftTest.java
new file mode 100644
index 0000000..dde3e7b
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/DropCompactStorageThriftTest.java
@@ -0,0 +1,525 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.cql3.validation.operations;
+
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+
+import org.junit.Test;
+
+import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.cql3.ColumnSpecification;
+import org.apache.cassandra.cql3.UntypedResultSet;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.AsciiType;
+import org.apache.cassandra.db.marshal.BytesType;
+import org.apache.cassandra.db.marshal.CompositeType;
+import org.apache.cassandra.db.marshal.EmptyType;
+import org.apache.cassandra.db.marshal.Int32Type;
+import org.apache.cassandra.db.marshal.MapType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.locator.SimpleStrategy;
+import org.apache.cassandra.thrift.Cassandra;
+import org.apache.cassandra.thrift.CfDef;
+import org.apache.cassandra.thrift.Column;
+import org.apache.cassandra.thrift.ColumnDef;
+import org.apache.cassandra.thrift.ColumnOrSuperColumn;
+import org.apache.cassandra.thrift.ColumnParent;
+import org.apache.cassandra.thrift.IndexType;
+import org.apache.cassandra.thrift.KsDef;
+import org.apache.cassandra.thrift.Mutation;
+import org.apache.cassandra.thrift.SuperColumn;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+import static org.apache.cassandra.thrift.ConsistencyLevel.ONE;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+public class DropCompactStorageThriftTest extends ThriftCQLTester
+{
+    @Test
+    public void thriftCreatedTableTest() throws Throwable
+    {
+        final String KEYSPACE = "thrift_created_table_test_ks";
+        final String TABLE = "test_table_1";
+
+        CfDef cfDef = new CfDef().setDefault_validation_class(Int32Type.instance.toString())
+                                 .setKey_validation_class(AsciiType.instance.toString())
+                                 .setComparator_type(AsciiType.instance.toString())
+                                 .setColumn_metadata(Arrays.asList(new ColumnDef(ByteBufferUtil.bytes("col1"),
+                                                                                 AsciiType.instance.toString())
+                                                                   .setIndex_name("col1Index")
+                                                                   .setIndex_type(IndexType.KEYS),
+                                                                   new ColumnDef(ByteBufferUtil.bytes("col2"),
+                                                                                 AsciiType.instance.toString())
+                                                                   .setIndex_name("col2Index")
+                                                                   .setIndex_type(IndexType.KEYS)))
+                                 .setKeyspace(KEYSPACE)
+                                 .setName(TABLE);
+
+        KsDef ksDef = new KsDef(KEYSPACE,
+                                SimpleStrategy.class.getName(),
+                                Arrays.asList(cfDef));
+        ksDef.setStrategy_options(Collections.singletonMap("replication_factor", "1"));
+
+        Cassandra.Client client = getClient();
+        client.system_add_keyspace(ksDef);
+        client.set_keyspace(KEYSPACE);
+
+        client.insert(UTF8Type.instance.decompose("key1"),
+                      new ColumnParent(TABLE),
+                      getColumnForInsert(ByteBufferUtil.bytes("col1"), ByteBufferUtil.bytes("val1")),
+                      ONE);
+
+        client.insert(UTF8Type.instance.decompose("key1"),
+                      new ColumnParent(TABLE),
+                      getColumnForInsert(ByteBufferUtil.bytes("col2"), ByteBufferUtil.bytes("val2")),
+                      ONE);
+
+        client.insert(UTF8Type.instance.decompose("key1"),
+                      new ColumnParent(TABLE),
+                      getColumnForInsert(ByteBufferUtil.bytes("dynamicKey1"), ByteBufferUtil.bytes(100)),
+                      ONE);
+
+        client.insert(UTF8Type.instance.decompose("key1"),
+                      new ColumnParent(TABLE),
+                      getColumnForInsert(ByteBufferUtil.bytes("dynamicKey2"), ByteBufferUtil.bytes(200)),
+                      ONE);
+
+
+        execute(String.format("ALTER TABLE %s.%s DROP COMPACT STORAGE;", KEYSPACE, TABLE));
+        UntypedResultSet resultSet = execute(String.format("select * from %s.%s",
+                                                           KEYSPACE, TABLE));
+        assertColumnType(AsciiType.instance, resultSet, "key");
+        assertColumnType(AsciiType.instance, resultSet, "column1");
+        assertColumnType(Int32Type.instance, resultSet, "value");
+        assertColumnType(AsciiType.instance, resultSet, "col1");
+        assertColumnType(AsciiType.instance, resultSet, "col2");
+
+        assertRows(resultSet,
+                   row("key1", "dynamicKey1", "val1", "val2", 100),
+                   row("key1", "dynamicKey2", "val1", "val2", 200));
+    }
+
+    @Test
+    public void thriftStaticCompatTableTest() throws Throwable
+    {
+        String KEYSPACE = keyspace();
+        String TABLE = createTable("CREATE TABLE %s (key ascii PRIMARY KEY, val ascii) WITH COMPACT STORAGE");
+
+        Cassandra.Client client = getClient();
+        client.set_keyspace(KEYSPACE);
+
+        client.insert(UTF8Type.instance.decompose("key1"),
+                      new ColumnParent(TABLE),
+                      getColumnForInsert(ByteBufferUtil.bytes("val"), ByteBufferUtil.bytes("val1")),
+                      ONE);
+
+        client.insert(UTF8Type.instance.decompose("key1"),
+                      new ColumnParent(TABLE),
+                      getColumnForInsert(ByteBufferUtil.bytes("dynamicKey1"), ByteBufferUtil.bytes("dynamicValue1")),
+                      ONE);
+
+        client.insert(UTF8Type.instance.decompose("key1"),
+                      new ColumnParent(TABLE),
+                      getColumnForInsert(ByteBufferUtil.bytes("dynamicKey2"), ByteBufferUtil.bytes("dynamicValue2")),
+                      ONE);
+
+        execute(String.format("ALTER TABLE %s.%s DROP COMPACT STORAGE;", KEYSPACE, TABLE));
+        UntypedResultSet resultSet = execute(String.format("select * from %s.%s",
+                                                           KEYSPACE, TABLE));
+        assertColumnType(AsciiType.instance, resultSet, "key");
+        assertColumnType(UTF8Type.instance, resultSet, "column1");
+        assertColumnType(AsciiType.instance, resultSet, "val");
+        assertColumnType(BytesType.instance, resultSet, "value");
+
+        // Values are interpreted as bytes by default:
+        assertRows(resultSet,
+                   row("key1", "dynamicKey1", "val1", ByteBufferUtil.bytes("dynamicValue1")),
+                   row("key1", "dynamicKey2", "val1", ByteBufferUtil.bytes("dynamicValue2")));
+    }
+
+    @Test
+    public void testSparseCompactTableIndex() throws Throwable
+    {
+        createTable("CREATE TABLE %s (key ascii PRIMARY KEY, val ascii) WITH COMPACT STORAGE");
+
+        // Indexes are allowed only on the sparse compact tables
+        createIndex("CREATE INDEX ON %s(val)");
+        for (int i = 0; i < 10; i++)
+            execute("INSERT INTO %s (key, val) VALUES (?, ?)", Integer.toString(i), Integer.toString(i * 10));
+
+        alterTable("ALTER TABLE %s DROP COMPACT STORAGE");
+
+        assertRows(execute("SELECT * FROM %s WHERE val = '50'"),
+                   row("5", null, "50", null));
+        assertRows(execute("SELECT * FROM %s WHERE key = '5'"),
+                   row("5", null, "50", null));
+    }
+
+    @Test
+    public void thriftCompatTableTest() throws Throwable
+    {
+        String KEYSPACE = keyspace();
+        String TABLE = createTable("CREATE TABLE %s (pkey ascii, ckey ascii, PRIMARY KEY (pkey, ckey)) WITH COMPACT STORAGE");
+
+        Cassandra.Client client = getClient();
+        client.set_keyspace(KEYSPACE);
+
+        client.insert(UTF8Type.instance.decompose("key1"),
+                      new ColumnParent(TABLE),
+                      getColumnForInsert(ByteBufferUtil.bytes("ckeyValue1"), ByteBufferUtil.EMPTY_BYTE_BUFFER),
+                      ONE);
+
+        client.insert(UTF8Type.instance.decompose("key1"),
+                      new ColumnParent(TABLE),
+                      getColumnForInsert(ByteBufferUtil.bytes("ckeyValue2"), ByteBufferUtil.EMPTY_BYTE_BUFFER),
+                      ONE);
+
+        execute(String.format("ALTER TABLE %s.%s DROP COMPACT STORAGE;", KEYSPACE, TABLE));
+        UntypedResultSet resultSet = execute(String.format("select * from %s.%s",
+                                                           KEYSPACE, TABLE));
+        assertColumnType(AsciiType.instance, resultSet, "pkey");
+        assertColumnType(AsciiType.instance, resultSet, "ckey");
+        assertColumnType(EmptyType.instance, resultSet, "value");
+
+        // Value is always empty
+        assertRows(resultSet,
+                   row("key1", "ckeyValue1", ByteBufferUtil.EMPTY_BYTE_BUFFER),
+                   row("key1", "ckeyValue2", ByteBufferUtil.EMPTY_BYTE_BUFFER));
+    }
+
+    @Test
+    public void thriftDenseTableTest() throws Throwable
+    {
+        String KEYSPACE = keyspace();
+        String TABLE = createTable("CREATE TABLE %s (pkey text, ckey text, v text, PRIMARY KEY (pkey, ckey)) WITH COMPACT STORAGE");
+
+        Cassandra.Client client = getClient();
+        client.set_keyspace(KEYSPACE);
+
+        client.insert(UTF8Type.instance.decompose("key1"),
+                      new ColumnParent(TABLE),
+                      getColumnForInsert(ByteBufferUtil.bytes("ckey1"), ByteBufferUtil.bytes("cvalue1")),
+                      ONE);
+
+        client.insert(UTF8Type.instance.decompose("key1"),
+                      new ColumnParent(TABLE),
+                      getColumnForInsert(ByteBufferUtil.bytes("ckey2"), ByteBufferUtil.bytes("cvalue2")),
+                      ONE);
+
+        execute(String.format("ALTER TABLE %s.%s DROP COMPACT STORAGE;", KEYSPACE, TABLE));
+        UntypedResultSet resultSet = execute(String.format("select * from %s.%s",
+                                                           KEYSPACE, TABLE));
+        assertColumnType(UTF8Type.instance, resultSet, "pkey");
+        assertColumnType(UTF8Type.instance, resultSet, "ckey");
+        assertColumnType(UTF8Type.instance, resultSet, "v");
+
+        assertRows(resultSet,
+                   row("key1", "ckey1", "cvalue1"),
+                   row("key1", "ckey2", "cvalue2"));
+    }
+
+    @Test
+    public void thriftTableWithIntKey() throws Throwable
+    {
+        final String KEYSPACE = "thrift_table_with_int_key_ks";
+        final String TABLE = "test_table_1";
+
+        ByteBuffer columnName = ByteBufferUtil.bytes("columnname");
+        CfDef cfDef = new CfDef().setDefault_validation_class(UTF8Type.instance.toString())
+                                 .setKey_validation_class(BytesType.instance.toString())
+                                 .setComparator_type(BytesType.instance.toString())
+                                 .setColumn_metadata(Arrays.asList(new ColumnDef(columnName,
+                                                                                 Int32Type.instance.toString())
+                                                                   .setIndex_name("col1Index")
+                                                                   .setIndex_type(IndexType.KEYS)))
+                                 .setKeyspace(KEYSPACE)
+                                 .setName(TABLE);
+
+        KsDef ksDef = new KsDef(KEYSPACE,
+                                SimpleStrategy.class.getName(),
+                                Arrays.asList(cfDef));
+        ksDef.setStrategy_options(Collections.singletonMap("replication_factor", "1"));
+
+        Cassandra.Client client = getClient();
+        client.system_add_keyspace(ksDef);
+        client.set_keyspace(KEYSPACE);
+
+        client.insert(UTF8Type.instance.decompose("key1"),
+                      new ColumnParent(TABLE),
+                      getColumnForInsert(columnName, ByteBufferUtil.bytes(100)),
+                      ONE);
+
+        execute(String.format("ALTER TABLE %s.%s DROP COMPACT STORAGE;", KEYSPACE, TABLE));
+        UntypedResultSet resultSet = execute(String.format("select * from %s.%s",
+                                                           KEYSPACE, TABLE));
+
+        assertEquals(resultSet.metadata()
+                              .stream()
+                              .filter((cs) -> cs.name.toString().equals(BytesType.instance.getString(columnName)))
+                              .findFirst()
+                              .get().type,
+                     Int32Type.instance);
+
+        assertRows(resultSet,
+                   row(UTF8Type.instance.decompose("key1"), null, 100, null));
+    }
+
+    @Test
+    public void thriftCompatTableWithSupercolumnsTest() throws Throwable
+    {
+        final String KEYSPACE = "thrift_compact_table_with_supercolumns_test";
+        final String TABLE = "test_table_1";
+
+        CfDef cfDef = new CfDef().setColumn_type("Super")
+                                 .setSubcomparator_type(Int32Type.instance.toString())
+                                 .setComparator_type(AsciiType.instance.toString())
+                                 .setDefault_validation_class(AsciiType.instance.toString())
+                                 .setKey_validation_class(AsciiType.instance.toString())
+                                 .setKeyspace(KEYSPACE)
+                                 .setName(TABLE);
+
+        KsDef ksDef = new KsDef(KEYSPACE,
+                                SimpleStrategy.class.getName(),
+                                Arrays.asList(cfDef));
+        ksDef.setStrategy_options(Collections.singletonMap("replication_factor", "1"));
+
+        Cassandra.Client client = getClient();
+        client.system_add_keyspace(ksDef);
+
+        client.set_keyspace(KEYSPACE);
+
+        Mutation mutation = new Mutation();
+        ColumnOrSuperColumn csoc = new ColumnOrSuperColumn();
+        csoc.setSuper_column(getSuperColumnForInsert(ByteBufferUtil.bytes("val1"),
+                                                     Arrays.asList(getColumnForInsert(ByteBufferUtil.bytes(1), ByteBufferUtil.bytes("value1")),
+                                                                   getColumnForInsert(ByteBufferUtil.bytes(2), ByteBufferUtil.bytes("value2")),
+                                                                   getColumnForInsert(ByteBufferUtil.bytes(3), ByteBufferUtil.bytes("value3")))));
+        mutation.setColumn_or_supercolumn(csoc);
+
+        Mutation mutation2 = new Mutation();
+        ColumnOrSuperColumn csoc2 = new ColumnOrSuperColumn();
+        csoc2.setSuper_column(getSuperColumnForInsert(ByteBufferUtil.bytes("val2"),
+                                                     Arrays.asList(getColumnForInsert(ByteBufferUtil.bytes(4), ByteBufferUtil.bytes("value7")),
+                                                                   getColumnForInsert(ByteBufferUtil.bytes(5), ByteBufferUtil.bytes("value8")),
+                                                                   getColumnForInsert(ByteBufferUtil.bytes(6), ByteBufferUtil.bytes("value9")))));
+        mutation2.setColumn_or_supercolumn(csoc2);
+
+        client.batch_mutate(Collections.singletonMap(ByteBufferUtil.bytes("key1"),
+                                                     Collections.singletonMap(TABLE, Arrays.asList(mutation, mutation2))),
+                            ONE);
+
+        execute(String.format("ALTER TABLE %s.%s DROP COMPACT STORAGE;", KEYSPACE, TABLE));
+        UntypedResultSet resultSet = execute(String.format("select * from %s.%s",
+                                                           KEYSPACE, TABLE));
+        assertColumnType(AsciiType.instance, resultSet, "key");
+        assertColumnType(AsciiType.instance, resultSet, "column1");
+        assertColumnType(MapType.getInstance(Int32Type.instance, AsciiType.instance, true), resultSet, "");
+
+        assertRows(resultSet,
+                   row("key1", "val1", map(1, "value1", 2, "value2", 3, "value3")),
+                   row("key1", "val2", map(4, "value7", 5, "value8", 6, "value9")));
+
+        assertRows(execute(String.format("SELECT \"\" FROM %s.%s;", KEYSPACE, TABLE)),
+                   row(map(1, "value1", 2, "value2", 3, "value3")),
+                   row(map(4, "value7", 5, "value8", 6, "value9")));
+
+        assertInvalidMessage("Range deletions are not supported for specific columns",
+                             String.format("DELETE \"\" FROM %s.%s WHERE key=?;", KEYSPACE, TABLE),
+                             "key1");
+
+        execute(String.format("TRUNCATE %s.%s;", KEYSPACE, TABLE));
+
+        execute(String.format("INSERT INTO %s.%s (key, column1, \"\") VALUES (?, ?, ?);", KEYSPACE, TABLE),
+                "key3", "val1", map(7, "value7", 8, "value8"));
+
+        assertRows(execute(String.format("SELECT \"\" FROM %s.%s;", KEYSPACE, TABLE)),
+                   row(map(7, "value7", 8, "value8")));
+    }
+
+    @Test
+    public void thriftCreatedTableWithCompositeColumnsTest() throws Throwable
+    {
+        final String KEYSPACE = "thrift_created_table_with_composites_test_ks";
+        final String TABLE = "test_table_1";
+
+        CompositeType type = CompositeType.getInstance(AsciiType.instance, AsciiType.instance, AsciiType.instance);
+        CfDef cfDef = new CfDef().setDefault_validation_class(AsciiType.instance.toString())
+                                 .setComparator_type(type.toString())
+                                 .setKey_validation_class(AsciiType.instance.toString())
+                                 .setKeyspace(KEYSPACE)
+                                 .setName(TABLE);
+
+        KsDef ksDef = new KsDef(KEYSPACE,
+                                SimpleStrategy.class.getName(),
+                                Arrays.asList(cfDef));
+        ksDef.setStrategy_options(Collections.singletonMap("replication_factor", "1"));
+
+        Cassandra.Client client = getClient();
+        client.system_add_keyspace(ksDef);
+        client.set_keyspace(KEYSPACE);
+
+        client.insert(UTF8Type.instance.decompose("key1"),
+                      new ColumnParent(TABLE),
+                      getColumnForInsert(type.decompose("a", "b", "c"), ByteBufferUtil.bytes("val1")),
+                      ONE);
+
+        client.insert(UTF8Type.instance.decompose("key1"),
+                      new ColumnParent(TABLE),
+                      getColumnForInsert(type.decompose("d", "e", "f"), ByteBufferUtil.bytes("val2")),
+                      ONE);
+
+        execute(String.format("ALTER TABLE %s.%s DROP COMPACT STORAGE;", KEYSPACE, TABLE));
+        UntypedResultSet resultSet = execute(String.format("select * from %s.%s",
+                                                           KEYSPACE, TABLE));
+
+        assertColumnType(AsciiType.instance, resultSet, "key");
+        assertColumnType(AsciiType.instance, resultSet, "column1");
+        assertColumnType(AsciiType.instance, resultSet, "column2");
+        assertColumnType(AsciiType.instance, resultSet, "column3");
+        assertColumnType(AsciiType.instance, resultSet, "value");
+
+        assertRows(resultSet,
+                   row("key1", "a", "b", "c", "val1"),
+                   row("key1", "d", "e", "f", "val2"));
+    }
+
+    @Test
+    public void compactTableWithoutClusteringKeyTest() throws Throwable
+    {
+        String KEYSPACE = keyspace();
+        String TABLE = createTable("CREATE TABLE %s (pkey text PRIMARY KEY, s1 text, s2 text) WITH COMPACT STORAGE");
+
+        Cassandra.Client client = getClient();
+        client.set_keyspace(KEYSPACE);
+
+        client.insert(UTF8Type.instance.decompose("key1"),
+                      new ColumnParent(TABLE),
+                      getColumnForInsert(ByteBufferUtil.bytes("ckey1"), ByteBufferUtil.bytes("val1")),
+                      ONE);
+
+        client.insert(UTF8Type.instance.decompose("key1"),
+                      new ColumnParent(TABLE),
+                      getColumnForInsert(ByteBufferUtil.bytes("ckey2"), ByteBufferUtil.bytes("val2")),
+                      ONE);
+
+        client.insert(UTF8Type.instance.decompose("key1"),
+                      new ColumnParent(TABLE),
+                      getColumnForInsert(ByteBufferUtil.bytes("s1"), ByteBufferUtil.bytes("s1Val")),
+                      ONE);
+
+        client.insert(UTF8Type.instance.decompose("key1"),
+                      new ColumnParent(TABLE),
+                      getColumnForInsert(ByteBufferUtil.bytes("s2"), ByteBufferUtil.bytes("s2Val")),
+                      ONE);
+
+        execute(String.format("ALTER TABLE %s.%s DROP COMPACT STORAGE;", KEYSPACE, TABLE));
+        UntypedResultSet resultSet = execute(String.format("select * from %s.%s",
+                                                           KEYSPACE, TABLE));
+
+        assertColumnType(UTF8Type.instance, resultSet, "pkey");
+        assertColumnType(UTF8Type.instance, resultSet, "s1");
+        assertColumnType(UTF8Type.instance, resultSet, "s2");
+        assertColumnType(UTF8Type.instance, resultSet, "column1");
+        assertColumnType(BytesType.instance, resultSet, "value");
+
+        assertRows(resultSet,
+                   row("key1", "ckey1", "s1Val", "s2Val", ByteBufferUtil.bytes("val1")),
+                   row("key1", "ckey2", "s1Val", "s2Val", ByteBufferUtil.bytes("val2")));
+    }
+
+    @Test
+    public void denseTableTestTest() throws Throwable
+    {
+        String KEYSPACE = keyspace();
+        String TABLE = createTable("CREATE TABLE %s (pkey text PRIMARY KEY, s text) WITH COMPACT STORAGE");
+
+        Cassandra.Client client = getClient();
+        client.set_keyspace(KEYSPACE);
+
+        client.insert(UTF8Type.instance.decompose("key1"),
+                      new ColumnParent(TABLE),
+                      getColumnForInsert(ByteBufferUtil.bytes("ckey1"), ByteBufferUtil.bytes("val1")),
+                      ONE);
+
+        client.insert(UTF8Type.instance.decompose("key1"),
+                      new ColumnParent(TABLE),
+                      getColumnForInsert(ByteBufferUtil.bytes("ckey2"), ByteBufferUtil.bytes("val2")),
+                      ONE);
+
+        client.insert(UTF8Type.instance.decompose("key1"),
+                      new ColumnParent(TABLE),
+                      getColumnForInsert(ByteBufferUtil.bytes("s"), ByteBufferUtil.bytes("sval1")),
+                      ONE);
+
+        client.insert(UTF8Type.instance.decompose("key1"),
+                      new ColumnParent(TABLE),
+                      getColumnForInsert(ByteBufferUtil.bytes("s"), ByteBufferUtil.bytes("sval2")),
+                      ONE);
+
+        // `s` becomes static, `column1` becomes a clustering key, `value` becomes visible
+        execute(String.format("ALTER TABLE %s.%s DROP COMPACT STORAGE;", KEYSPACE, TABLE));
+        UntypedResultSet resultSet = execute(String.format("select * from %s.%s",
+                                                           KEYSPACE, TABLE));
+        assertColumnType(UTF8Type.instance, resultSet, "pkey");
+        assertColumnType(UTF8Type.instance, resultSet, "s");
+        assertColumnType(UTF8Type.instance, resultSet, "column1");
+        assertColumnType(BytesType.instance, resultSet, "value");
+
+        assertRows(resultSet,
+                   row("key1", "ckey1", "sval2", ByteBufferUtil.bytes("val1")),
+                   row("key1", "ckey2", "sval2", ByteBufferUtil.bytes("val2")));
+    }
+
+    private Column getColumnForInsert(ByteBuffer columnName, ByteBuffer value)
+    {
+        Column column = new Column();
+        column.setName(columnName);
+        column.setValue(value);
+        column.setTimestamp(System.currentTimeMillis());
+        return column;
+    }
+
+    private SuperColumn getSuperColumnForInsert(ByteBuffer columnName, List<Column> columns)
+    {
+        SuperColumn column = new SuperColumn();
+        column.setName(columnName);
+        for (Column c : columns)
+            column.addToColumns(c);
+        return column;
+    }
+
+    private static void assertColumnType(AbstractType t, UntypedResultSet resultSet, String columnName)
+    {
+        for (ColumnSpecification columnSpecification : resultSet.metadata())
+        {
+            if (columnSpecification.name.toString().equals(columnName))
+            {
+                assertEquals(t, columnSpecification.type);
+                return;
+            }
+        }
+
+        fail(String.format("Could not find a column with name '%s'", columnName));
+    }
+}


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


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

Posted by if...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/07fbd8ee/test/unit/org/apache/cassandra/cql3/validation/operations/DeleteTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/DeleteTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/DeleteTest.java
index ef1b5ac..0c88044 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/DeleteTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/DeleteTest.java
@@ -146,20 +146,6 @@ public class DeleteTest extends CQLTester
 
         assertRows(execute("SELECT * FROM %s"),
                    row("abc", 4, "xyz", "some other value"));
-
-        createTable("CREATE TABLE %s (username varchar, id int, name varchar, stuff varchar, PRIMARY KEY(username, id, name)) WITH COMPACT STORAGE");
-
-        execute("INSERT INTO %s (username, id, name, stuff) VALUES (?, ?, ?, ?)", "abc", 2, "rst", "some value");
-        execute("INSERT INTO %s (username, id, name, stuff) VALUES (?, ?, ?, ?)", "abc", 4, "xyz", "some other value");
-
-        assertRows(execute("SELECT * FROM %s"),
-                   row("abc", 2, "rst", "some value"),
-                   row("abc", 4, "xyz", "some other value"));
-
-        execute("DELETE FROM %s WHERE username='abc' AND id=2");
-
-        assertRows(execute("SELECT * FROM %s"),
-                   row("abc", 4, "xyz", "some other value"));
     }
 
     /**
@@ -438,64 +424,54 @@ public class DeleteTest extends CQLTester
 
     private void testDeleteWithNoClusteringColumns(boolean forceFlush) throws Throwable
     {
-        for (String compactOption : new String[] {"", " WITH COMPACT STORAGE" })
-        {
-            createTable("CREATE TABLE %s (partitionKey int PRIMARY KEY," +
-                                      "value int)" + compactOption);
+        createTable("CREATE TABLE %s (partitionKey int PRIMARY KEY," +
+                    "value int)");
 
-            execute("INSERT INTO %s (partitionKey, value) VALUES (0, 0)");
-            execute("INSERT INTO %s (partitionKey, value) VALUES (1, 1)");
-            execute("INSERT INTO %s (partitionKey, value) VALUES (2, 2)");
-            execute("INSERT INTO %s (partitionKey, value) VALUES (3, 3)");
-            flush(forceFlush);
+        execute("INSERT INTO %s (partitionKey, value) VALUES (0, 0)");
+        execute("INSERT INTO %s (partitionKey, value) VALUES (1, 1)");
+        execute("INSERT INTO %s (partitionKey, value) VALUES (2, 2)");
+        execute("INSERT INTO %s (partitionKey, value) VALUES (3, 3)");
+        flush(forceFlush);
 
-            execute("DELETE value FROM %s WHERE partitionKey = ?", 0);
-            flush(forceFlush);
+        execute("DELETE value FROM %s WHERE partitionKey = ?", 0);
+        flush(forceFlush);
 
-            if (isEmpty(compactOption))
-            {
-                assertRows(execute("SELECT * FROM %s WHERE partitionKey = ?", 0),
-                           row(0, null));
-            }
-            else
-            {
-                assertEmpty(execute("SELECT * FROM %s WHERE partitionKey = ?", 0));
-            }
+        assertRows(execute("SELECT * FROM %s WHERE partitionKey = ?", 0),
+                   row(0, null));
 
-            execute("DELETE FROM %s WHERE partitionKey IN (?, ?)", 0, 1);
-            flush(forceFlush);
-            assertRows(execute("SELECT * FROM %s"),
-                       row(2, 2),
-                       row(3, 3));
+        execute("DELETE FROM %s WHERE partitionKey IN (?, ?)", 0, 1);
+        flush(forceFlush);
+        assertRows(execute("SELECT * FROM %s"),
+                   row(2, 2),
+                   row(3, 3));
 
-            // test invalid queries
+        // test invalid queries
 
-            // token function
-            assertInvalidMessage("The token function cannot be used in WHERE clauses for DELETE statements",
-                                 "DELETE FROM %s WHERE token(partitionKey) = token(?)", 0);
+        // token function
+        assertInvalidMessage("The token function cannot be used in WHERE clauses for DELETE statements",
+                             "DELETE FROM %s WHERE token(partitionKey) = token(?)", 0);
 
-            // multiple time same primary key element in WHERE clause
-            assertInvalidMessage("partitionkey cannot be restricted by more than one relation if it includes an Equal",
-                                 "DELETE FROM %s WHERE partitionKey = ? AND partitionKey = ?", 0, 1);
+        // multiple time same primary key element in WHERE clause
+        assertInvalidMessage("partitionkey cannot be restricted by more than one relation if it includes an Equal",
+                             "DELETE FROM %s WHERE partitionKey = ? AND partitionKey = ?", 0, 1);
 
-            // unknown identifiers
-            assertInvalidMessage("Undefined column name unknown",
-                                 "DELETE unknown FROM %s WHERE partitionKey = ?", 0);
+        // unknown identifiers
+        assertInvalidMessage("Undefined column name unknown",
+                             "DELETE unknown FROM %s WHERE partitionKey = ?", 0);
 
-            assertInvalidMessage("Undefined column name partitionkey1",
-                                 "DELETE FROM %s WHERE partitionKey1 = ?", 0);
+        assertInvalidMessage("Undefined column name partitionkey1",
+                             "DELETE FROM %s WHERE partitionKey1 = ?", 0);
 
-            // Invalid operator in the where clause
-            assertInvalidMessage("Only EQ and IN relation are supported on the partition key (unless you use the token() function)",
-                                 "DELETE FROM %s WHERE partitionKey > ? ", 0);
+        // Invalid operator in the where clause
+        assertInvalidMessage("Only EQ and IN relation are supported on the partition key (unless you use the token() function)",
+                             "DELETE FROM %s WHERE partitionKey > ? ", 0);
 
-            assertInvalidMessage("Cannot use CONTAINS on non-collection column partitionkey",
-                                 "DELETE FROM %s WHERE partitionKey CONTAINS ?", 0);
+        assertInvalidMessage("Cannot use CONTAINS on non-collection column partitionkey",
+                             "DELETE FROM %s WHERE partitionKey CONTAINS ?", 0);
 
-            // Non primary key in the where clause
-            assertInvalidMessage("Non PRIMARY KEY columns found in where clause: value",
-                                 "DELETE FROM %s WHERE partitionKey = ? AND value = ?", 0, 1);
-        }
+        // Non primary key in the where clause
+        assertInvalidMessage("Non PRIMARY KEY columns found in where clause: value",
+                             "DELETE FROM %s WHERE partitionKey = ? AND value = ?", 0, 1);
     }
 
     @Test
@@ -507,87 +483,77 @@ public class DeleteTest extends CQLTester
 
     private void testDeleteWithOneClusteringColumns(boolean forceFlush) throws Throwable
     {
-        for (String compactOption : new String[] {"", " WITH COMPACT STORAGE" })
-        {
-            createTable("CREATE TABLE %s (partitionKey int," +
-                                      "clustering int," +
-                                      "value int," +
-                                      " PRIMARY KEY (partitionKey, clustering))" + compactOption);
-
-            execute("INSERT INTO %s (partitionKey, clustering, value) VALUES (0, 0, 0)");
-            execute("INSERT INTO %s (partitionKey, clustering, value) VALUES (0, 1, 1)");
-            execute("INSERT INTO %s (partitionKey, clustering, value) VALUES (0, 2, 2)");
-            execute("INSERT INTO %s (partitionKey, clustering, value) VALUES (0, 3, 3)");
-            execute("INSERT INTO %s (partitionKey, clustering, value) VALUES (0, 4, 4)");
-            execute("INSERT INTO %s (partitionKey, clustering, value) VALUES (0, 5, 5)");
-            execute("INSERT INTO %s (partitionKey, clustering, value) VALUES (1, 0, 6)");
-            flush(forceFlush);
-
-            execute("DELETE value FROM %s WHERE partitionKey = ? AND clustering = ?", 0, 1);
-            flush(forceFlush);
-            if (isEmpty(compactOption))
-            {
-                assertRows(execute("SELECT * FROM %s WHERE partitionKey = ? AND clustering = ?", 0, 1),
-                           row(0, 1, null));
-            }
-            else
-            {
-                assertEmpty(execute("SELECT * FROM %s WHERE partitionKey = ? AND clustering = ?", 0, 1));
-            }
+        createTable("CREATE TABLE %s (partitionKey int," +
+                    "clustering int," +
+                    "value int," +
+                    " PRIMARY KEY (partitionKey, clustering))");
+
+        execute("INSERT INTO %s (partitionKey, clustering, value) VALUES (0, 0, 0)");
+        execute("INSERT INTO %s (partitionKey, clustering, value) VALUES (0, 1, 1)");
+        execute("INSERT INTO %s (partitionKey, clustering, value) VALUES (0, 2, 2)");
+        execute("INSERT INTO %s (partitionKey, clustering, value) VALUES (0, 3, 3)");
+        execute("INSERT INTO %s (partitionKey, clustering, value) VALUES (0, 4, 4)");
+        execute("INSERT INTO %s (partitionKey, clustering, value) VALUES (0, 5, 5)");
+        execute("INSERT INTO %s (partitionKey, clustering, value) VALUES (1, 0, 6)");
+        flush(forceFlush);
 
-            execute("DELETE FROM %s WHERE partitionKey = ? AND clustering = ?", 0, 1);
-            flush(forceFlush);
-            assertEmpty(execute("SELECT value FROM %s WHERE partitionKey = ? AND clustering = ?", 0, 1));
+        execute("DELETE value FROM %s WHERE partitionKey = ? AND clustering = ?", 0, 1);
+        flush(forceFlush);
+        assertRows(execute("SELECT * FROM %s WHERE partitionKey = ? AND clustering = ?", 0, 1),
+                   row(0, 1, null));
 
-            execute("DELETE FROM %s WHERE partitionKey IN (?, ?) AND clustering = ?", 0, 1, 0);
-            flush(forceFlush);
-            assertRows(execute("SELECT * FROM %s WHERE partitionKey IN (?, ?)", 0, 1),
-                       row(0, 2, 2),
-                       row(0, 3, 3),
-                       row(0, 4, 4),
-                       row(0, 5, 5));
+        execute("DELETE FROM %s WHERE partitionKey = ? AND clustering = ?", 0, 1);
+        flush(forceFlush);
+        assertEmpty(execute("SELECT value FROM %s WHERE partitionKey = ? AND clustering = ?", 0, 1));
 
-            execute("DELETE FROM %s WHERE partitionKey = ? AND (clustering) IN ((?), (?))", 0, 4, 5);
-            flush(forceFlush);
-            assertRows(execute("SELECT * FROM %s WHERE partitionKey IN (?, ?)", 0, 1),
-                       row(0, 2, 2),
-                       row(0, 3, 3));
+        execute("DELETE FROM %s WHERE partitionKey IN (?, ?) AND clustering = ?", 0, 1, 0);
+        flush(forceFlush);
+        assertRows(execute("SELECT * FROM %s WHERE partitionKey IN (?, ?)", 0, 1),
+                   row(0, 2, 2),
+                   row(0, 3, 3),
+                   row(0, 4, 4),
+                   row(0, 5, 5));
+
+        execute("DELETE FROM %s WHERE partitionKey = ? AND (clustering) IN ((?), (?))", 0, 4, 5);
+        flush(forceFlush);
+        assertRows(execute("SELECT * FROM %s WHERE partitionKey IN (?, ?)", 0, 1),
+                   row(0, 2, 2),
+                   row(0, 3, 3));
 
-            // test invalid queries
+        // test invalid queries
 
-            // missing primary key element
-            assertInvalidMessage("Some partition key parts are missing: partitionkey",
-                                 "DELETE FROM %s WHERE clustering = ?", 1);
+        // missing primary key element
+        assertInvalidMessage("Some partition key parts are missing: partitionkey",
+                             "DELETE FROM %s WHERE clustering = ?", 1);
 
-            // token function
-            assertInvalidMessage("The token function cannot be used in WHERE clauses for DELETE statements",
-                                 "DELETE FROM %s WHERE token(partitionKey) = token(?) AND clustering = ? ", 0, 1);
+        // token function
+        assertInvalidMessage("The token function cannot be used in WHERE clauses for DELETE statements",
+                             "DELETE FROM %s WHERE token(partitionKey) = token(?) AND clustering = ? ", 0, 1);
 
-            // multiple time same primary key element in WHERE clause
-            assertInvalidMessage("clustering cannot be restricted by more than one relation if it includes an Equal",
-                                 "DELETE FROM %s WHERE partitionKey = ? AND clustering = ? AND clustering = ?", 0, 1, 1);
+        // multiple time same primary key element in WHERE clause
+        assertInvalidMessage("clustering cannot be restricted by more than one relation if it includes an Equal",
+                             "DELETE FROM %s WHERE partitionKey = ? AND clustering = ? AND clustering = ?", 0, 1, 1);
 
-            // unknown identifiers
-            assertInvalidMessage("Undefined column name value1",
-                                 "DELETE value1 FROM %s WHERE partitionKey = ? AND clustering = ?", 0, 1);
+        // unknown identifiers
+        assertInvalidMessage("Undefined column name value1",
+                             "DELETE value1 FROM %s WHERE partitionKey = ? AND clustering = ?", 0, 1);
 
-            assertInvalidMessage("Undefined column name partitionkey1",
-                                 "DELETE FROM %s WHERE partitionKey1 = ? AND clustering = ?", 0, 1);
+        assertInvalidMessage("Undefined column name partitionkey1",
+                             "DELETE FROM %s WHERE partitionKey1 = ? AND clustering = ?", 0, 1);
 
-            assertInvalidMessage("Undefined column name clustering_3",
-                                 "DELETE FROM %s WHERE partitionKey = ? AND clustering_3 = ?", 0, 1);
+        assertInvalidMessage("Undefined column name clustering_3",
+                             "DELETE FROM %s WHERE partitionKey = ? AND clustering_3 = ?", 0, 1);
 
-            // Invalid operator in the where clause
-            assertInvalidMessage("Only EQ and IN relation are supported on the partition key (unless you use the token() function)",
-                                 "DELETE FROM %s WHERE partitionKey > ? AND clustering = ?", 0, 1);
+        // Invalid operator in the where clause
+        assertInvalidMessage("Only EQ and IN relation are supported on the partition key (unless you use the token() function)",
+                             "DELETE FROM %s WHERE partitionKey > ? AND clustering = ?", 0, 1);
 
-            assertInvalidMessage("Cannot use CONTAINS on non-collection column partitionkey",
-                                 "DELETE FROM %s WHERE partitionKey CONTAINS ? AND clustering = ?", 0, 1);
+        assertInvalidMessage("Cannot use CONTAINS on non-collection column partitionkey",
+                             "DELETE FROM %s WHERE partitionKey CONTAINS ? AND clustering = ?", 0, 1);
 
-            // Non primary key in the where clause
-            assertInvalidMessage("Non PRIMARY KEY columns found in where clause: value",
-                                 "DELETE FROM %s WHERE partitionKey = ? AND clustering = ? AND value = ?", 0, 1, 3);
-        }
+        // Non primary key in the where clause
+        assertInvalidMessage("Non PRIMARY KEY columns found in where clause: value",
+                             "DELETE FROM %s WHERE partitionKey = ? AND clustering = ? AND value = ?", 0, 1, 3);
     }
 
     @Test
@@ -599,125 +565,96 @@ public class DeleteTest extends CQLTester
 
     private void testDeleteWithTwoClusteringColumns(boolean forceFlush) throws Throwable
     {
-        for (String compactOption : new String[] { "", " WITH COMPACT STORAGE" })
-        {
-            createTable("CREATE TABLE %s (partitionKey int," +
-                                      "clustering_1 int," +
-                                      "clustering_2 int," +
-                                      "value int," +
-                                      " PRIMARY KEY (partitionKey, clustering_1, clustering_2))" + compactOption);
+        createTable("CREATE TABLE %s (partitionKey int," +
+                    "clustering_1 int," +
+                    "clustering_2 int," +
+                    "value int," +
+                    " PRIMARY KEY (partitionKey, clustering_1, clustering_2))");
 
-            execute("INSERT INTO %s (partitionKey, clustering_1, clustering_2, value) VALUES (0, 0, 0, 0)");
-            execute("INSERT INTO %s (partitionKey, clustering_1, clustering_2, value) VALUES (0, 0, 1, 1)");
-            execute("INSERT INTO %s (partitionKey, clustering_1, clustering_2, value) VALUES (0, 0, 2, 2)");
-            execute("INSERT INTO %s (partitionKey, clustering_1, clustering_2, value) VALUES (0, 0, 3, 3)");
-            execute("INSERT INTO %s (partitionKey, clustering_1, clustering_2, value) VALUES (0, 1, 1, 4)");
-            execute("INSERT INTO %s (partitionKey, clustering_1, clustering_2, value) VALUES (0, 1, 2, 5)");
-            execute("INSERT INTO %s (partitionKey, clustering_1, clustering_2, value) VALUES (1, 0, 0, 6)");
-            flush(forceFlush);
+        execute("INSERT INTO %s (partitionKey, clustering_1, clustering_2, value) VALUES (0, 0, 0, 0)");
+        execute("INSERT INTO %s (partitionKey, clustering_1, clustering_2, value) VALUES (0, 0, 1, 1)");
+        execute("INSERT INTO %s (partitionKey, clustering_1, clustering_2, value) VALUES (0, 0, 2, 2)");
+        execute("INSERT INTO %s (partitionKey, clustering_1, clustering_2, value) VALUES (0, 0, 3, 3)");
+        execute("INSERT INTO %s (partitionKey, clustering_1, clustering_2, value) VALUES (0, 1, 1, 4)");
+        execute("INSERT INTO %s (partitionKey, clustering_1, clustering_2, value) VALUES (0, 1, 2, 5)");
+        execute("INSERT INTO %s (partitionKey, clustering_1, clustering_2, value) VALUES (1, 0, 0, 6)");
+        flush(forceFlush);
 
-            execute("DELETE value FROM %s WHERE partitionKey = ? AND clustering_1 = ? AND clustering_2 = ?", 0, 1, 1);
-            flush(forceFlush);
+        execute("DELETE value FROM %s WHERE partitionKey = ? AND clustering_1 = ? AND clustering_2 = ?", 0, 1, 1);
+        flush(forceFlush);
 
-            if (isEmpty(compactOption))
-            {
-                assertRows(execute("SELECT * FROM %s WHERE partitionKey = ? AND clustering_1 = ? AND clustering_2 = ?",
-                                   0, 1, 1),
-                           row(0, 1, 1, null));
-            }
-            else
-            {
-                assertEmpty(execute("SELECT * FROM %s WHERE partitionKey = ? AND clustering_1 = ? AND clustering_2 = ?",
-                                   0, 1, 1));
-            }
+        assertRows(execute("SELECT * FROM %s WHERE partitionKey = ? AND clustering_1 = ? AND clustering_2 = ?",
+                           0, 1, 1),
+                   row(0, 1, 1, null));
 
-            execute("DELETE FROM %s WHERE partitionKey = ? AND (clustering_1, clustering_2) = (?, ?)", 0, 1, 1);
-            flush(forceFlush);
-            assertEmpty(execute("SELECT value FROM %s WHERE partitionKey = ? AND clustering_1 = ? AND clustering_2 = ?",
-                                0, 1, 1));
-
-            execute("DELETE FROM %s WHERE partitionKey IN (?, ?) AND clustering_1 = ? AND clustering_2 = ?", 0, 1, 0, 0);
-            flush(forceFlush);
-            assertRows(execute("SELECT * FROM %s WHERE partitionKey IN (?, ?)", 0, 1),
-                       row(0, 0, 1, 1),
-                       row(0, 0, 2, 2),
-                       row(0, 0, 3, 3),
-                       row(0, 1, 2, 5));
-
-            Object[][] rows;
-            if (isEmpty(compactOption))
-            {
-                rows = new Object[][]{row(0, 0, 1, 1),
-                                      row(0, 0, 2, null),
-                                      row(0, 0, 3, null),
-                                      row(0, 1, 2, 5)};
-            }
-            else
-            {
-                rows = new Object[][]{row(0, 0, 1, 1), row(0, 1, 2, 5)};
-            }
+        execute("DELETE FROM %s WHERE partitionKey = ? AND (clustering_1, clustering_2) = (?, ?)", 0, 1, 1);
+        flush(forceFlush);
+        assertEmpty(execute("SELECT value FROM %s WHERE partitionKey = ? AND clustering_1 = ? AND clustering_2 = ?",
+                            0, 1, 1));
 
-            execute("DELETE value FROM %s WHERE partitionKey = ? AND clustering_1 = ? AND clustering_2 IN (?, ?)", 0, 0, 2, 3);
-            flush(forceFlush);
-            assertRows(execute("SELECT * FROM %s WHERE partitionKey IN (?, ?)", 0, 1), rows);
+        execute("DELETE FROM %s WHERE partitionKey IN (?, ?) AND clustering_1 = ? AND clustering_2 = ?", 0, 1, 0, 0);
+        flush(forceFlush);
+        assertRows(execute("SELECT * FROM %s WHERE partitionKey IN (?, ?)", 0, 1),
+                   row(0, 0, 1, 1),
+                   row(0, 0, 2, 2),
+                   row(0, 0, 3, 3),
+                   row(0, 1, 2, 5));
 
-            if (isEmpty(compactOption))
-            {
-                rows = new Object[][]{row(0, 0, 1, 1),
-                                      row(0, 0, 3, null)};
-            }
-            else
-            {
-                rows = new Object[][]{row(0, 0, 1, 1)};
-            }
+        execute("DELETE value FROM %s WHERE partitionKey = ? AND clustering_1 = ? AND clustering_2 IN (?, ?)", 0, 0, 2, 3);
+        flush(forceFlush);
+        assertRows(execute("SELECT * FROM %s WHERE partitionKey IN (?, ?)", 0, 1), row(0, 0, 1, 1),
+                   row(0, 0, 2, null),
+                   row(0, 0, 3, null),
+                   row(0, 1, 2, 5));
 
-            execute("DELETE FROM %s WHERE partitionKey = ? AND (clustering_1, clustering_2) IN ((?, ?), (?, ?))", 0, 0, 2, 1, 2);
-            flush(forceFlush);
-            assertRows(execute("SELECT * FROM %s WHERE partitionKey IN (?, ?)", 0, 1), rows);
+        execute("DELETE FROM %s WHERE partitionKey = ? AND (clustering_1, clustering_2) IN ((?, ?), (?, ?))", 0, 0, 2, 1, 2);
+        flush(forceFlush);
+        assertRows(execute("SELECT * FROM %s WHERE partitionKey IN (?, ?)", 0, 1),
+                   row(0, 0, 1, 1),
+                   row(0, 0, 3, null));
 
-            execute("DELETE FROM %s WHERE partitionKey = ? AND (clustering_1) IN ((?), (?)) AND clustering_2 = ?", 0, 0, 2, 3);
-            flush(forceFlush);
-            assertRows(execute("SELECT * FROM %s WHERE partitionKey IN (?, ?)", 0, 1),
-                       row(0, 0, 1, 1));
+        execute("DELETE FROM %s WHERE partitionKey = ? AND (clustering_1) IN ((?), (?)) AND clustering_2 = ?", 0, 0, 2, 3);
+        flush(forceFlush);
+        assertRows(execute("SELECT * FROM %s WHERE partitionKey IN (?, ?)", 0, 1),
+                   row(0, 0, 1, 1));
 
-            // test invalid queries
+        // test invalid queries
 
-            // missing primary key element
-            assertInvalidMessage("Some partition key parts are missing: partitionkey",
-                                 "DELETE FROM %s WHERE clustering_1 = ? AND clustering_2 = ?", 1, 1);
+        // missing primary key element
+        assertInvalidMessage("Some partition key parts are missing: partitionkey",
+                             "DELETE FROM %s WHERE clustering_1 = ? AND clustering_2 = ?", 1, 1);
 
-            assertInvalidMessage("PRIMARY KEY column \"clustering_2\" cannot be restricted as preceding column \"clustering_1\" is not restricted",
-                                 "DELETE FROM %s WHERE partitionKey = ? AND clustering_2 = ?", 0, 1);
+        assertInvalidMessage("PRIMARY KEY column \"clustering_2\" cannot be restricted as preceding column \"clustering_1\" is not restricted",
+                             "DELETE FROM %s WHERE partitionKey = ? AND clustering_2 = ?", 0, 1);
 
-            // token function
-            assertInvalidMessage("The token function cannot be used in WHERE clauses for DELETE statements",
-                                 "DELETE FROM %s WHERE token(partitionKey) = token(?) AND clustering_1 = ? AND clustering_2 = ?", 0, 1, 1);
+        // token function
+        assertInvalidMessage("The token function cannot be used in WHERE clauses for DELETE statements",
+                             "DELETE FROM %s WHERE token(partitionKey) = token(?) AND clustering_1 = ? AND clustering_2 = ?", 0, 1, 1);
 
-            // multiple time same primary key element in WHERE clause
-            assertInvalidMessage("clustering_1 cannot be restricted by more than one relation if it includes an Equal",
-                                 "DELETE FROM %s WHERE partitionKey = ? AND clustering_1 = ? AND clustering_2 = ? AND clustering_1 = ?", 0, 1, 1, 1);
+        // multiple time same primary key element in WHERE clause
+        assertInvalidMessage("clustering_1 cannot be restricted by more than one relation if it includes an Equal",
+                             "DELETE FROM %s WHERE partitionKey = ? AND clustering_1 = ? AND clustering_2 = ? AND clustering_1 = ?", 0, 1, 1, 1);
 
-            // unknown identifiers
-            assertInvalidMessage("Undefined column name value1",
-                                 "DELETE value1 FROM %s WHERE partitionKey = ? AND clustering_1 = ? AND clustering_2 = ?", 0, 1, 1);
+        // unknown identifiers
+        assertInvalidMessage("Undefined column name value1",
+                             "DELETE value1 FROM %s WHERE partitionKey = ? AND clustering_1 = ? AND clustering_2 = ?", 0, 1, 1);
 
-            assertInvalidMessage("Undefined column name partitionkey1",
-                                 "DELETE FROM %s WHERE partitionKey1 = ? AND clustering_1 = ? AND clustering_2 = ?", 0, 1, 1);
+        assertInvalidMessage("Undefined column name partitionkey1",
+                             "DELETE FROM %s WHERE partitionKey1 = ? AND clustering_1 = ? AND clustering_2 = ?", 0, 1, 1);
 
-            assertInvalidMessage("Undefined column name clustering_3",
-                                 "DELETE FROM %s WHERE partitionKey = ? AND clustering_1 = ? AND clustering_3 = ?", 0, 1, 1);
+        assertInvalidMessage("Undefined column name clustering_3",
+                             "DELETE FROM %s WHERE partitionKey = ? AND clustering_1 = ? AND clustering_3 = ?", 0, 1, 1);
 
-            // Invalid operator in the where clause
-            assertInvalidMessage("Only EQ and IN relation are supported on the partition key (unless you use the token() function)",
-                                 "DELETE FROM %s WHERE partitionKey > ? AND clustering_1 = ? AND clustering_2 = ?", 0, 1, 1);
+        // Invalid operator in the where clause
+        assertInvalidMessage("Only EQ and IN relation are supported on the partition key (unless you use the token() function)",
+                             "DELETE FROM %s WHERE partitionKey > ? AND clustering_1 = ? AND clustering_2 = ?", 0, 1, 1);
 
-            assertInvalidMessage("Cannot use CONTAINS on non-collection column partitionkey",
-                                 "DELETE FROM %s WHERE partitionKey CONTAINS ? AND clustering_1 = ? AND clustering_2 = ?", 0, 1, 1);
+        assertInvalidMessage("Cannot use CONTAINS on non-collection column partitionkey",
+                             "DELETE FROM %s WHERE partitionKey CONTAINS ? AND clustering_1 = ? AND clustering_2 = ?", 0, 1, 1);
 
-            // Non primary key in the where clause
-            assertInvalidMessage("Non PRIMARY KEY columns found in where clause: value",
-                                 "DELETE FROM %s WHERE partitionKey = ? AND clustering_1 = ? AND clustering_2 = ? AND value = ?", 0, 1, 1, 3);
-        }
+        // Non primary key in the where clause
+        assertInvalidMessage("Non PRIMARY KEY columns found in where clause: value",
+                             "DELETE FROM %s WHERE partitionKey = ? AND clustering_1 = ? AND clustering_2 = ? AND value = ?", 0, 1, 1, 3);
     }
 
     @Test
@@ -762,95 +699,92 @@ public class DeleteTest extends CQLTester
 
     private void testDeleteWithRangeAndOneClusteringColumn(boolean forceFlush) throws Throwable
     {
-        for (String compactOption : new String[] { "", " WITH COMPACT STORAGE" })
-        {
-            createTable("CREATE TABLE %s (partitionKey int," +
-                                          "clustering int," +
-                                          "value int," +
-                                          " PRIMARY KEY (partitionKey, clustering))" + compactOption);
-
-            int value = 0;
-            for (int partitionKey = 0; partitionKey < 5; partitionKey++)
-                for (int clustering1 = 0; clustering1 < 5; clustering1++)
-                        execute("INSERT INTO %s (partitionKey, clustering, value) VALUES (?, ?, ?)",
-                                partitionKey, clustering1, value++);
-
-            flush(forceFlush);
-
-            // test delete partition
-            execute("DELETE FROM %s WHERE partitionKey = ?", 1);
-            flush(forceFlush);
-            assertEmpty(execute("SELECT * FROM %s WHERE partitionKey = ?", 1));
-
-            // test slices on the first clustering column
-
-            execute("DELETE FROM %s WHERE partitionKey = ? AND  clustering >= ?", 0, 4);
-            flush(forceFlush);
-            assertRows(execute("SELECT * FROM %s WHERE partitionKey = ?", 0),
-                       row(0, 0, 0),
-                       row(0, 1, 1),
-                       row(0, 2, 2),
-                       row(0, 3, 3));
-
-            execute("DELETE FROM %s WHERE partitionKey = ? AND  clustering > ?", 0, 2);
-            flush(forceFlush);
-            assertRows(execute("SELECT * FROM %s WHERE partitionKey = ?", 0),
-                       row(0, 0, 0),
-                       row(0, 1, 1),
-                       row(0, 2, 2));
-
-            execute("DELETE FROM %s WHERE partitionKey = ? AND clustering <= ?", 0, 0);
-            flush(forceFlush);
-            assertRows(execute("SELECT * FROM %s WHERE partitionKey = ?", 0),
-                       row(0, 1, 1),
-                       row(0, 2, 2));
-
-            execute("DELETE FROM %s WHERE partitionKey = ? AND clustering < ?", 0, 2);
-            flush(forceFlush);
-            assertRows(execute("SELECT * FROM %s WHERE partitionKey = ?", 0),
-                       row(0, 2, 2));
-
-            execute("DELETE FROM %s WHERE partitionKey = ? AND clustering >= ? AND clustering < ?", 2, 0, 3);
-            flush(forceFlush);
-            assertRows(execute("SELECT * FROM %s WHERE partitionKey = ?", 2),
-                       row(2, 3, 13),
-                       row(2, 4, 14));
-
-            execute("DELETE FROM %s WHERE partitionKey = ? AND clustering > ? AND clustering <= ?", 2, 3, 5);
-            flush(forceFlush);
-            assertRows(execute("SELECT * FROM %s WHERE partitionKey = ?", 2),
-                       row(2, 3, 13));
-
-            execute("DELETE FROM %s WHERE partitionKey = ? AND clustering < ? AND clustering > ?", 2, 3, 5);
-            flush(forceFlush);
-            assertRows(execute("SELECT * FROM %s WHERE partitionKey = ?", 2),
-                       row(2, 3, 13));
-
-            // test multi-column slices
-            execute("DELETE FROM %s WHERE partitionKey = ? AND (clustering) > (?)", 3, 2);
-            flush(forceFlush);
-            assertRows(execute("SELECT * FROM %s WHERE partitionKey = ?", 3),
-                       row(3, 0, 15),
-                       row(3, 1, 16),
-                       row(3, 2, 17));
-
-            execute("DELETE FROM %s WHERE partitionKey = ? AND (clustering) < (?)", 3, 1);
-            flush(forceFlush);
-            assertRows(execute("SELECT * FROM %s WHERE partitionKey = ?", 3),
-                       row(3, 1, 16),
-                       row(3, 2, 17));
-
-            execute("DELETE FROM %s WHERE partitionKey = ? AND (clustering) >= (?) AND (clustering) <= (?)", 3, 0, 1);
-            flush(forceFlush);
-            assertRows(execute("SELECT * FROM %s WHERE partitionKey = ?", 3),
-                       row(3, 2, 17));
-
-            // Test invalid queries
-            assertInvalidMessage("Range deletions are not supported for specific columns",
-                                 "DELETE value FROM %s WHERE partitionKey = ? AND clustering >= ?", 2, 1);
-            assertInvalidMessage("Range deletions are not supported for specific columns",
-                                 "DELETE value FROM %s WHERE partitionKey = ?", 2);
-        }
+        createTable("CREATE TABLE %s (partitionKey int," +
+                    "clustering int," +
+                    "value int," +
+                    " PRIMARY KEY (partitionKey, clustering))");
+
+        int value = 0;
+        for (int partitionKey = 0; partitionKey < 5; partitionKey++)
+            for (int clustering1 = 0; clustering1 < 5; clustering1++)
+                execute("INSERT INTO %s (partitionKey, clustering, value) VALUES (?, ?, ?)",
+                        partitionKey, clustering1, value++);
+
+        flush(forceFlush);
+
+        // test delete partition
+        execute("DELETE FROM %s WHERE partitionKey = ?", 1);
+        flush(forceFlush);
+        assertEmpty(execute("SELECT * FROM %s WHERE partitionKey = ?", 1));
+
+        // test slices on the first clustering column
+
+        execute("DELETE FROM %s WHERE partitionKey = ? AND  clustering >= ?", 0, 4);
+        flush(forceFlush);
+        assertRows(execute("SELECT * FROM %s WHERE partitionKey = ?", 0),
+                   row(0, 0, 0),
+                   row(0, 1, 1),
+                   row(0, 2, 2),
+                   row(0, 3, 3));
+
+        execute("DELETE FROM %s WHERE partitionKey = ? AND  clustering > ?", 0, 2);
+        flush(forceFlush);
+        assertRows(execute("SELECT * FROM %s WHERE partitionKey = ?", 0),
+                   row(0, 0, 0),
+                   row(0, 1, 1),
+                   row(0, 2, 2));
+
+        execute("DELETE FROM %s WHERE partitionKey = ? AND clustering <= ?", 0, 0);
+        flush(forceFlush);
+        assertRows(execute("SELECT * FROM %s WHERE partitionKey = ?", 0),
+                   row(0, 1, 1),
+                   row(0, 2, 2));
+
+        execute("DELETE FROM %s WHERE partitionKey = ? AND clustering < ?", 0, 2);
+        flush(forceFlush);
+        assertRows(execute("SELECT * FROM %s WHERE partitionKey = ?", 0),
+                   row(0, 2, 2));
+
+        execute("DELETE FROM %s WHERE partitionKey = ? AND clustering >= ? AND clustering < ?", 2, 0, 3);
+        flush(forceFlush);
+        assertRows(execute("SELECT * FROM %s WHERE partitionKey = ?", 2),
+                   row(2, 3, 13),
+                   row(2, 4, 14));
+
+        execute("DELETE FROM %s WHERE partitionKey = ? AND clustering > ? AND clustering <= ?", 2, 3, 5);
+        flush(forceFlush);
+        assertRows(execute("SELECT * FROM %s WHERE partitionKey = ?", 2),
+                   row(2, 3, 13));
+
+        execute("DELETE FROM %s WHERE partitionKey = ? AND clustering < ? AND clustering > ?", 2, 3, 5);
+        flush(forceFlush);
+        assertRows(execute("SELECT * FROM %s WHERE partitionKey = ?", 2),
+                   row(2, 3, 13));
+
+        // test multi-column slices
+        execute("DELETE FROM %s WHERE partitionKey = ? AND (clustering) > (?)", 3, 2);
+        flush(forceFlush);
+        assertRows(execute("SELECT * FROM %s WHERE partitionKey = ?", 3),
+                   row(3, 0, 15),
+                   row(3, 1, 16),
+                   row(3, 2, 17));
+
+        execute("DELETE FROM %s WHERE partitionKey = ? AND (clustering) < (?)", 3, 1);
+        flush(forceFlush);
+        assertRows(execute("SELECT * FROM %s WHERE partitionKey = ?", 3),
+                   row(3, 1, 16),
+                   row(3, 2, 17));
+
+        execute("DELETE FROM %s WHERE partitionKey = ? AND (clustering) >= (?) AND (clustering) <= (?)", 3, 0, 1);
+        flush(forceFlush);
+        assertRows(execute("SELECT * FROM %s WHERE partitionKey = ?", 3),
+                   row(3, 2, 17));
+
+        // Test invalid queries
+        assertInvalidMessage("Range deletions are not supported for specific columns",
+                             "DELETE value FROM %s WHERE partitionKey = ? AND clustering >= ?", 2, 1);
+        assertInvalidMessage("Range deletions are not supported for specific columns",
+                             "DELETE value FROM %s WHERE partitionKey = ?", 2);
     }
 
     @Test
@@ -862,194 +796,193 @@ public class DeleteTest extends CQLTester
 
     private void testDeleteWithRangeAndTwoClusteringColumns(boolean forceFlush) throws Throwable
     {
-        for (String compactOption : new String[] { "", " WITH COMPACT STORAGE" })
-        {
-            createTable("CREATE TABLE %s (partitionKey int," +
+        createTable("CREATE TABLE %s (partitionKey int," +
                     "clustering_1 int," +
                     "clustering_2 int," +
                     "value int," +
-                    " PRIMARY KEY (partitionKey, clustering_1, clustering_2))" + compactOption);
-
-            int value = 0;
-            for (int partitionKey = 0; partitionKey < 5; partitionKey++)
-                for (int clustering1 = 0; clustering1 < 5; clustering1++)
-                    for (int clustering2 = 0; clustering2 < 5; clustering2++) {
-                        execute("INSERT INTO %s (partitionKey, clustering_1, clustering_2, value) VALUES (?, ?, ?, ?)",
-                                partitionKey, clustering1, clustering2, value++);}
-            flush(forceFlush);
-
-            // test unspecified second clustering column
-            execute("DELETE FROM %s WHERE partitionKey = ? AND clustering_1 = ?", 0, 1);
-            flush(forceFlush);
-            assertRows(execute("SELECT * FROM %s WHERE partitionKey = ? AND clustering_1 < ?", 0, 2),
-                       row(0, 0, 0, 0),
-                       row(0, 0, 1, 1),
-                       row(0, 0, 2, 2),
-                       row(0, 0, 3, 3),
-                       row(0, 0, 4, 4));
-
-            // test delete partition
-            execute("DELETE FROM %s WHERE partitionKey = ?", 1);
-            flush(forceFlush);
-            assertEmpty(execute("SELECT * FROM %s WHERE partitionKey = ?", 1));
-
-            // test slices on the second clustering column
-
-            execute("DELETE FROM %s WHERE partitionKey = ? AND clustering_1 = ? AND clustering_2 < ?", 0, 0, 2);
-            flush(forceFlush);
-            assertRows(execute("SELECT * FROM %s WHERE partitionKey = ? AND clustering_1 < ?", 0, 2),
-                       row(0, 0, 2, 2),
-                       row(0, 0, 3, 3),
-                       row(0, 0, 4, 4));
-
-            execute("DELETE FROM %s WHERE partitionKey = ? AND clustering_1 = ? AND clustering_2 <= ?", 0, 0, 3);
-            flush(forceFlush);
-            assertRows(execute("SELECT * FROM %s WHERE partitionKey = ? AND clustering_1 < ?", 0, 2),
-                       row(0, 0, 4, 4));
-
-            execute("DELETE FROM %s WHERE partitionKey = ? AND  clustering_1 = ? AND clustering_2 > ? ", 0, 2, 2);
-            flush(forceFlush);
-            assertRows(execute("SELECT * FROM %s WHERE partitionKey = ? AND  clustering_1 = ?", 0, 2),
-                       row(0, 2, 0, 10),
-                       row(0, 2, 1, 11),
-                       row(0, 2, 2, 12));
-
-            execute("DELETE FROM %s WHERE partitionKey = ? AND  clustering_1 = ? AND clustering_2 >= ? ", 0, 2, 1);
-            flush(forceFlush);
-            assertRows(execute("SELECT * FROM %s WHERE partitionKey = ? AND  clustering_1 = ?", 0, 2),
-                       row(0, 2, 0, 10));
-
-            execute("DELETE FROM %s WHERE partitionKey = ? AND  clustering_1 = ? AND clustering_2 > ? AND clustering_2 < ? ",
-                    0, 3, 1, 4);
-            flush(forceFlush);
-            assertRows(execute("SELECT * FROM %s WHERE partitionKey = ? AND  clustering_1 = ?", 0, 3),
-                       row(0, 3, 0, 15),
-                       row(0, 3, 1, 16),
-                       row(0, 3, 4, 19));
-
-            execute("DELETE FROM %s WHERE partitionKey = ? AND  clustering_1 = ? AND clustering_2 > ? AND clustering_2 < ? ",
-                    0, 3, 4, 1);
-            flush(forceFlush);
-            assertRows(execute("SELECT * FROM %s WHERE partitionKey = ? AND  clustering_1 = ?", 0, 3),
-                       row(0, 3, 0, 15),
-                       row(0, 3, 1, 16),
-                       row(0, 3, 4, 19));
-
-            execute("DELETE FROM %s WHERE partitionKey = ? AND  clustering_1 = ? AND clustering_2 >= ? AND clustering_2 <= ? ",
-                    0, 3, 1, 4);
-            flush(forceFlush);
-            assertRows(execute("SELECT * FROM %s WHERE partitionKey = ? AND  clustering_1 = ?", 0, 3),
-                       row(0, 3, 0, 15));
-
-            // test slices on the first clustering column
-
-            execute("DELETE FROM %s WHERE partitionKey = ? AND  clustering_1 >= ?", 0, 4);
-            flush(forceFlush);
-            assertRows(execute("SELECT * FROM %s WHERE partitionKey = ?", 0),
-                       row(0, 0, 4, 4),
-                       row(0, 2, 0, 10),
-                       row(0, 3, 0, 15));
-
-            execute("DELETE FROM %s WHERE partitionKey = ? AND  clustering_1 > ?", 0, 3);
-            flush(forceFlush);
-            assertRows(execute("SELECT * FROM %s WHERE partitionKey = ?", 0),
-                       row(0, 0, 4, 4),
-                       row(0, 2, 0, 10),
-                       row(0, 3, 0, 15));
-
-            execute("DELETE FROM %s WHERE partitionKey = ? AND clustering_1 < ?", 0, 3);
-            flush(forceFlush);
-            assertRows(execute("SELECT * FROM %s WHERE partitionKey = ?", 0),
-                       row(0, 3, 0, 15));
-
-            execute("DELETE FROM %s WHERE partitionKey = ? AND clustering_1 >= ? AND clustering_1 < ?", 2, 0, 3);
-            flush(forceFlush);
-            assertRows(execute("SELECT * FROM %s WHERE partitionKey = ?", 2),
-                       row(2, 3, 0, 65),
-                       row(2, 3, 1, 66),
-                       row(2, 3, 2, 67),
-                       row(2, 3, 3, 68),
-                       row(2, 3, 4, 69),
-                       row(2, 4, 0, 70),
-                       row(2, 4, 1, 71),
-                       row(2, 4, 2, 72),
-                       row(2, 4, 3, 73),
-                       row(2, 4, 4, 74));
-
-            execute("DELETE FROM %s WHERE partitionKey = ? AND clustering_1 > ? AND clustering_1 <= ?", 2, 3, 5);
-            flush(forceFlush);
-            assertRows(execute("SELECT * FROM %s WHERE partitionKey = ?", 2),
-                       row(2, 3, 0, 65),
-                       row(2, 3, 1, 66),
-                       row(2, 3, 2, 67),
-                       row(2, 3, 3, 68),
-                       row(2, 3, 4, 69));
-
-            execute("DELETE FROM %s WHERE partitionKey = ? AND clustering_1 < ? AND clustering_1 > ?", 2, 3, 5);
-            flush(forceFlush);
-            assertRows(execute("SELECT * FROM %s WHERE partitionKey = ?", 2),
-                       row(2, 3, 0, 65),
-                       row(2, 3, 1, 66),
-                       row(2, 3, 2, 67),
-                       row(2, 3, 3, 68),
-                       row(2, 3, 4, 69));
-
-            // test multi-column slices
-            execute("DELETE FROM %s WHERE partitionKey = ? AND (clustering_1, clustering_2) > (?, ?)", 2, 3, 3);
-            flush(forceFlush);
-            assertRows(execute("SELECT * FROM %s WHERE partitionKey = ?", 2),
-                       row(2, 3, 0, 65),
-                       row(2, 3, 1, 66),
-                       row(2, 3, 2, 67),
-                       row(2, 3, 3, 68));
-
-            execute("DELETE FROM %s WHERE partitionKey = ? AND (clustering_1, clustering_2) < (?, ?)", 2, 3, 1);
-            flush(forceFlush);
-            assertRows(execute("SELECT * FROM %s WHERE partitionKey = ?", 2),
-                       row(2, 3, 1, 66),
-                       row(2, 3, 2, 67),
-                       row(2, 3, 3, 68));
-
-            execute("DELETE FROM %s WHERE partitionKey = ? AND (clustering_1, clustering_2) >= (?, ?) AND (clustering_1) <= (?)", 2, 3, 2, 4);
-            flush(forceFlush);
-            assertRows(execute("SELECT * FROM %s WHERE partitionKey = ?", 2),
-                       row(2, 3, 1, 66));
-
-            // Test with a mix of single column and multi-column restrictions
-            execute("DELETE FROM %s WHERE partitionKey = ? AND clustering_1 = ? AND (clustering_2) < (?)", 3, 0, 3);
-            flush(forceFlush);
-            assertRows(execute("SELECT * FROM %s WHERE partitionKey = ? AND clustering_1 = ?", 3, 0),
-                       row(3, 0, 3, 78),
-                       row(3, 0, 4, 79));
-
-            execute("DELETE FROM %s WHERE partitionKey = ? AND clustering_1 IN (?, ?) AND (clustering_2) >= (?)", 3, 0, 1, 3);
-            flush(forceFlush);
-            assertRows(execute("SELECT * FROM %s WHERE partitionKey = ? AND clustering_1 IN (?, ?)", 3, 0, 1),
-                       row(3, 1, 0, 80),
-                       row(3, 1, 1, 81),
-                       row(3, 1, 2, 82));
-
-            execute("DELETE FROM %s WHERE partitionKey = ? AND (clustering_1) IN ((?), (?)) AND clustering_2 < ?", 3, 0, 1, 1);
-            flush(forceFlush);
-            assertRows(execute("SELECT * FROM %s WHERE partitionKey = ? AND clustering_1 IN (?, ?)", 3, 0, 1),
-                       row(3, 1, 1, 81),
-                       row(3, 1, 2, 82));
-
-            execute("DELETE FROM %s WHERE partitionKey = ? AND (clustering_1) = (?) AND clustering_2 >= ?", 3, 1, 2);
-            flush(forceFlush);
-            assertRows(execute("SELECT * FROM %s WHERE partitionKey = ? AND clustering_1 IN (?, ?)", 3, 0, 1),
-                       row(3, 1, 1, 81));
-
-            // Test invalid queries
-            assertInvalidMessage("Range deletions are not supported for specific columns",
-                                 "DELETE value FROM %s WHERE partitionKey = ? AND (clustering_1, clustering_2) >= (?, ?)", 2, 3, 1);
-            assertInvalidMessage("Range deletions are not supported for specific columns",
-                                 "DELETE value FROM %s WHERE partitionKey = ? AND clustering_1 >= ?", 2, 3);
-            assertInvalidMessage("Range deletions are not supported for specific columns",
-                                 "DELETE value FROM %s WHERE partitionKey = ? AND clustering_1 = ?", 2, 3);
-            assertInvalidMessage("Range deletions are not supported for specific columns",
-                                 "DELETE value FROM %s WHERE partitionKey = ?", 2);
-        }
+                    " PRIMARY KEY (partitionKey, clustering_1, clustering_2))");
+
+        int value = 0;
+        for (int partitionKey = 0; partitionKey < 5; partitionKey++)
+            for (int clustering1 = 0; clustering1 < 5; clustering1++)
+                for (int clustering2 = 0; clustering2 < 5; clustering2++)
+                {
+                    execute("INSERT INTO %s (partitionKey, clustering_1, clustering_2, value) VALUES (?, ?, ?, ?)",
+                            partitionKey, clustering1, clustering2, value++);
+                }
+        flush(forceFlush);
+
+        // test unspecified second clustering column
+        execute("DELETE FROM %s WHERE partitionKey = ? AND clustering_1 = ?", 0, 1);
+        flush(forceFlush);
+        assertRows(execute("SELECT * FROM %s WHERE partitionKey = ? AND clustering_1 < ?", 0, 2),
+                   row(0, 0, 0, 0),
+                   row(0, 0, 1, 1),
+                   row(0, 0, 2, 2),
+                   row(0, 0, 3, 3),
+                   row(0, 0, 4, 4));
+
+        // test delete partition
+        execute("DELETE FROM %s WHERE partitionKey = ?", 1);
+        flush(forceFlush);
+        assertEmpty(execute("SELECT * FROM %s WHERE partitionKey = ?", 1));
+
+        // test slices on the second clustering column
+
+        execute("DELETE FROM %s WHERE partitionKey = ? AND clustering_1 = ? AND clustering_2 < ?", 0, 0, 2);
+        flush(forceFlush);
+        assertRows(execute("SELECT * FROM %s WHERE partitionKey = ? AND clustering_1 < ?", 0, 2),
+                   row(0, 0, 2, 2),
+                   row(0, 0, 3, 3),
+                   row(0, 0, 4, 4));
+
+        execute("DELETE FROM %s WHERE partitionKey = ? AND clustering_1 = ? AND clustering_2 <= ?", 0, 0, 3);
+        flush(forceFlush);
+        assertRows(execute("SELECT * FROM %s WHERE partitionKey = ? AND clustering_1 < ?", 0, 2),
+                   row(0, 0, 4, 4));
+
+        execute("DELETE FROM %s WHERE partitionKey = ? AND  clustering_1 = ? AND clustering_2 > ? ", 0, 2, 2);
+        flush(forceFlush);
+        assertRows(execute("SELECT * FROM %s WHERE partitionKey = ? AND  clustering_1 = ?", 0, 2),
+                   row(0, 2, 0, 10),
+                   row(0, 2, 1, 11),
+                   row(0, 2, 2, 12));
+
+        execute("DELETE FROM %s WHERE partitionKey = ? AND  clustering_1 = ? AND clustering_2 >= ? ", 0, 2, 1);
+        flush(forceFlush);
+        assertRows(execute("SELECT * FROM %s WHERE partitionKey = ? AND  clustering_1 = ?", 0, 2),
+                   row(0, 2, 0, 10));
+
+        execute("DELETE FROM %s WHERE partitionKey = ? AND  clustering_1 = ? AND clustering_2 > ? AND clustering_2 < ? ",
+                0, 3, 1, 4);
+        flush(forceFlush);
+        assertRows(execute("SELECT * FROM %s WHERE partitionKey = ? AND  clustering_1 = ?", 0, 3),
+                   row(0, 3, 0, 15),
+                   row(0, 3, 1, 16),
+                   row(0, 3, 4, 19));
+
+        execute("DELETE FROM %s WHERE partitionKey = ? AND  clustering_1 = ? AND clustering_2 > ? AND clustering_2 < ? ",
+                0, 3, 4, 1);
+        flush(forceFlush);
+        assertRows(execute("SELECT * FROM %s WHERE partitionKey = ? AND  clustering_1 = ?", 0, 3),
+                   row(0, 3, 0, 15),
+                   row(0, 3, 1, 16),
+                   row(0, 3, 4, 19));
+
+        execute("DELETE FROM %s WHERE partitionKey = ? AND  clustering_1 = ? AND clustering_2 >= ? AND clustering_2 <= ? ",
+                0, 3, 1, 4);
+        flush(forceFlush);
+        assertRows(execute("SELECT * FROM %s WHERE partitionKey = ? AND  clustering_1 = ?", 0, 3),
+                   row(0, 3, 0, 15));
+
+        // test slices on the first clustering column
+
+        execute("DELETE FROM %s WHERE partitionKey = ? AND  clustering_1 >= ?", 0, 4);
+        flush(forceFlush);
+        assertRows(execute("SELECT * FROM %s WHERE partitionKey = ?", 0),
+                   row(0, 0, 4, 4),
+                   row(0, 2, 0, 10),
+                   row(0, 3, 0, 15));
+
+        execute("DELETE FROM %s WHERE partitionKey = ? AND  clustering_1 > ?", 0, 3);
+        flush(forceFlush);
+        assertRows(execute("SELECT * FROM %s WHERE partitionKey = ?", 0),
+                   row(0, 0, 4, 4),
+                   row(0, 2, 0, 10),
+                   row(0, 3, 0, 15));
+
+        execute("DELETE FROM %s WHERE partitionKey = ? AND clustering_1 < ?", 0, 3);
+        flush(forceFlush);
+        assertRows(execute("SELECT * FROM %s WHERE partitionKey = ?", 0),
+                   row(0, 3, 0, 15));
+
+        execute("DELETE FROM %s WHERE partitionKey = ? AND clustering_1 >= ? AND clustering_1 < ?", 2, 0, 3);
+        flush(forceFlush);
+        assertRows(execute("SELECT * FROM %s WHERE partitionKey = ?", 2),
+                   row(2, 3, 0, 65),
+                   row(2, 3, 1, 66),
+                   row(2, 3, 2, 67),
+                   row(2, 3, 3, 68),
+                   row(2, 3, 4, 69),
+                   row(2, 4, 0, 70),
+                   row(2, 4, 1, 71),
+                   row(2, 4, 2, 72),
+                   row(2, 4, 3, 73),
+                   row(2, 4, 4, 74));
+
+        execute("DELETE FROM %s WHERE partitionKey = ? AND clustering_1 > ? AND clustering_1 <= ?", 2, 3, 5);
+        flush(forceFlush);
+        assertRows(execute("SELECT * FROM %s WHERE partitionKey = ?", 2),
+                   row(2, 3, 0, 65),
+                   row(2, 3, 1, 66),
+                   row(2, 3, 2, 67),
+                   row(2, 3, 3, 68),
+                   row(2, 3, 4, 69));
+
+        execute("DELETE FROM %s WHERE partitionKey = ? AND clustering_1 < ? AND clustering_1 > ?", 2, 3, 5);
+        flush(forceFlush);
+        assertRows(execute("SELECT * FROM %s WHERE partitionKey = ?", 2),
+                   row(2, 3, 0, 65),
+                   row(2, 3, 1, 66),
+                   row(2, 3, 2, 67),
+                   row(2, 3, 3, 68),
+                   row(2, 3, 4, 69));
+
+        // test multi-column slices
+        execute("DELETE FROM %s WHERE partitionKey = ? AND (clustering_1, clustering_2) > (?, ?)", 2, 3, 3);
+        flush(forceFlush);
+        assertRows(execute("SELECT * FROM %s WHERE partitionKey = ?", 2),
+                   row(2, 3, 0, 65),
+                   row(2, 3, 1, 66),
+                   row(2, 3, 2, 67),
+                   row(2, 3, 3, 68));
+
+        execute("DELETE FROM %s WHERE partitionKey = ? AND (clustering_1, clustering_2) < (?, ?)", 2, 3, 1);
+        flush(forceFlush);
+        assertRows(execute("SELECT * FROM %s WHERE partitionKey = ?", 2),
+                   row(2, 3, 1, 66),
+                   row(2, 3, 2, 67),
+                   row(2, 3, 3, 68));
+
+        execute("DELETE FROM %s WHERE partitionKey = ? AND (clustering_1, clustering_2) >= (?, ?) AND (clustering_1) <= (?)", 2, 3, 2, 4);
+        flush(forceFlush);
+        assertRows(execute("SELECT * FROM %s WHERE partitionKey = ?", 2),
+                   row(2, 3, 1, 66));
+
+        // Test with a mix of single column and multi-column restrictions
+        execute("DELETE FROM %s WHERE partitionKey = ? AND clustering_1 = ? AND (clustering_2) < (?)", 3, 0, 3);
+        flush(forceFlush);
+        assertRows(execute("SELECT * FROM %s WHERE partitionKey = ? AND clustering_1 = ?", 3, 0),
+                   row(3, 0, 3, 78),
+                   row(3, 0, 4, 79));
+
+        execute("DELETE FROM %s WHERE partitionKey = ? AND clustering_1 IN (?, ?) AND (clustering_2) >= (?)", 3, 0, 1, 3);
+        flush(forceFlush);
+        assertRows(execute("SELECT * FROM %s WHERE partitionKey = ? AND clustering_1 IN (?, ?)", 3, 0, 1),
+                   row(3, 1, 0, 80),
+                   row(3, 1, 1, 81),
+                   row(3, 1, 2, 82));
+
+        execute("DELETE FROM %s WHERE partitionKey = ? AND (clustering_1) IN ((?), (?)) AND clustering_2 < ?", 3, 0, 1, 1);
+        flush(forceFlush);
+        assertRows(execute("SELECT * FROM %s WHERE partitionKey = ? AND clustering_1 IN (?, ?)", 3, 0, 1),
+                   row(3, 1, 1, 81),
+                   row(3, 1, 2, 82));
+
+        execute("DELETE FROM %s WHERE partitionKey = ? AND (clustering_1) = (?) AND clustering_2 >= ?", 3, 1, 2);
+        flush(forceFlush);
+        assertRows(execute("SELECT * FROM %s WHERE partitionKey = ? AND clustering_1 IN (?, ?)", 3, 0, 1),
+                   row(3, 1, 1, 81));
+
+        // Test invalid queries
+        assertInvalidMessage("Range deletions are not supported for specific columns",
+                             "DELETE value FROM %s WHERE partitionKey = ? AND (clustering_1, clustering_2) >= (?, ?)", 2, 3, 1);
+        assertInvalidMessage("Range deletions are not supported for specific columns",
+                             "DELETE value FROM %s WHERE partitionKey = ? AND clustering_1 >= ?", 2, 3);
+        assertInvalidMessage("Range deletions are not supported for specific columns",
+                             "DELETE value FROM %s WHERE partitionKey = ? AND clustering_1 = ?", 2, 3);
+        assertInvalidMessage("Range deletions are not supported for specific columns",
+                             "DELETE value FROM %s WHERE partitionKey = ?", 2);
     }
 
     @Test
@@ -1217,109 +1150,93 @@ public class DeleteTest extends CQLTester
     @Test
     public void testDeleteWithEmptyRestrictionValue() throws Throwable
     {
-        for (String options : new String[] { "", " WITH COMPACT STORAGE" })
-        {
-            createTable("CREATE TABLE %s (pk blob, c blob, v blob, PRIMARY KEY (pk, c))" + options);
+        createTable("CREATE TABLE %s (pk blob, c blob, v blob, PRIMARY KEY (pk, c))");
 
-            if (StringUtils.isEmpty(options))
-            {
-                execute("INSERT INTO %s (pk, c, v) VALUES (?, ?, ?)", bytes("foo123"), EMPTY_BYTE_BUFFER, bytes("1"));
-                execute("DELETE FROM %s WHERE pk = textAsBlob('foo123') AND c = textAsBlob('');");
+        execute("INSERT INTO %s (pk, c, v) VALUES (?, ?, ?)", bytes("foo123"), EMPTY_BYTE_BUFFER, bytes("1"));
+        execute("DELETE FROM %s WHERE pk = textAsBlob('foo123') AND c = textAsBlob('');");
 
-                assertEmpty(execute("SELECT * FROM %s"));
+        assertEmpty(execute("SELECT * FROM %s"));
 
-                execute("INSERT INTO %s (pk, c, v) VALUES (?, ?, ?)", bytes("foo123"), EMPTY_BYTE_BUFFER, bytes("1"));
-                execute("DELETE FROM %s WHERE pk = textAsBlob('foo123') AND c IN (textAsBlob(''), textAsBlob('1'));");
+        execute("INSERT INTO %s (pk, c, v) VALUES (?, ?, ?)", bytes("foo123"), EMPTY_BYTE_BUFFER, bytes("1"));
+        execute("DELETE FROM %s WHERE pk = textAsBlob('foo123') AND c IN (textAsBlob(''), textAsBlob('1'));");
 
-                assertEmpty(execute("SELECT * FROM %s"));
+        assertEmpty(execute("SELECT * FROM %s"));
 
-                execute("INSERT INTO %s (pk, c, v) VALUES (?, ?, ?)", bytes("foo123"), EMPTY_BYTE_BUFFER, bytes("1"));
-                execute("INSERT INTO %s (pk, c, v) VALUES (?, ?, ?)", bytes("foo123"), bytes("1"), bytes("1"));
-                execute("INSERT INTO %s (pk, c, v) VALUES (?, ?, ?)", bytes("foo123"), bytes("2"), bytes("2"));
+        execute("INSERT INTO %s (pk, c, v) VALUES (?, ?, ?)", bytes("foo123"), EMPTY_BYTE_BUFFER, bytes("1"));
+        execute("INSERT INTO %s (pk, c, v) VALUES (?, ?, ?)", bytes("foo123"), bytes("1"), bytes("1"));
+        execute("INSERT INTO %s (pk, c, v) VALUES (?, ?, ?)", bytes("foo123"), bytes("2"), bytes("2"));
 
-                execute("DELETE FROM %s WHERE pk = textAsBlob('foo123') AND c > textAsBlob('')");
+        execute("DELETE FROM %s WHERE pk = textAsBlob('foo123') AND c > textAsBlob('')");
 
-                assertRows(execute("SELECT * FROM %s"),
-                           row(bytes("foo123"), EMPTY_BYTE_BUFFER, bytes("1")));
+        assertRows(execute("SELECT * FROM %s"),
+                   row(bytes("foo123"), EMPTY_BYTE_BUFFER, bytes("1")));
 
-                execute("DELETE FROM %s WHERE pk = textAsBlob('foo123') AND c >= textAsBlob('')");
+        execute("DELETE FROM %s WHERE pk = textAsBlob('foo123') AND c >= textAsBlob('')");
 
-                assertEmpty(execute("SELECT * FROM %s"));
-            }
-            else
-            {
-                assertInvalid("Invalid empty or null value for column c",
-                              "DELETE FROM %s WHERE pk = textAsBlob('foo123') AND c = textAsBlob('')");
-                assertInvalid("Invalid empty or null value for column c",
-                              "DELETE FROM %s WHERE pk = textAsBlob('foo123') AND c IN (textAsBlob(''), textAsBlob('1'))");
-            }
+        assertEmpty(execute("SELECT * FROM %s"));
 
-            execute("INSERT INTO %s (pk, c, v) VALUES (?, ?, ?)", bytes("foo123"), bytes("1"), bytes("1"));
-            execute("INSERT INTO %s (pk, c, v) VALUES (?, ?, ?)", bytes("foo123"), bytes("2"), bytes("2"));
+        execute("INSERT INTO %s (pk, c, v) VALUES (?, ?, ?)", bytes("foo123"), bytes("1"), bytes("1"));
+        execute("INSERT INTO %s (pk, c, v) VALUES (?, ?, ?)", bytes("foo123"), bytes("2"), bytes("2"));
 
-            execute("DELETE FROM %s WHERE pk = textAsBlob('foo123') AND c > textAsBlob('')");
+        execute("DELETE FROM %s WHERE pk = textAsBlob('foo123') AND c > textAsBlob('')");
 
-            assertEmpty(execute("SELECT * FROM %s"));
+        assertEmpty(execute("SELECT * FROM %s"));
 
-            execute("INSERT INTO %s (pk, c, v) VALUES (?, ?, ?)", bytes("foo123"), bytes("1"), bytes("1"));
-            execute("INSERT INTO %s (pk, c, v) VALUES (?, ?, ?)", bytes("foo123"), bytes("2"), bytes("2"));
+        execute("INSERT INTO %s (pk, c, v) VALUES (?, ?, ?)", bytes("foo123"), bytes("1"), bytes("1"));
+        execute("INSERT INTO %s (pk, c, v) VALUES (?, ?, ?)", bytes("foo123"), bytes("2"), bytes("2"));
 
-            execute("DELETE FROM %s WHERE pk = textAsBlob('foo123') AND c <= textAsBlob('')");
-            execute("DELETE FROM %s WHERE pk = textAsBlob('foo123') AND c < textAsBlob('')");
+        execute("DELETE FROM %s WHERE pk = textAsBlob('foo123') AND c <= textAsBlob('')");
+        execute("DELETE FROM %s WHERE pk = textAsBlob('foo123') AND c < textAsBlob('')");
 
-            assertRows(execute("SELECT * FROM %s"),
-                       row(bytes("foo123"), bytes("1"), bytes("1")),
-                       row(bytes("foo123"), bytes("2"), bytes("2")));
-        }
+        assertRows(execute("SELECT * FROM %s"),
+                   row(bytes("foo123"), bytes("1"), bytes("1")),
+                   row(bytes("foo123"), bytes("2"), bytes("2")));
     }
 
     @Test
     public void testDeleteWithMultipleClusteringColumnsAndEmptyRestrictionValue() throws Throwable
     {
-        for (String options : new String[] { "", " WITH COMPACT STORAGE" })
-        {
-            createTable("CREATE TABLE %s (pk blob, c1 blob, c2 blob, v blob, PRIMARY KEY (pk, c1, c2))" + options);
+        createTable("CREATE TABLE %s (pk blob, c1 blob, c2 blob, v blob, PRIMARY KEY (pk, c1, c2))");
 
-            execute("INSERT INTO %s (pk, c1, c2, v) VALUES (?, ?, ?, ?)", bytes("foo123"), EMPTY_BYTE_BUFFER, bytes("1"), bytes("1"));
-            execute("DELETE FROM %s WHERE pk = textAsBlob('foo123') AND c1 = textAsBlob('');");
+        execute("INSERT INTO %s (pk, c1, c2, v) VALUES (?, ?, ?, ?)", bytes("foo123"), EMPTY_BYTE_BUFFER, bytes("1"), bytes("1"));
+        execute("DELETE FROM %s WHERE pk = textAsBlob('foo123') AND c1 = textAsBlob('');");
 
-            assertEmpty(execute("SELECT * FROM %s"));
+        assertEmpty(execute("SELECT * FROM %s"));
 
-            execute("INSERT INTO %s (pk, c1, c2, v) VALUES (?, ?, ?, ?)", bytes("foo123"), EMPTY_BYTE_BUFFER, bytes("1"), bytes("1"));
-            execute("DELETE FROM %s WHERE pk = textAsBlob('foo123') AND c1 IN (textAsBlob(''), textAsBlob('1')) AND c2 = textAsBlob('1');");
+        execute("INSERT INTO %s (pk, c1, c2, v) VALUES (?, ?, ?, ?)", bytes("foo123"), EMPTY_BYTE_BUFFER, bytes("1"), bytes("1"));
+        execute("DELETE FROM %s WHERE pk = textAsBlob('foo123') AND c1 IN (textAsBlob(''), textAsBlob('1')) AND c2 = textAsBlob('1');");
 
-            assertEmpty(execute("SELECT * FROM %s"));
+        assertEmpty(execute("SELECT * FROM %s"));
 
-            execute("INSERT INTO %s (pk, c1, c2, v) VALUES (?, ?, ?, ?)", bytes("foo123"), EMPTY_BYTE_BUFFER, bytes("1"), bytes("0"));
-            execute("INSERT INTO %s (pk, c1, c2, v) VALUES (?, ?, ?, ?)", bytes("foo123"), bytes("1"), bytes("1"), bytes("1"));
-            execute("INSERT INTO %s (pk, c1, c2, v) VALUES (?, ?, ?, ?)", bytes("foo123"), bytes("1"), bytes("2"), bytes("3"));
+        execute("INSERT INTO %s (pk, c1, c2, v) VALUES (?, ?, ?, ?)", bytes("foo123"), EMPTY_BYTE_BUFFER, bytes("1"), bytes("0"));
+        execute("INSERT INTO %s (pk, c1, c2, v) VALUES (?, ?, ?, ?)", bytes("foo123"), bytes("1"), bytes("1"), bytes("1"));
+        execute("INSERT INTO %s (pk, c1, c2, v) VALUES (?, ?, ?, ?)", bytes("foo123"), bytes("1"), bytes("2"), bytes("3"));
 
-            execute("DELETE FROM %s WHERE pk = textAsBlob('foo123') AND c1 > textAsBlob('')");
+        execute("DELETE FROM %s WHERE pk = textAsBlob('foo123') AND c1 > textAsBlob('')");
 
-            assertRows(execute("SELECT * FROM %s"),
-                       row(bytes("foo123"), EMPTY_BYTE_BUFFER, bytes("1"), bytes("0")));
+        assertRows(execute("SELECT * FROM %s"),
+                   row(bytes("foo123"), EMPTY_BYTE_BUFFER, bytes("1"), bytes("0")));
 
-            execute("DELETE FROM %s WHERE pk = textAsBlob('foo123') AND c1 >= textAsBlob('')");
+        execute("DELETE FROM %s WHERE pk = textAsBlob('foo123') AND c1 >= textAsBlob('')");
 
-            assertEmpty(execute("SELECT * FROM %s"));
+        assertEmpty(execute("SELECT * FROM %s"));
 
-            execute("INSERT INTO %s (pk, c1, c2, v) VALUES (?, ?, ?, ?)", bytes("foo123"), bytes("1"), bytes("1"), bytes("1"));
-            execute("INSERT INTO %s (pk, c1, c2, v) VALUES (?, ?, ?, ?)", bytes("foo123"), bytes("1"), bytes("2"), bytes("3"));
+        execute("INSERT INTO %s (pk, c1, c2, v) VALUES (?, ?, ?, ?)", bytes("foo123"), bytes("1"), bytes("1"), bytes("1"));
+        execute("INSERT INTO %s (pk, c1, c2, v) VALUES (?, ?, ?, ?)", bytes("foo123"), bytes("1"), bytes("2"), bytes("3"));
 
-            execute("DELETE FROM %s WHERE pk = textAsBlob('foo123') AND c1 > textAsBlob('')");
+        execute("DELETE FROM %s WHERE pk = textAsBlob('foo123') AND c1 > textAsBlob('')");
 
-            assertEmpty(execute("SELECT * FROM %s"));
+        assertEmpty(execute("SELECT * FROM %s"));
 
-            execute("INSERT INTO %s (pk, c1, c2, v) VALUES (?, ?, ?, ?)", bytes("foo123"), bytes("1"), bytes("1"), bytes("1"));
-            execute("INSERT INTO %s (pk, c1, c2, v) VALUES (?, ?, ?, ?)", bytes("foo123"), bytes("1"), bytes("2"), bytes("3"));
+        execute("INSERT INTO %s (pk, c1, c2, v) VALUES (?, ?, ?, ?)", bytes("foo123"), bytes("1"), bytes("1"), bytes("1"));
+        execute("INSERT INTO %s (pk, c1, c2, v) VALUES (?, ?, ?, ?)", bytes("foo123"), bytes("1"), bytes("2"), bytes("3"));
 
-            execute("DELETE FROM %s WHERE pk = textAsBlob('foo123') AND c1 <= textAsBlob('')");
-            execute("DELETE FROM %s WHERE pk = textAsBlob('foo123') AND c1 < textAsBlob('')");
+        execute("DELETE FROM %s WHERE pk = textAsBlob('foo123') AND c1 <= textAsBlob('')");
+        execute("DELETE FROM %s WHERE pk = textAsBlob('foo123') AND c1 < textAsBlob('')");
 
-            assertRows(execute("SELECT * FROM %s"),
-                       row(bytes("foo123"), bytes("1"), bytes("1"), bytes("1")),
-                       row(bytes("foo123"), bytes("1"), bytes("2"), bytes("3")));
-        }
+        assertRows(execute("SELECT * FROM %s"),
+                   row(bytes("foo123"), bytes("1"), bytes("1"), bytes("1")),
+                   row(bytes("foo123"), bytes("1"), bytes("2"), bytes("3")));
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/cassandra/blob/07fbd8ee/test/unit/org/apache/cassandra/cql3/validation/operations/InsertTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/InsertTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/InsertTest.java
index 488e1c7..0f01f3e 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/InsertTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/InsertTest.java
@@ -118,54 +118,6 @@ public class InsertTest extends CQLTester
     }
 
     @Test
-    public void testInsertWithCompactFormat() throws Throwable
-    {
-        testInsertWithCompactFormat(false);
-        testInsertWithCompactFormat(true);
-    }
-
-    private void testInsertWithCompactFormat(boolean forceFlush) throws Throwable
-    {
-        createTable("CREATE TABLE %s (partitionKey int," +
-                                      "clustering int," +
-                                      "value int," +
-                                      " PRIMARY KEY (partitionKey, clustering)) WITH COMPACT STORAGE");
-
-        execute("INSERT INTO %s (partitionKey, clustering, value) VALUES (0, 0, 0)");
-        execute("INSERT INTO %s (partitionKey, clustering, value) VALUES (0, 1, 1)");
-        flush(forceFlush);
-
-        assertRows(execute("SELECT * FROM %s"),
-                   row(0, 0, 0),
-                   row(0, 1, 1));
-
-        // Invalid Null values for the clustering key or the regular column
-        assertInvalidMessage("Some clustering keys are missing: clustering",
-                             "INSERT INTO %s (partitionKey, value) VALUES (0, 0)");
-        assertInvalidMessage("Column value is mandatory for this COMPACT STORAGE table",
-                             "INSERT INTO %s (partitionKey, clustering) VALUES (0, 0)");
-
-        // Missing primary key columns
-        assertInvalidMessage("Some partition key parts are missing: partitionkey",
-                             "INSERT INTO %s (clustering, value) VALUES (0, 1)");
-
-        // multiple time the same value
-        assertInvalidMessage("The column names contains duplicates",
-                             "INSERT INTO %s (partitionKey, clustering, value, value) VALUES (0, 0, 2, 2)");
-
-        // multiple time same primary key element in WHERE clause
-        assertInvalidMessage("The column names contains duplicates",
-                             "INSERT INTO %s (partitionKey, clustering, clustering, value) VALUES (0, 0, 0, 2)");
-
-        // unknown identifiers
-        assertInvalidMessage("Undefined column name clusteringx",
-                             "INSERT INTO %s (partitionKey, clusteringx, value) VALUES (0, 0, 2)");
-
-        assertInvalidMessage("Undefined column name valuex",
-                             "INSERT INTO %s (partitionKey, clustering, valuex) VALUES (0, 0, 2)");
-    }
-
-    @Test
     public void testInsertWithTwoClusteringColumns() throws Throwable
     {
         testInsertWithTwoClusteringColumns(false);
@@ -211,59 +163,6 @@ public class InsertTest extends CQLTester
     }
 
     @Test
-    public void testInsertWithCompactStorageAndTwoClusteringColumns() throws Throwable
-    {
-        testInsertWithCompactStorageAndTwoClusteringColumns(false);
-        testInsertWithCompactStorageAndTwoClusteringColumns(true);
-    }
-
-    private void testInsertWithCompactStorageAndTwoClusteringColumns(boolean forceFlush) throws Throwable
-    {
-        createTable("CREATE TABLE %s (partitionKey int," +
-                                      "clustering_1 int," +
-                                      "clustering_2 int," +
-                                      "value int," +
-                                      " PRIMARY KEY (partitionKey, clustering_1, clustering_2)) WITH COMPACT STORAGE");
-
-        execute("INSERT INTO %s (partitionKey, clustering_1, value) VALUES (0, 0, 0)");
-        execute("INSERT INTO %s (partitionKey, clustering_1, clustering_2, value) VALUES (0, 0, 0, 0)");
-        execute("INSERT INTO %s (partitionKey, clustering_1, clustering_2, value) VALUES (0, 0, 1, 1)");
-        flush(forceFlush);
-
-        assertRows(execute("SELECT * FROM %s"),
-                   row(0, 0, null, 0),
-                   row(0, 0, 0, 0),
-                   row(0, 0, 1, 1));
-
-        // Invalid Null values for the clustering key or the regular column
-        assertInvalidMessage("PRIMARY KEY column \"clustering_2\" cannot be restricted as preceding column \"clustering_1\" is not restricted",
-                             "INSERT INTO %s (partitionKey, clustering_2, value) VALUES (0, 0, 0)");
-        assertInvalidMessage("Column value is mandatory for this COMPACT STORAGE table",
-                             "INSERT INTO %s (partitionKey, clustering_1, clustering_2) VALUES (0, 0, 0)");
-
-        // Missing primary key columns
-        assertInvalidMessage("Some partition key parts are missing: partitionkey",
-                             "INSERT INTO %s (clustering_1, clustering_2, value) VALUES (0, 0, 1)");
-        assertInvalidMessage("PRIMARY KEY column \"clustering_2\" cannot be restricted as preceding column \"clustering_1\" is not restricted",
-                             "INSERT INTO %s (partitionKey, clustering_2, value) VALUES (0, 0, 2)");
-
-        // multiple time the same value
-        assertInvalidMessage("The column names contains duplicates",
-                             "INSERT INTO %s (partitionKey, clustering_1, value, clustering_2, value) VALUES (0, 0, 2, 0, 2)");
-
-        // multiple time same primary key element in WHERE clause
-        assertInvalidMessage("The column names contains duplicates",
-                             "INSERT INTO %s (partitionKey, clustering_1, clustering_1, clustering_2, value) VALUES (0, 0, 0, 0, 2)");
-
-        // unknown identifiers
-        assertInvalidMessage("Undefined column name clustering_1x",
-                             "INSERT INTO %s (partitionKey, clustering_1x, clustering_2, value) VALUES (0, 0, 0, 2)");
-
-        assertInvalidMessage("Undefined column name valuex",
-                             "INSERT INTO %s (partitionKey, clustering_1, clustering_2, valuex) VALUES (0, 0, 0, 2)");
-    }
-
-    @Test
     public void testInsertWithAStaticColumn() throws Throwable
     {
         testInsertWithAStaticColumn(false);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/07fbd8ee/test/unit/org/apache/cassandra/cql3/validation/operations/InsertUpdateIfConditionTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/InsertUpdateIfConditionTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/InsertUpdateIfConditionTest.java
index a0ae214..eea8f3b 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/InsertUpdateIfConditionTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/InsertUpdateIfConditionTest.java
@@ -545,24 +545,6 @@ public class InsertUpdateIfConditionTest extends CQLTester
                    row(true));
     }
 
-    /**
-     * Test for CAS with compact storage table, and #6813 in particular,
-     * migrated from cql_tests.py:TestCQL.cas_and_compact_test()
-     */
-    @Test
-    public void testCompactStorage() throws Throwable
-    {
-        createTable("CREATE TABLE %s (partition text, key text, owner text, PRIMARY KEY (partition, key) ) WITH COMPACT STORAGE");
-
-        execute("INSERT INTO %s (partition, key, owner) VALUES ('a', 'b', null)");
-        assertRows(execute("UPDATE %s SET owner='z' WHERE partition='a' AND key='b' IF owner=null"), row(true));
-
-        assertRows(execute("UPDATE %s SET owner='b' WHERE partition='a' AND key='b' IF owner='a'"), row(false, "z"));
-        assertRows(execute("UPDATE %s SET owner='b' WHERE partition='a' AND key='b' IF owner='z'"), row(true));
-
-        assertRows(execute("INSERT INTO %s (partition, key, owner) VALUES ('a', 'c', 'x') IF NOT EXISTS"), row(true));
-    }
-
     @Test
     public void testWholeUDT() throws Throwable
     {


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


[05/25] cassandra git commit: Allow dropping COMPACT STORAGE flag

Posted by if...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/test/unit/org/apache/cassandra/cql3/validation/operations/DropCompactStorageThriftTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/DropCompactStorageThriftTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/DropCompactStorageThriftTest.java
new file mode 100644
index 0000000..dde3e7b
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/DropCompactStorageThriftTest.java
@@ -0,0 +1,525 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.cql3.validation.operations;
+
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+
+import org.junit.Test;
+
+import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.cql3.ColumnSpecification;
+import org.apache.cassandra.cql3.UntypedResultSet;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.AsciiType;
+import org.apache.cassandra.db.marshal.BytesType;
+import org.apache.cassandra.db.marshal.CompositeType;
+import org.apache.cassandra.db.marshal.EmptyType;
+import org.apache.cassandra.db.marshal.Int32Type;
+import org.apache.cassandra.db.marshal.MapType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.locator.SimpleStrategy;
+import org.apache.cassandra.thrift.Cassandra;
+import org.apache.cassandra.thrift.CfDef;
+import org.apache.cassandra.thrift.Column;
+import org.apache.cassandra.thrift.ColumnDef;
+import org.apache.cassandra.thrift.ColumnOrSuperColumn;
+import org.apache.cassandra.thrift.ColumnParent;
+import org.apache.cassandra.thrift.IndexType;
+import org.apache.cassandra.thrift.KsDef;
+import org.apache.cassandra.thrift.Mutation;
+import org.apache.cassandra.thrift.SuperColumn;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+import static org.apache.cassandra.thrift.ConsistencyLevel.ONE;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+public class DropCompactStorageThriftTest extends ThriftCQLTester
+{
+    @Test
+    public void thriftCreatedTableTest() throws Throwable
+    {
+        final String KEYSPACE = "thrift_created_table_test_ks";
+        final String TABLE = "test_table_1";
+
+        CfDef cfDef = new CfDef().setDefault_validation_class(Int32Type.instance.toString())
+                                 .setKey_validation_class(AsciiType.instance.toString())
+                                 .setComparator_type(AsciiType.instance.toString())
+                                 .setColumn_metadata(Arrays.asList(new ColumnDef(ByteBufferUtil.bytes("col1"),
+                                                                                 AsciiType.instance.toString())
+                                                                   .setIndex_name("col1Index")
+                                                                   .setIndex_type(IndexType.KEYS),
+                                                                   new ColumnDef(ByteBufferUtil.bytes("col2"),
+                                                                                 AsciiType.instance.toString())
+                                                                   .setIndex_name("col2Index")
+                                                                   .setIndex_type(IndexType.KEYS)))
+                                 .setKeyspace(KEYSPACE)
+                                 .setName(TABLE);
+
+        KsDef ksDef = new KsDef(KEYSPACE,
+                                SimpleStrategy.class.getName(),
+                                Arrays.asList(cfDef));
+        ksDef.setStrategy_options(Collections.singletonMap("replication_factor", "1"));
+
+        Cassandra.Client client = getClient();
+        client.system_add_keyspace(ksDef);
+        client.set_keyspace(KEYSPACE);
+
+        client.insert(UTF8Type.instance.decompose("key1"),
+                      new ColumnParent(TABLE),
+                      getColumnForInsert(ByteBufferUtil.bytes("col1"), ByteBufferUtil.bytes("val1")),
+                      ONE);
+
+        client.insert(UTF8Type.instance.decompose("key1"),
+                      new ColumnParent(TABLE),
+                      getColumnForInsert(ByteBufferUtil.bytes("col2"), ByteBufferUtil.bytes("val2")),
+                      ONE);
+
+        client.insert(UTF8Type.instance.decompose("key1"),
+                      new ColumnParent(TABLE),
+                      getColumnForInsert(ByteBufferUtil.bytes("dynamicKey1"), ByteBufferUtil.bytes(100)),
+                      ONE);
+
+        client.insert(UTF8Type.instance.decompose("key1"),
+                      new ColumnParent(TABLE),
+                      getColumnForInsert(ByteBufferUtil.bytes("dynamicKey2"), ByteBufferUtil.bytes(200)),
+                      ONE);
+
+
+        execute(String.format("ALTER TABLE %s.%s DROP COMPACT STORAGE;", KEYSPACE, TABLE));
+        UntypedResultSet resultSet = execute(String.format("select * from %s.%s",
+                                                           KEYSPACE, TABLE));
+        assertColumnType(AsciiType.instance, resultSet, "key");
+        assertColumnType(AsciiType.instance, resultSet, "column1");
+        assertColumnType(Int32Type.instance, resultSet, "value");
+        assertColumnType(AsciiType.instance, resultSet, "col1");
+        assertColumnType(AsciiType.instance, resultSet, "col2");
+
+        assertRows(resultSet,
+                   row("key1", "dynamicKey1", "val1", "val2", 100),
+                   row("key1", "dynamicKey2", "val1", "val2", 200));
+    }
+
+    @Test
+    public void thriftStaticCompatTableTest() throws Throwable
+    {
+        String KEYSPACE = keyspace();
+        String TABLE = createTable("CREATE TABLE %s (key ascii PRIMARY KEY, val ascii) WITH COMPACT STORAGE");
+
+        Cassandra.Client client = getClient();
+        client.set_keyspace(KEYSPACE);
+
+        client.insert(UTF8Type.instance.decompose("key1"),
+                      new ColumnParent(TABLE),
+                      getColumnForInsert(ByteBufferUtil.bytes("val"), ByteBufferUtil.bytes("val1")),
+                      ONE);
+
+        client.insert(UTF8Type.instance.decompose("key1"),
+                      new ColumnParent(TABLE),
+                      getColumnForInsert(ByteBufferUtil.bytes("dynamicKey1"), ByteBufferUtil.bytes("dynamicValue1")),
+                      ONE);
+
+        client.insert(UTF8Type.instance.decompose("key1"),
+                      new ColumnParent(TABLE),
+                      getColumnForInsert(ByteBufferUtil.bytes("dynamicKey2"), ByteBufferUtil.bytes("dynamicValue2")),
+                      ONE);
+
+        execute(String.format("ALTER TABLE %s.%s DROP COMPACT STORAGE;", KEYSPACE, TABLE));
+        UntypedResultSet resultSet = execute(String.format("select * from %s.%s",
+                                                           KEYSPACE, TABLE));
+        assertColumnType(AsciiType.instance, resultSet, "key");
+        assertColumnType(UTF8Type.instance, resultSet, "column1");
+        assertColumnType(AsciiType.instance, resultSet, "val");
+        assertColumnType(BytesType.instance, resultSet, "value");
+
+        // Values are interpreted as bytes by default:
+        assertRows(resultSet,
+                   row("key1", "dynamicKey1", "val1", ByteBufferUtil.bytes("dynamicValue1")),
+                   row("key1", "dynamicKey2", "val1", ByteBufferUtil.bytes("dynamicValue2")));
+    }
+
+    @Test
+    public void testSparseCompactTableIndex() throws Throwable
+    {
+        createTable("CREATE TABLE %s (key ascii PRIMARY KEY, val ascii) WITH COMPACT STORAGE");
+
+        // Indexes are allowed only on the sparse compact tables
+        createIndex("CREATE INDEX ON %s(val)");
+        for (int i = 0; i < 10; i++)
+            execute("INSERT INTO %s (key, val) VALUES (?, ?)", Integer.toString(i), Integer.toString(i * 10));
+
+        alterTable("ALTER TABLE %s DROP COMPACT STORAGE");
+
+        assertRows(execute("SELECT * FROM %s WHERE val = '50'"),
+                   row("5", null, "50", null));
+        assertRows(execute("SELECT * FROM %s WHERE key = '5'"),
+                   row("5", null, "50", null));
+    }
+
+    @Test
+    public void thriftCompatTableTest() throws Throwable
+    {
+        String KEYSPACE = keyspace();
+        String TABLE = createTable("CREATE TABLE %s (pkey ascii, ckey ascii, PRIMARY KEY (pkey, ckey)) WITH COMPACT STORAGE");
+
+        Cassandra.Client client = getClient();
+        client.set_keyspace(KEYSPACE);
+
+        client.insert(UTF8Type.instance.decompose("key1"),
+                      new ColumnParent(TABLE),
+                      getColumnForInsert(ByteBufferUtil.bytes("ckeyValue1"), ByteBufferUtil.EMPTY_BYTE_BUFFER),
+                      ONE);
+
+        client.insert(UTF8Type.instance.decompose("key1"),
+                      new ColumnParent(TABLE),
+                      getColumnForInsert(ByteBufferUtil.bytes("ckeyValue2"), ByteBufferUtil.EMPTY_BYTE_BUFFER),
+                      ONE);
+
+        execute(String.format("ALTER TABLE %s.%s DROP COMPACT STORAGE;", KEYSPACE, TABLE));
+        UntypedResultSet resultSet = execute(String.format("select * from %s.%s",
+                                                           KEYSPACE, TABLE));
+        assertColumnType(AsciiType.instance, resultSet, "pkey");
+        assertColumnType(AsciiType.instance, resultSet, "ckey");
+        assertColumnType(EmptyType.instance, resultSet, "value");
+
+        // Value is always empty
+        assertRows(resultSet,
+                   row("key1", "ckeyValue1", ByteBufferUtil.EMPTY_BYTE_BUFFER),
+                   row("key1", "ckeyValue2", ByteBufferUtil.EMPTY_BYTE_BUFFER));
+    }
+
+    @Test
+    public void thriftDenseTableTest() throws Throwable
+    {
+        String KEYSPACE = keyspace();
+        String TABLE = createTable("CREATE TABLE %s (pkey text, ckey text, v text, PRIMARY KEY (pkey, ckey)) WITH COMPACT STORAGE");
+
+        Cassandra.Client client = getClient();
+        client.set_keyspace(KEYSPACE);
+
+        client.insert(UTF8Type.instance.decompose("key1"),
+                      new ColumnParent(TABLE),
+                      getColumnForInsert(ByteBufferUtil.bytes("ckey1"), ByteBufferUtil.bytes("cvalue1")),
+                      ONE);
+
+        client.insert(UTF8Type.instance.decompose("key1"),
+                      new ColumnParent(TABLE),
+                      getColumnForInsert(ByteBufferUtil.bytes("ckey2"), ByteBufferUtil.bytes("cvalue2")),
+                      ONE);
+
+        execute(String.format("ALTER TABLE %s.%s DROP COMPACT STORAGE;", KEYSPACE, TABLE));
+        UntypedResultSet resultSet = execute(String.format("select * from %s.%s",
+                                                           KEYSPACE, TABLE));
+        assertColumnType(UTF8Type.instance, resultSet, "pkey");
+        assertColumnType(UTF8Type.instance, resultSet, "ckey");
+        assertColumnType(UTF8Type.instance, resultSet, "v");
+
+        assertRows(resultSet,
+                   row("key1", "ckey1", "cvalue1"),
+                   row("key1", "ckey2", "cvalue2"));
+    }
+
+    @Test
+    public void thriftTableWithIntKey() throws Throwable
+    {
+        final String KEYSPACE = "thrift_table_with_int_key_ks";
+        final String TABLE = "test_table_1";
+
+        ByteBuffer columnName = ByteBufferUtil.bytes("columnname");
+        CfDef cfDef = new CfDef().setDefault_validation_class(UTF8Type.instance.toString())
+                                 .setKey_validation_class(BytesType.instance.toString())
+                                 .setComparator_type(BytesType.instance.toString())
+                                 .setColumn_metadata(Arrays.asList(new ColumnDef(columnName,
+                                                                                 Int32Type.instance.toString())
+                                                                   .setIndex_name("col1Index")
+                                                                   .setIndex_type(IndexType.KEYS)))
+                                 .setKeyspace(KEYSPACE)
+                                 .setName(TABLE);
+
+        KsDef ksDef = new KsDef(KEYSPACE,
+                                SimpleStrategy.class.getName(),
+                                Arrays.asList(cfDef));
+        ksDef.setStrategy_options(Collections.singletonMap("replication_factor", "1"));
+
+        Cassandra.Client client = getClient();
+        client.system_add_keyspace(ksDef);
+        client.set_keyspace(KEYSPACE);
+
+        client.insert(UTF8Type.instance.decompose("key1"),
+                      new ColumnParent(TABLE),
+                      getColumnForInsert(columnName, ByteBufferUtil.bytes(100)),
+                      ONE);
+
+        execute(String.format("ALTER TABLE %s.%s DROP COMPACT STORAGE;", KEYSPACE, TABLE));
+        UntypedResultSet resultSet = execute(String.format("select * from %s.%s",
+                                                           KEYSPACE, TABLE));
+
+        assertEquals(resultSet.metadata()
+                              .stream()
+                              .filter((cs) -> cs.name.toString().equals(BytesType.instance.getString(columnName)))
+                              .findFirst()
+                              .get().type,
+                     Int32Type.instance);
+
+        assertRows(resultSet,
+                   row(UTF8Type.instance.decompose("key1"), null, 100, null));
+    }
+
+    @Test
+    public void thriftCompatTableWithSupercolumnsTest() throws Throwable
+    {
+        final String KEYSPACE = "thrift_compact_table_with_supercolumns_test";
+        final String TABLE = "test_table_1";
+
+        CfDef cfDef = new CfDef().setColumn_type("Super")
+                                 .setSubcomparator_type(Int32Type.instance.toString())
+                                 .setComparator_type(AsciiType.instance.toString())
+                                 .setDefault_validation_class(AsciiType.instance.toString())
+                                 .setKey_validation_class(AsciiType.instance.toString())
+                                 .setKeyspace(KEYSPACE)
+                                 .setName(TABLE);
+
+        KsDef ksDef = new KsDef(KEYSPACE,
+                                SimpleStrategy.class.getName(),
+                                Arrays.asList(cfDef));
+        ksDef.setStrategy_options(Collections.singletonMap("replication_factor", "1"));
+
+        Cassandra.Client client = getClient();
+        client.system_add_keyspace(ksDef);
+
+        client.set_keyspace(KEYSPACE);
+
+        Mutation mutation = new Mutation();
+        ColumnOrSuperColumn csoc = new ColumnOrSuperColumn();
+        csoc.setSuper_column(getSuperColumnForInsert(ByteBufferUtil.bytes("val1"),
+                                                     Arrays.asList(getColumnForInsert(ByteBufferUtil.bytes(1), ByteBufferUtil.bytes("value1")),
+                                                                   getColumnForInsert(ByteBufferUtil.bytes(2), ByteBufferUtil.bytes("value2")),
+                                                                   getColumnForInsert(ByteBufferUtil.bytes(3), ByteBufferUtil.bytes("value3")))));
+        mutation.setColumn_or_supercolumn(csoc);
+
+        Mutation mutation2 = new Mutation();
+        ColumnOrSuperColumn csoc2 = new ColumnOrSuperColumn();
+        csoc2.setSuper_column(getSuperColumnForInsert(ByteBufferUtil.bytes("val2"),
+                                                     Arrays.asList(getColumnForInsert(ByteBufferUtil.bytes(4), ByteBufferUtil.bytes("value7")),
+                                                                   getColumnForInsert(ByteBufferUtil.bytes(5), ByteBufferUtil.bytes("value8")),
+                                                                   getColumnForInsert(ByteBufferUtil.bytes(6), ByteBufferUtil.bytes("value9")))));
+        mutation2.setColumn_or_supercolumn(csoc2);
+
+        client.batch_mutate(Collections.singletonMap(ByteBufferUtil.bytes("key1"),
+                                                     Collections.singletonMap(TABLE, Arrays.asList(mutation, mutation2))),
+                            ONE);
+
+        execute(String.format("ALTER TABLE %s.%s DROP COMPACT STORAGE;", KEYSPACE, TABLE));
+        UntypedResultSet resultSet = execute(String.format("select * from %s.%s",
+                                                           KEYSPACE, TABLE));
+        assertColumnType(AsciiType.instance, resultSet, "key");
+        assertColumnType(AsciiType.instance, resultSet, "column1");
+        assertColumnType(MapType.getInstance(Int32Type.instance, AsciiType.instance, true), resultSet, "");
+
+        assertRows(resultSet,
+                   row("key1", "val1", map(1, "value1", 2, "value2", 3, "value3")),
+                   row("key1", "val2", map(4, "value7", 5, "value8", 6, "value9")));
+
+        assertRows(execute(String.format("SELECT \"\" FROM %s.%s;", KEYSPACE, TABLE)),
+                   row(map(1, "value1", 2, "value2", 3, "value3")),
+                   row(map(4, "value7", 5, "value8", 6, "value9")));
+
+        assertInvalidMessage("Range deletions are not supported for specific columns",
+                             String.format("DELETE \"\" FROM %s.%s WHERE key=?;", KEYSPACE, TABLE),
+                             "key1");
+
+        execute(String.format("TRUNCATE %s.%s;", KEYSPACE, TABLE));
+
+        execute(String.format("INSERT INTO %s.%s (key, column1, \"\") VALUES (?, ?, ?);", KEYSPACE, TABLE),
+                "key3", "val1", map(7, "value7", 8, "value8"));
+
+        assertRows(execute(String.format("SELECT \"\" FROM %s.%s;", KEYSPACE, TABLE)),
+                   row(map(7, "value7", 8, "value8")));
+    }
+
+    @Test
+    public void thriftCreatedTableWithCompositeColumnsTest() throws Throwable
+    {
+        final String KEYSPACE = "thrift_created_table_with_composites_test_ks";
+        final String TABLE = "test_table_1";
+
+        CompositeType type = CompositeType.getInstance(AsciiType.instance, AsciiType.instance, AsciiType.instance);
+        CfDef cfDef = new CfDef().setDefault_validation_class(AsciiType.instance.toString())
+                                 .setComparator_type(type.toString())
+                                 .setKey_validation_class(AsciiType.instance.toString())
+                                 .setKeyspace(KEYSPACE)
+                                 .setName(TABLE);
+
+        KsDef ksDef = new KsDef(KEYSPACE,
+                                SimpleStrategy.class.getName(),
+                                Arrays.asList(cfDef));
+        ksDef.setStrategy_options(Collections.singletonMap("replication_factor", "1"));
+
+        Cassandra.Client client = getClient();
+        client.system_add_keyspace(ksDef);
+        client.set_keyspace(KEYSPACE);
+
+        client.insert(UTF8Type.instance.decompose("key1"),
+                      new ColumnParent(TABLE),
+                      getColumnForInsert(type.decompose("a", "b", "c"), ByteBufferUtil.bytes("val1")),
+                      ONE);
+
+        client.insert(UTF8Type.instance.decompose("key1"),
+                      new ColumnParent(TABLE),
+                      getColumnForInsert(type.decompose("d", "e", "f"), ByteBufferUtil.bytes("val2")),
+                      ONE);
+
+        execute(String.format("ALTER TABLE %s.%s DROP COMPACT STORAGE;", KEYSPACE, TABLE));
+        UntypedResultSet resultSet = execute(String.format("select * from %s.%s",
+                                                           KEYSPACE, TABLE));
+
+        assertColumnType(AsciiType.instance, resultSet, "key");
+        assertColumnType(AsciiType.instance, resultSet, "column1");
+        assertColumnType(AsciiType.instance, resultSet, "column2");
+        assertColumnType(AsciiType.instance, resultSet, "column3");
+        assertColumnType(AsciiType.instance, resultSet, "value");
+
+        assertRows(resultSet,
+                   row("key1", "a", "b", "c", "val1"),
+                   row("key1", "d", "e", "f", "val2"));
+    }
+
+    @Test
+    public void compactTableWithoutClusteringKeyTest() throws Throwable
+    {
+        String KEYSPACE = keyspace();
+        String TABLE = createTable("CREATE TABLE %s (pkey text PRIMARY KEY, s1 text, s2 text) WITH COMPACT STORAGE");
+
+        Cassandra.Client client = getClient();
+        client.set_keyspace(KEYSPACE);
+
+        client.insert(UTF8Type.instance.decompose("key1"),
+                      new ColumnParent(TABLE),
+                      getColumnForInsert(ByteBufferUtil.bytes("ckey1"), ByteBufferUtil.bytes("val1")),
+                      ONE);
+
+        client.insert(UTF8Type.instance.decompose("key1"),
+                      new ColumnParent(TABLE),
+                      getColumnForInsert(ByteBufferUtil.bytes("ckey2"), ByteBufferUtil.bytes("val2")),
+                      ONE);
+
+        client.insert(UTF8Type.instance.decompose("key1"),
+                      new ColumnParent(TABLE),
+                      getColumnForInsert(ByteBufferUtil.bytes("s1"), ByteBufferUtil.bytes("s1Val")),
+                      ONE);
+
+        client.insert(UTF8Type.instance.decompose("key1"),
+                      new ColumnParent(TABLE),
+                      getColumnForInsert(ByteBufferUtil.bytes("s2"), ByteBufferUtil.bytes("s2Val")),
+                      ONE);
+
+        execute(String.format("ALTER TABLE %s.%s DROP COMPACT STORAGE;", KEYSPACE, TABLE));
+        UntypedResultSet resultSet = execute(String.format("select * from %s.%s",
+                                                           KEYSPACE, TABLE));
+
+        assertColumnType(UTF8Type.instance, resultSet, "pkey");
+        assertColumnType(UTF8Type.instance, resultSet, "s1");
+        assertColumnType(UTF8Type.instance, resultSet, "s2");
+        assertColumnType(UTF8Type.instance, resultSet, "column1");
+        assertColumnType(BytesType.instance, resultSet, "value");
+
+        assertRows(resultSet,
+                   row("key1", "ckey1", "s1Val", "s2Val", ByteBufferUtil.bytes("val1")),
+                   row("key1", "ckey2", "s1Val", "s2Val", ByteBufferUtil.bytes("val2")));
+    }
+
+    @Test
+    public void denseTableTestTest() throws Throwable
+    {
+        String KEYSPACE = keyspace();
+        String TABLE = createTable("CREATE TABLE %s (pkey text PRIMARY KEY, s text) WITH COMPACT STORAGE");
+
+        Cassandra.Client client = getClient();
+        client.set_keyspace(KEYSPACE);
+
+        client.insert(UTF8Type.instance.decompose("key1"),
+                      new ColumnParent(TABLE),
+                      getColumnForInsert(ByteBufferUtil.bytes("ckey1"), ByteBufferUtil.bytes("val1")),
+                      ONE);
+
+        client.insert(UTF8Type.instance.decompose("key1"),
+                      new ColumnParent(TABLE),
+                      getColumnForInsert(ByteBufferUtil.bytes("ckey2"), ByteBufferUtil.bytes("val2")),
+                      ONE);
+
+        client.insert(UTF8Type.instance.decompose("key1"),
+                      new ColumnParent(TABLE),
+                      getColumnForInsert(ByteBufferUtil.bytes("s"), ByteBufferUtil.bytes("sval1")),
+                      ONE);
+
+        client.insert(UTF8Type.instance.decompose("key1"),
+                      new ColumnParent(TABLE),
+                      getColumnForInsert(ByteBufferUtil.bytes("s"), ByteBufferUtil.bytes("sval2")),
+                      ONE);
+
+        // `s` becomes static, `column1` becomes a clustering key, `value` becomes visible
+        execute(String.format("ALTER TABLE %s.%s DROP COMPACT STORAGE;", KEYSPACE, TABLE));
+        UntypedResultSet resultSet = execute(String.format("select * from %s.%s",
+                                                           KEYSPACE, TABLE));
+        assertColumnType(UTF8Type.instance, resultSet, "pkey");
+        assertColumnType(UTF8Type.instance, resultSet, "s");
+        assertColumnType(UTF8Type.instance, resultSet, "column1");
+        assertColumnType(BytesType.instance, resultSet, "value");
+
+        assertRows(resultSet,
+                   row("key1", "ckey1", "sval2", ByteBufferUtil.bytes("val1")),
+                   row("key1", "ckey2", "sval2", ByteBufferUtil.bytes("val2")));
+    }
+
+    private Column getColumnForInsert(ByteBuffer columnName, ByteBuffer value)
+    {
+        Column column = new Column();
+        column.setName(columnName);
+        column.setValue(value);
+        column.setTimestamp(System.currentTimeMillis());
+        return column;
+    }
+
+    private SuperColumn getSuperColumnForInsert(ByteBuffer columnName, List<Column> columns)
+    {
+        SuperColumn column = new SuperColumn();
+        column.setName(columnName);
+        for (Column c : columns)
+            column.addToColumns(c);
+        return column;
+    }
+
+    private static void assertColumnType(AbstractType t, UntypedResultSet resultSet, String columnName)
+    {
+        for (ColumnSpecification columnSpecification : resultSet.metadata())
+        {
+            if (columnSpecification.name.toString().equals(columnName))
+            {
+                assertEquals(t, columnSpecification.type);
+                return;
+            }
+        }
+
+        fail(String.format("Could not find a column with name '%s'", columnName));
+    }
+}


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


[14/25] cassandra git commit: Merge branch 'cassandra-3.0' into cassandra-3.11

Posted by if...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d429cd0/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java
index fdbcf7a,a5fa12d..de42647
--- a/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java
@@@ -87,12 -93,16 +85,12 @@@ public class AlterTableStatement extend
          if (meta.isView())
              throw new InvalidRequestException("Cannot use ALTER TABLE on Materialized View");
  
-         CFMetaData cfm = meta.copy();
+         CFMetaData cfm;
 -
 -        CQL3Type validator = this.validator == null ? null : this.validator.prepare(keyspace());
          ColumnIdentifier columnName = null;
          ColumnDefinition def = null;
 -        if (rawColumnName != null)
 -        {
 -            columnName = rawColumnName.prepare(meta);
 -            def = meta.getColumnDefinition(columnName);
 -        }
 +        CQL3Type.Raw dataType = null;
 +        boolean isStatic = false;
 +        CQL3Type validator = null;
  
          List<ViewDefinition> viewUpdates = null;
          Iterable<ViewDefinition> views = View.findAll(keyspace(), columnFamily());
@@@ -102,146 -112,141 +100,156 @@@
              case ALTER:
                  throw new InvalidRequestException("Altering of types is not allowed");
              case ADD:
-                 if (cfm.isDense())
 -                assert columnName != null;
+                 if (meta.isDense())
                      throw new InvalidRequestException("Cannot add new column to a COMPACT STORAGE table");
  
+                 cfm = meta.copy();
+ 
 -                if (isStatic)
 +                for (AlterTableStatementColumn colData : colNameList)
                  {
 -                    if (!cfm.isCompound())
 -                        throw new InvalidRequestException("Static columns are not allowed in COMPACT STORAGE tables");
 -                    if (cfm.clusteringColumns().isEmpty())
 -                        throw new InvalidRequestException("Static columns are only useful (and thus allowed) if the table has at least one clustering column");
 -                }
 +                    columnName = colData.getColumnName().getIdentifier(cfm);
 +                    def = cfm.getColumnDefinition(columnName);
 +                    dataType = colData.getColumnType();
 +                    assert dataType != null;
 +                    isStatic = colData.getStaticType();
 +                    validator = dataType.prepare(keyspace());
  
 -                if (def != null)
 -                {
 -                    switch (def.kind)
 +
 +                    if (isStatic)
                      {
 -                        case PARTITION_KEY:
 -                        case CLUSTERING:
 -                            throw new InvalidRequestException(String.format("Invalid column name %s because it conflicts with a PRIMARY KEY part", columnName));
 -                        default:
 -                            throw new InvalidRequestException(String.format("Invalid column name %s because it conflicts with an existing column", columnName));
 +                        if (!cfm.isCompound())
 +                            throw new InvalidRequestException("Static columns are not allowed in COMPACT STORAGE tables");
 +                        if (cfm.clusteringColumns().isEmpty())
 +                            throw new InvalidRequestException("Static columns are only useful (and thus allowed) if the table has at least one clustering column");
                      }
 -                }
  
 -                // Cannot re-add a dropped counter column. See #7831.
 -                if (meta.isCounter() && meta.getDroppedColumns().containsKey(columnName.bytes))
 -                    throw new InvalidRequestException(String.format("Cannot re-add previously dropped counter column %s", columnName));
 +                    if (def != null)
 +                    {
 +                        switch (def.kind)
 +                        {
 +                            case PARTITION_KEY:
 +                            case CLUSTERING:
 +                                throw new InvalidRequestException(String.format("Invalid column name %s because it conflicts with a PRIMARY KEY part", columnName));
 +                            default:
 +                                throw new InvalidRequestException(String.format("Invalid column name %s because it conflicts with an existing column", columnName));
 +                        }
 +                    }
  
 -                AbstractType<?> type = validator.getType();
 -                if (type.isCollection() && type.isMultiCell())
 -                {
 -                    if (!cfm.isCompound())
 -                        throw new InvalidRequestException("Cannot use non-frozen collections in COMPACT STORAGE tables");
 -                    if (cfm.isSuper())
 -                        throw new InvalidRequestException("Cannot use non-frozen collections with super column families");
 -
 -                    // If there used to be a non-frozen collection column with the same name (that has been dropped),
 -                    // we could still have some data using the old type, and so we can't allow adding a collection
 -                    // with the same name unless the types are compatible (see #6276).
 -                    CFMetaData.DroppedColumn dropped = cfm.getDroppedColumns().get(columnName.bytes);
 -                    if (dropped != null && dropped.type instanceof CollectionType
 -                        && dropped.type.isMultiCell() && !type.isCompatibleWith(dropped.type))
 +                    // Cannot re-add a dropped counter column. See #7831.
 +                    if (meta.isCounter() && meta.getDroppedColumns().containsKey(columnName.bytes))
 +                        throw new InvalidRequestException(String.format("Cannot re-add previously dropped counter column %s", columnName));
 +
 +                    AbstractType<?> type = validator.getType();
 +                    if (type.isCollection() && type.isMultiCell())
                      {
 -                        String message =
 -                            String.format("Cannot add a collection with the name %s because a collection with the same name"
 -                                          + " and a different type (%s) has already been used in the past",
 -                                          columnName,
 -                                          dropped.type.asCQL3Type());
 -                        throw new InvalidRequestException(message);
 +                        if (!cfm.isCompound())
 +                            throw new InvalidRequestException("Cannot use non-frozen collections in COMPACT STORAGE tables");
 +                        if (cfm.isSuper())
 +                            throw new InvalidRequestException("Cannot use non-frozen collections with super column families");
 +
 +                        // If there used to be a non-frozen collection column with the same name (that has been dropped),
 +                        // we could still have some data using the old type, and so we can't allow adding a collection
 +                        // with the same name unless the types are compatible (see #6276).
 +                        CFMetaData.DroppedColumn dropped = cfm.getDroppedColumns().get(columnName.bytes);
 +                        if (dropped != null && dropped.type instanceof CollectionType
 +                            && dropped.type.isMultiCell() && !type.isCompatibleWith(dropped.type))
 +                        {
 +                            String message =
 +                                String.format("Cannot add a collection with the name %s because a collection with the same name"
 +                                              + " and a different type (%s) has already been used in the past",
 +                                              columnName,
 +                                              dropped.type.asCQL3Type());
 +                            throw new InvalidRequestException(message);
 +                        }
                      }
 -                }
  
 -                cfm.addColumnDefinition(isStatic
 -                                        ? ColumnDefinition.staticDef(cfm, columnName.bytes, type)
 -                                        : ColumnDefinition.regularDef(cfm, columnName.bytes, type));
 +                    cfm.addColumnDefinition(isStatic
 +                                            ? ColumnDefinition.staticDef(cfm, columnName.bytes, type)
 +                                            : ColumnDefinition.regularDef(cfm, columnName.bytes, type));
  
 -                // Adding a column to a table which has an include all view requires the column to be added to the view
 -                // as well
 -                if (!isStatic)
 -                {
 -                    for (ViewDefinition view : views)
 +                    // Adding a column to a table which has an include all view requires the column to be added to the view
 +                    // as well
 +                    if (!isStatic)
                      {
 -                        if (view.includeAllColumns)
 +                        for (ViewDefinition view : views)
                          {
 -                            ViewDefinition viewCopy = view.copy();
 -                            viewCopy.metadata.addColumnDefinition(ColumnDefinition.regularDef(viewCopy.metadata, columnName.bytes, type));
 -                            if (viewUpdates == null)
 -                                viewUpdates = new ArrayList<>();
 -                            viewUpdates.add(viewCopy);
 +                            if (view.includeAllColumns)
 +                            {
 +                                ViewDefinition viewCopy = view.copy();
 +                                viewCopy.metadata.addColumnDefinition(ColumnDefinition.regularDef(viewCopy.metadata, columnName.bytes, type));
 +                                if (viewUpdates == null)
 +                                    viewUpdates = new ArrayList<>();
 +                                viewUpdates.add(viewCopy);
 +                            }
                          }
                      }
                  }
                  break;
  
              case DROP:
-                 if (!cfm.isCQLTable())
 -                assert columnName != null;
+                 if (!meta.isCQLTable())
                      throw new InvalidRequestException("Cannot drop columns from a non-CQL3 table");
  
 -                if (def == null)
 -                    throw new InvalidRequestException(String.format("Column %s was not found in table %s", columnName, columnFamily()));
 -
+                 cfm = meta.copy();
+ 
 -                switch (def.kind)
 +                for (AlterTableStatementColumn colData : colNameList)
                  {
 -                    case PARTITION_KEY:
 -                    case CLUSTERING:
 -                        throw new InvalidRequestException(String.format("Cannot drop PRIMARY KEY part %s", columnName));
 -                    case REGULAR:
 -                    case STATIC:
 -                        ColumnDefinition toDelete = null;
 -                        for (ColumnDefinition columnDef : cfm.partitionColumns())
 -                        {
 -                            if (columnDef.name.equals(columnName))
 -                            {
 -                                toDelete = columnDef;
 -                                break;
 -                            }
 -                        }
 -                        assert toDelete != null;
 -                        cfm.removeColumnDefinition(toDelete);
 -                        cfm.recordColumnDrop(toDelete, deleteTimestamp == null ? queryState.getTimestamp() : deleteTimestamp);
 -                        break;
 -                }
 +                    columnName = colData.getColumnName().getIdentifier(cfm);
 +                    def = cfm.getColumnDefinition(columnName);
  
 -                // If the dropped column is required by any secondary indexes
 -                // we reject the operation, as the indexes must be dropped first
 -                Indexes allIndexes = cfm.getIndexes();
 -                if (!allIndexes.isEmpty())
 -                {
 -                    ColumnFamilyStore store = Keyspace.openAndGetStore(cfm);
 -                    Set<IndexMetadata> dependentIndexes = store.indexManager.getDependentIndexes(def);
 -                    if (!dependentIndexes.isEmpty())
 -                        throw new InvalidRequestException(String.format("Cannot drop column %s because it has " +
 -                                                                        "dependent secondary indexes (%s)",
 -                                                                        def,
 -                                                                        dependentIndexes.stream()
 -                                                                                        .map(i -> i.name)
 -                                                                                        .collect(Collectors.joining(","))));
 -                }
 +                    if (def == null)
 +                        throw new InvalidRequestException(String.format("Column %s was not found in table %s", columnName, columnFamily()));
  
 -                if (!Iterables.isEmpty(views))
 -                    throw new InvalidRequestException(String.format("Cannot drop column %s on base table with materialized views.",
 -                                                                    columnName.toString(),
 -                                                                    keyspace()));
 +                    switch (def.kind)
 +                    {
 +                         case PARTITION_KEY:
 +                         case CLUSTERING:
 +                              throw new InvalidRequestException(String.format("Cannot drop PRIMARY KEY part %s", columnName));
 +                         case REGULAR:
 +                         case STATIC:
 +                              ColumnDefinition toDelete = null;
 +                              for (ColumnDefinition columnDef : cfm.partitionColumns())
 +                              {
 +                                   if (columnDef.name.equals(columnName))
 +                                   {
 +                                       toDelete = columnDef;
 +                                       break;
 +                                   }
 +                               }
 +                             assert toDelete != null;
 +                             cfm.removeColumnDefinition(toDelete);
 +                             cfm.recordColumnDrop(toDelete, deleteTimestamp  == null ? queryState.getTimestamp() : deleteTimestamp);
 +                             break;
 +                    }
 +
 +                    // If the dropped column is required by any secondary indexes
 +                    // we reject the operation, as the indexes must be dropped first
 +                    Indexes allIndexes = cfm.getIndexes();
 +                    if (!allIndexes.isEmpty())
 +                    {
 +                        ColumnFamilyStore store = Keyspace.openAndGetStore(cfm);
 +                        Set<IndexMetadata> dependentIndexes = store.indexManager.getDependentIndexes(def);
 +                        if (!dependentIndexes.isEmpty())
 +                            throw new InvalidRequestException(String.format("Cannot drop column %s because it has " +
 +                                                                            "dependent secondary indexes (%s)",
 +                                                                            def,
 +                                                                            dependentIndexes.stream()
 +                                                                                            .map(i -> i.name)
 +                                                                                            .collect(Collectors.joining(","))));
 +                    }
 +
 +                    if (!Iterables.isEmpty(views))
 +                        throw new InvalidRequestException(String.format("Cannot drop column %s on base table with materialized views.",
 +                                                                        columnName.toString(),
 +                                                                        keyspace()));
 +                }
                  break;
+             case DROP_COMPACT_STORAGE:
+                 if (!meta.isCompactTable())
+                     throw new InvalidRequestException("Cannot DROP COMPACT STORAGE on table without COMPACT STORAGE");
+ 
+                 cfm = meta.asNonCompact();
+                 break;
              case OPTS:
                  if (attrs == null)
                      throw new InvalidRequestException("ALTER TABLE WITH invoked, but no parameters found");
@@@ -265,10 -272,13 +275,12 @@@
  
                  break;
              case RENAME:
+                 cfm = meta.copy();
+ 
 -                for (Map.Entry<ColumnIdentifier.Raw, ColumnIdentifier> entry : renames.entrySet())
 +                for (Map.Entry<ColumnDefinition.Raw, ColumnDefinition.Raw> entry : renames.entrySet())
                  {
 -                    ColumnIdentifier from = entry.getKey().prepare(cfm);
 -                    ColumnIdentifier to = entry.getValue();
 -
 +                    ColumnIdentifier from = entry.getKey().getIdentifier(cfm);
 +                    ColumnIdentifier to = entry.getValue().getIdentifier(cfm);
                      cfm.renameColumn(from, to);
  
                      // If the view includes a renamed column, it must be renamed in the view table and the definition.

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d429cd0/src/java/org/apache/cassandra/cql3/statements/AuthenticationStatement.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d429cd0/src/java/org/apache/cassandra/cql3/statements/AuthorizationStatement.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d429cd0/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d429cd0/src/java/org/apache/cassandra/cql3/statements/CreateAggregateStatement.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d429cd0/src/java/org/apache/cassandra/cql3/statements/CreateIndexStatement.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/statements/CreateIndexStatement.java
index 204edf4,47d54fe..88afc6b
--- a/src/java/org/apache/cassandra/cql3/statements/CreateIndexStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/CreateIndexStatement.java
@@@ -107,18 -102,24 +107,23 @@@ public class CreateIndexStatement exten
              if (cd == null)
                  throw new InvalidRequestException("No column definition found for column " + target.column);
  
 +            if (cd.type.referencesDuration())
 +            {
 +                checkFalse(cd.type.isCollection(), "Secondary indexes are not supported on collections containing durations");
 +                checkFalse(cd.type.isTuple(), "Secondary indexes are not supported on tuples containing durations");
 +                checkFalse(cd.type.isUDT(), "Secondary indexes are not supported on UDTs containing durations");
 +                throw invalidRequest("Secondary indexes are not supported on duration columns");
 +            }
 +
              // TODO: we could lift that limitation
-             if (cfm.isCompactTable() && cd.isPrimaryKeyColumn())
-                 throw new InvalidRequestException("Secondary indexes are not supported on PRIMARY KEY columns in COMPACT STORAGE tables");
+             if (cfm.isCompactTable())
+             {
+                 if (cd.isPrimaryKeyColumn())
+                     throw new InvalidRequestException("Secondary indexes are not supported on PRIMARY KEY columns in COMPACT STORAGE tables");
+                 if (cfm.compactValueColumn().equals(cd))
+                     throw new InvalidRequestException("Secondary indexes are not supported on compact value column of COMPACT STORAGE tables");
+             }
  
 -            // It would be possible to support 2ndary index on static columns (but not without modifications of at least ExtendedFilter and
 -            // CompositesIndex) and maybe we should, but that means a query like:
 -            //     SELECT * FROM foo WHERE static_column = 'bar'
 -            // would pull the full partition every time the static column of partition is 'bar', which sounds like offering a
 -            // fair potential for foot-shooting, so I prefer leaving that to a follow up ticket once we have identified cases where
 -            // such indexing is actually useful.
 -            if (!cfm.isCompactTable() && cd.isStatic())
 -                throw new InvalidRequestException("Secondary indexes are not allowed on static columns");
 -
              if (cd.kind == ColumnDefinition.Kind.PARTITION_KEY && cfm.getKeyValidatorAsClusteringComparator().size() == 1)
                  throw new InvalidRequestException(String.format("Cannot create secondary index on partition key column %s", target.column));
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d429cd0/src/java/org/apache/cassandra/cql3/statements/CreateTableStatement.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d429cd0/src/java/org/apache/cassandra/cql3/statements/CreateViewStatement.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d429cd0/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
index c8c7570,8ae4d64..56d47b3
--- a/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
@@@ -27,9 -27,9 +27,10 @@@ import org.slf4j.LoggerFactory
  import org.apache.cassandra.auth.Permission;
  import org.apache.cassandra.config.CFMetaData;
  import org.apache.cassandra.config.ColumnDefinition;
++import org.apache.cassandra.config.Schema;
 +import org.apache.cassandra.config.ColumnDefinition.Raw;
  import org.apache.cassandra.config.ViewDefinition;
  import org.apache.cassandra.cql3.*;
 -import org.apache.cassandra.cql3.ColumnIdentifier.Raw;
  import org.apache.cassandra.cql3.functions.Function;
  import org.apache.cassandra.cql3.restrictions.StatementRestrictions;
  import org.apache.cassandra.cql3.selection.Selection;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d429cd0/src/java/org/apache/cassandra/cql3/statements/ParsedStatement.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d429cd0/src/java/org/apache/cassandra/cql3/statements/SchemaAlteringStatement.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d429cd0/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
index 33a69e7,1e867bc..d86a47d
--- a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
@@@ -945,17 -885,16 +945,17 @@@ public class SelectStatement implement
              this.selectClause = selectClause;
              this.whereClause = whereClause;
              this.limit = limit;
 +            this.perPartitionLimit = perPartitionLimit;
          }
  
-         public ParsedStatement.Prepared prepare() throws InvalidRequestException
+         public ParsedStatement.Prepared prepare(ClientState clientState) throws InvalidRequestException
          {
-             return prepare(false);
+             return prepare(false, clientState);
          }
  
-         public ParsedStatement.Prepared prepare(boolean forView) throws InvalidRequestException
+         public ParsedStatement.Prepared prepare(boolean forView, ClientState clientState) throws InvalidRequestException
          {
-             CFMetaData cfm = ThriftValidation.validateColumnFamily(keyspace(), columnFamily());
+             CFMetaData cfm = ThriftValidation.validateColumnFamilyWithCompactMode(keyspace(), columnFamily(), clientState.isNoCompactMode());
              VariableSpecifications boundNames = getBoundVariables();
  
              Selection selection = selectClause.isEmpty()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d429cd0/src/java/org/apache/cassandra/cql3/statements/TruncateStatement.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d429cd0/src/java/org/apache/cassandra/cql3/statements/UseStatement.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d429cd0/src/java/org/apache/cassandra/db/view/View.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d429cd0/src/java/org/apache/cassandra/index/internal/keys/KeysSearcher.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d429cd0/src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java
index 9e42101,39f7339..694fe37
--- a/src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java
@@@ -555,51 -527,24 +555,51 @@@ public class CQLSSTableWriter implement
              }
          }
  
 -        @SuppressWarnings("resource")
 -        public CQLSSTableWriter build()
 +        private Types createTypes(String keyspace)
          {
 -            if (directory == null)
 -                throw new IllegalStateException("No ouptut directory specified, you should provide a directory with inDirectory()");
 -            if (schema == null)
 -                throw new IllegalStateException("Missing schema, you should provide the schema for the SSTable to create with forTable()");
 -            if (insert == null)
 -                throw new IllegalStateException("No insert statement specified, you should provide an insert statement through using()");
 +            Types.RawBuilder builder = Types.rawBuilder(keyspace);
 +            for (CreateTypeStatement st : typeStatements)
 +                st.addToRawBuilder(builder);
 +            return builder.build();
 +        }
  
 -            AbstractSSTableSimpleWriter writer = sorted
 -                                               ? new SSTableSimpleWriter(directory, schema, insert.updatedColumns())
 -                                               : new SSTableSimpleUnsortedWriter(directory, schema, insert.updatedColumns(), bufferSizeInMB);
 +        /**
 +         * Creates the table according to schema statement
 +         *
 +         * @param types types this table should be created with
 +         */
 +        private CFMetaData createTable(Types types)
 +        {
 +            CreateTableStatement statement = (CreateTableStatement) schemaStatement.prepare(types).statement;
 +            statement.validate(ClientState.forInternalCalls());
  
 -            if (formatType != null)
 -                writer.setSSTableFormatType(formatType);
 +            CFMetaData cfMetaData = statement.getCFMetaData();
 +
 +            if (partitioner != null)
 +                return cfMetaData.copy(partitioner);
 +            else
 +                return cfMetaData;
 +        }
 +
 +        /**
 +         * Prepares insert statement for writing data to SSTable
 +         *
 +         * @return prepared Insert statement and it's bound names
 +         */
 +        private Pair<UpdateStatement, List<ColumnSpecification>> prepareInsert()
 +        {
-             ParsedStatement.Prepared cqlStatement = insertStatement.prepare();
++            ParsedStatement.Prepared cqlStatement = insertStatement.prepare(ClientState.forInternalCalls());
 +            UpdateStatement insert = (UpdateStatement) cqlStatement.statement;
 +            insert.validate(ClientState.forInternalCalls());
 +
 +            if (insert.hasConditions())
 +                throw new IllegalArgumentException("Conditional statements are not supported");
 +            if (insert.isCounter())
 +                throw new IllegalArgumentException("Counter update statements are not supported");
 +            if (cqlStatement.boundNames.isEmpty())
 +                throw new IllegalArgumentException("Provided insert statement has no bind variables");
  
 -            return new CQLSSTableWriter(writer, insert, boundNames);
 +            return Pair.create(insert, cqlStatement.boundNames);
          }
      }
  }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d429cd0/src/java/org/apache/cassandra/repair/RepairRunnable.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/repair/RepairRunnable.java
index c9eed54,77726d4..52cc29d
--- a/src/java/org/apache/cassandra/repair/RepairRunnable.java
+++ b/src/java/org/apache/cassandra/repair/RepairRunnable.java
@@@ -389,8 -385,8 +390,8 @@@ public class RepairRunnable extends Wra
                      throw new Exception("no tracestate");
  
                  String format = "select event_id, source, activity from %s.%s where session_id = ? and event_id > ? and event_id < ?;";
 -                String query = String.format(format, TraceKeyspace.NAME, TraceKeyspace.EVENTS);
 +                String query = String.format(format, SchemaConstants.TRACE_KEYSPACE_NAME, TraceKeyspace.EVENTS);
-                 SelectStatement statement = (SelectStatement) QueryProcessor.parseStatement(query).prepare().statement;
+                 SelectStatement statement = (SelectStatement) QueryProcessor.parseStatement(query).prepare(ClientState.forInternalCalls()).statement;
  
                  ByteBuffer sessionIdBytes = ByteBufferUtil.bytes(sessionId);
                  InetAddress source = FBUtilities.getBroadcastAddress();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d429cd0/src/java/org/apache/cassandra/service/ClientState.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d429cd0/src/java/org/apache/cassandra/thrift/ThriftValidation.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d429cd0/src/java/org/apache/cassandra/transport/messages/StartupMessage.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/transport/messages/StartupMessage.java
index bf4a619,774be6a..eb82292
--- a/src/java/org/apache/cassandra/transport/messages/StartupMessage.java
+++ b/src/java/org/apache/cassandra/transport/messages/StartupMessage.java
@@@ -35,7 -35,7 +35,8 @@@ public class StartupMessage extends Mes
  {
      public static final String CQL_VERSION = "CQL_VERSION";
      public static final String COMPRESSION = "COMPRESSION";
 +    public static final String PROTOCOL_VERSIONS = "PROTOCOL_VERSIONS";
+     public static final String NO_COMPACT = "NO_COMPACT";
  
      public static final Message.Codec<StartupMessage> codec = new Message.Codec<StartupMessage>()
      {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d429cd0/test/unit/org/apache/cassandra/cql3/ViewTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/cql3/ViewTest.java
index 7717a4d,136ae1c..eb9c855
--- a/test/unit/org/apache/cassandra/cql3/ViewTest.java
+++ b/test/unit/org/apache/cassandra/cql3/ViewTest.java
@@@ -48,8 -48,9 +48,10 @@@ import org.apache.cassandra.db.Keyspace
  import org.apache.cassandra.db.SystemKeyspace;
  import org.apache.cassandra.db.compaction.CompactionManager;
  import org.apache.cassandra.db.marshal.AsciiType;
+ import org.apache.cassandra.exceptions.InvalidRequestException;
+ import org.apache.cassandra.exceptions.SyntaxException;
  import org.apache.cassandra.schema.KeyspaceParams;
 +import org.apache.cassandra.transport.ProtocolVersion;
  import org.apache.cassandra.utils.FBUtilities;
  
  import static org.junit.Assert.assertEquals;
@@@ -1407,4 -1344,49 +1408,17 @@@ public class ViewTest extends CQLTeste
  
          assertRows(execute("SELECT count(*) FROM mv_test"), row(1024L));
      }
+ 
 -    @Test
 -    public void testFrozenCollectionsWithComplicatedInnerType() throws Throwable
 -    {
 -        createTable("CREATE TABLE %s (k int, intval int,  listval frozen<list<tuple<text,text>>>, PRIMARY KEY (k))");
 -
 -        execute("USE " + keyspace());
 -        executeNet(protocolVersion, "USE " + keyspace());
 -
 -        createView("mv",
 -                   "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE k IS NOT NULL AND listval IS NOT NULL PRIMARY KEY (k, listval)");
 -
 -        updateView("INSERT INTO %s (k, intval, listval) VALUES (?, ?, fromJson(?))",
 -                   0,
 -                   0,
 -                   "[[\"a\", \"1\"], [\"b\", \"2\"], [\"c\", \"3\"]]");
 -
 -        // verify input
 -        assertRows(execute("SELECT k, toJson(listval) FROM %s WHERE k = ?", 0),
 -                   row(0, "[[\"a\", \"1\"], [\"b\", \"2\"], [\"c\", \"3\"]]"));
 -        assertRows(execute("SELECT k, toJson(listval) from mv"),
 -                   row(0, "[[\"a\", \"1\"], [\"b\", \"2\"], [\"c\", \"3\"]]"));
 -
 -        // update listval with the same value and it will be compared in view generator
 -        updateView("INSERT INTO %s (k, listval) VALUES (?, fromJson(?))",
 -                   0,
 -                   "[[\"a\", \"1\"], [\"b\", \"2\"], [\"c\", \"3\"]]");
 -        // verify result
 -        assertRows(execute("SELECT k, toJson(listval) FROM %s WHERE k = ?", 0),
 -                   row(0, "[[\"a\", \"1\"], [\"b\", \"2\"], [\"c\", \"3\"]]"));
 -        assertRows(execute("SELECT k, toJson(listval) from mv"),
 -                   row(0, "[[\"a\", \"1\"], [\"b\", \"2\"], [\"c\", \"3\"]]"));
 -    }
+ 
+     @Test(expected = SyntaxException.class)
+     public void emptyViewNameTest() throws Throwable
+     {
+         execute("CREATE MATERIALIZED VIEW \"\" AS SELECT a, b FROM tbl WHERE b IS NOT NULL PRIMARY KEY (b, a)");
+     }
+ 
+      @Test(expected = SyntaxException.class)
+      public void emptyBaseTableNameTest() throws Throwable
+      {
+          execute("CREATE MATERIALIZED VIEW myview AS SELECT a, b FROM \"\" WHERE b IS NOT NULL PRIMARY KEY (b, a)");
+      }
  }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d429cd0/test/unit/org/apache/cassandra/cql3/validation/entities/UFTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/cql3/validation/entities/UFTest.java
index af9ec1a,6e6af19..6f3616c
--- a/test/unit/org/apache/cassandra/cql3/validation/entities/UFTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/UFTest.java
@@@ -36,11 -34,13 +36,12 @@@ import org.apache.cassandra.cql3.functi
  import org.apache.cassandra.cql3.functions.UDFunction;
  import org.apache.cassandra.db.marshal.CollectionType;
  import org.apache.cassandra.exceptions.InvalidRequestException;
+ import org.apache.cassandra.exceptions.SyntaxException;
  import org.apache.cassandra.schema.KeyspaceMetadata;
  import org.apache.cassandra.service.ClientState;
 -import org.apache.cassandra.transport.Event;
 -import org.apache.cassandra.transport.Server;
 +import org.apache.cassandra.transport.*;
 +import org.apache.cassandra.transport.ProtocolVersion;
  import org.apache.cassandra.transport.messages.ResultMessage;
 -import org.apache.cassandra.utils.ByteBufferUtil;
  
  public class UFTest extends CQLTester
  {
@@@ -872,222 -863,134 +873,246 @@@
      }
  
      @Test
 -    public void testEmptyString() throws Throwable
 +    public void testArgumentGenerics() throws Throwable
      {
          createTable("CREATE TABLE %s (key int primary key, sval text, aval ascii, bval blob, empty_int int)");
 -        execute("INSERT INTO %s (key, sval, aval, bval, empty_int) VALUES (?, ?, ?, ?, blobAsInt(0x))", 1, "", "", ByteBuffer.allocate(0));
  
 -        String fNameSRC = createFunction(KEYSPACE_PER_TEST, "text",
 -                                         "CREATE OR REPLACE FUNCTION %s(val text) " +
 -                                         "CALLED ON NULL INPUT " +
 -                                         "RETURNS text " +
 -                                         "LANGUAGE JAVA\n" +
 -                                         "AS 'return val;'");
 +        String typeName = createType("CREATE TYPE %s (txt text, i int)");
  
 -        String fNameSCC = createFunction(KEYSPACE_PER_TEST, "text",
 -                                         "CREATE OR REPLACE FUNCTION %s(val text) " +
 -                                         "CALLED ON NULL INPUT " +
 -                                         "RETURNS text " +
 -                                         "LANGUAGE JAVA\n" +
 -                                         "AS 'return \"\";'");
 +        createFunction(KEYSPACE, "map<text,bigint>,list<text>",
 +                       "CREATE FUNCTION IF NOT EXISTS %s(state map<text,bigint>, styles list<text>)\n" +
 +                       "  RETURNS NULL ON NULL INPUT\n" +
 +                       "  RETURNS map<text,bigint>\n" +
 +                       "  LANGUAGE java\n" +
 +                       "  AS $$\n" +
 +                       "    for (String style : styles) {\n" +
 +                       "      if (state.containsKey(style)) {\n" +
 +                       "        state.put(style, state.get(style) + 1L);\n" +
 +                       "      } else {\n" +
 +                       "        state.put(style, 1L);\n" +
 +                       "      }\n" +
 +                       "    }\n" +
 +                       "    return state;\n" +
 +                       "  $$");
 +
 +        createFunction(KEYSPACE, "text",
 +                                  "CREATE OR REPLACE FUNCTION %s("                 +
 +                                  "  listText list<text>,"                         +
 +                                  "  setText set<text>,"                           +
 +                                  "  mapTextInt map<text, int>,"                   +
 +                                  "  mapListTextSetInt map<frozen<list<text>>, frozen<set<int>>>," +
 +                                  "  mapTextTuple map<text, frozen<tuple<int, text>>>," +
 +                                  "  mapTextType map<text, frozen<" + typeName + ">>" +
 +                                  ") "                                             +
 +                                  "CALLED ON NULL INPUT "                          +
 +                                  "RETURNS map<frozen<list<text>>, frozen<set<int>>> " +
 +                                  "LANGUAGE JAVA\n"                                +
 +                                  "AS $$" +
 +                                  "     for (String s : listtext) {};" +
 +                                  "     for (String s : settext) {};" +
 +                                  "     for (String s : maptextint.keySet()) {};" +
 +                                  "     for (Integer s : maptextint.values()) {};" +
 +                                  "     for (java.util.List<String> l : maplisttextsetint.keySet()) {};" +
 +                                  "     for (java.util.Set<Integer> s : maplisttextsetint.values()) {};" +
 +                                  "     for (com.datastax.driver.core.TupleValue t : maptexttuple.values()) {};" +
 +                                  "     for (com.datastax.driver.core.UDTValue u : maptexttype.values()) {};" +
 +                                  "     return maplisttextsetint;" +
 +                                  "$$");
 +    }
  
 -        String fNameSRN = createFunction(KEYSPACE_PER_TEST, "text",
 -                                         "CREATE OR REPLACE FUNCTION %s(val text) " +
 -                                         "RETURNS NULL ON NULL INPUT " +
 -                                         "RETURNS text " +
 -                                         "LANGUAGE JAVA\n" +
 -                                         "AS 'return val;'");
 +    @Test
 +    public void testArgAndReturnTypes() throws Throwable
 +    {
  
 -        String fNameSCN = createFunction(KEYSPACE_PER_TEST, "text",
 -                                         "CREATE OR REPLACE FUNCTION %s(val text) " +
 -                                         "RETURNS NULL ON NULL INPUT " +
 -                                         "RETURNS text " +
 -                                         "LANGUAGE JAVA\n" +
 -                                         "AS 'return \"\";'");
 -
 -        String fNameBRC = createFunction(KEYSPACE_PER_TEST, "blob",
 -                                         "CREATE OR REPLACE FUNCTION %s(val blob) " +
 -                                         "CALLED ON NULL INPUT " +
 -                                         "RETURNS blob " +
 -                                         "LANGUAGE JAVA\n" +
 -                                         "AS 'return val;'");
 -
 -        String fNameBCC = createFunction(KEYSPACE_PER_TEST, "blob",
 -                                         "CREATE OR REPLACE FUNCTION %s(val blob) " +
 -                                         "CALLED ON NULL INPUT " +
 -                                         "RETURNS blob " +
 -                                         "LANGUAGE JAVA\n" +
 -                                         "AS 'return ByteBuffer.allocate(0);'");
 -
 -        String fNameBRN = createFunction(KEYSPACE_PER_TEST, "blob",
 -                                         "CREATE OR REPLACE FUNCTION %s(val blob) " +
 -                                         "RETURNS NULL ON NULL INPUT " +
 -                                         "RETURNS blob " +
 -                                         "LANGUAGE JAVA\n" +
 -                                         "AS 'return val;'");
 +        String type = KEYSPACE + '.' + createType("CREATE TYPE %s (txt text, i int)");
  
 -        String fNameBCN = createFunction(KEYSPACE_PER_TEST, "blob",
 -                                         "CREATE OR REPLACE FUNCTION %s(val blob) " +
 -                                         "RETURNS NULL ON NULL INPUT " +
 -                                         "RETURNS blob " +
 -                                         "LANGUAGE JAVA\n" +
 -                                         "AS 'return ByteBuffer.allocate(0);'");
 +        createTable("CREATE TABLE %s (key int primary key, udt frozen<" + type + ">)");
 +        execute("INSERT INTO %s (key, udt) VALUES (1, {txt: 'foo', i: 42})");
  
 -        String fNameIRC = createFunction(KEYSPACE_PER_TEST, "int",
 -                                         "CREATE OR REPLACE FUNCTION %s(val int) " +
 -                                         "CALLED ON NULL INPUT " +
 -                                         "RETURNS int " +
 -                                         "LANGUAGE JAVA\n" +
 -                                         "AS 'return val;'");
 +        // Java UDFs
  
 -        String fNameICC = createFunction(KEYSPACE_PER_TEST, "int",
 -                                         "CREATE OR REPLACE FUNCTION %s(val int) " +
 -                                         "CALLED ON NULL INPUT " +
 -                                         "RETURNS int " +
 -                                         "LANGUAGE JAVA\n" +
 -                                         "AS 'return 0;'");
 +        String f = createFunction(KEYSPACE, "int",
 +                                  "CREATE OR REPLACE FUNCTION %s(val int) " +
 +                                  "RETURNS NULL ON NULL INPUT " +
 +                                  "RETURNS " + type + ' ' +
 +                                  "LANGUAGE JAVA\n" +
 +                                  "AS 'return udfContext.newReturnUDTValue();';");
 +
 +        assertRows(execute("SELECT " + f + "(key) FROM %s"),
 +                   row(userType("txt", null, "i", null)));
 +
 +        f = createFunction(KEYSPACE, "int",
 +                           "CREATE OR REPLACE FUNCTION %s(val " + type + ") " +
 +                           "RETURNS NULL ON NULL INPUT " +
 +                           "RETURNS " + type + ' ' +
 +                           "LANGUAGE JAVA\n" +
 +                           "AS $$" +
 +                           "   com.datastax.driver.core.UDTValue udt = udfContext.newArgUDTValue(\"val\");" +
 +                           "   udt.setString(\"txt\", \"baz\");" +
 +                           "   udt.setInt(\"i\", 88);" +
 +                           "   return udt;" +
 +                           "$$;");
 +
 +        assertRows(execute("SELECT " + f + "(udt) FROM %s"),
 +                   row(userType("txt", "baz", "i", 88)));
 +
 +        f = createFunction(KEYSPACE, "int",
 +                           "CREATE OR REPLACE FUNCTION %s(val " + type + ") " +
 +                           "RETURNS NULL ON NULL INPUT " +
 +                           "RETURNS tuple<text, int>" +
 +                           "LANGUAGE JAVA\n" +
 +                           "AS $$" +
 +                           "   com.datastax.driver.core.TupleValue tv = udfContext.newReturnTupleValue();" +
 +                           "   tv.setString(0, \"baz\");" +
 +                           "   tv.setInt(1, 88);" +
 +                           "   return tv;" +
 +                           "$$;");
 +
 +        assertRows(execute("SELECT " + f + "(udt) FROM %s"),
 +                   row(tuple("baz", 88)));
 +
 +        // JavaScript UDFs
 +
 +        f = createFunction(KEYSPACE, "int",
 +                           "CREATE OR REPLACE FUNCTION %s(val int) " +
 +                           "RETURNS NULL ON NULL INPUT " +
 +                           "RETURNS " + type + ' ' +
 +                           "LANGUAGE JAVASCRIPT\n" +
 +                           "AS $$" +
 +                           "   udt = udfContext.newReturnUDTValue();" +
 +                           "   udt;" +
 +                           "$$;");
 +
 +        assertRows(execute("SELECT " + f + "(key) FROM %s"),
 +                   row(userType("txt", null, "i", null)));
 +
 +        f = createFunction(KEYSPACE, "int",
 +                           "CREATE OR REPLACE FUNCTION %s(val " + type + ") " +
 +                           "RETURNS NULL ON NULL INPUT " +
 +                           "RETURNS " + type + ' ' +
 +                           "LANGUAGE JAVASCRIPT\n" +
 +                           "AS $$" +
 +                           "   udt = udfContext.newArgUDTValue(0);" +
 +                           "   udt.setString(\"txt\", \"baz\");" +
 +                           "   udt.setInt(\"i\", 88);" +
 +                           "   udt;" +
 +                           "$$;");
 +
 +        assertRows(execute("SELECT " + f + "(udt) FROM %s"),
 +                   row(userType("txt", "baz", "i", 88)));
 +
 +        f = createFunction(KEYSPACE, "int",
 +                           "CREATE OR REPLACE FUNCTION %s(val " + type + ") " +
 +                           "RETURNS NULL ON NULL INPUT " +
 +                           "RETURNS tuple<text, int>" +
 +                           "LANGUAGE JAVASCRIPT\n" +
 +                           "AS $$" +
 +                           "   tv = udfContext.newReturnTupleValue();" +
 +                           "   tv.setString(0, \"baz\");" +
 +                           "   tv.setInt(1, 88);" +
 +                           "   tv;" +
 +                           "$$;");
 +
 +        assertRows(execute("SELECT " + f + "(udt) FROM %s"),
 +                   row(tuple("baz", 88)));
 +
 +        createFunction(KEYSPACE, "map",
 +                       "CREATE FUNCTION %s(my_map map<text, text>)\n" +
 +                       "         CALLED ON NULL INPUT\n" +
 +                       "         RETURNS text\n" +
 +                       "         LANGUAGE java\n" +
 +                       "         AS $$\n" +
 +                       "             String buffer = \"\";\n" +
 +                       "             for(java.util.Map.Entry<String, String> entry: my_map.entrySet()) {\n" +
 +                       "                 buffer = buffer + entry.getKey() + \": \" + entry.getValue() + \", \";\n" +
 +                       "             }\n" +
 +                       "             return buffer;\n" +
 +                       "         $$;\n");
 +    }
  
 -        String fNameIRN = createFunction(KEYSPACE_PER_TEST, "int",
 -                                         "CREATE OR REPLACE FUNCTION %s(val int) " +
 -                                         "RETURNS NULL ON NULL INPUT " +
 -                                         "RETURNS int " +
 -                                         "LANGUAGE JAVA\n" +
 -                                         "AS 'return val;'");
 +    @Test
 +    public void testImportJavaUtil() throws Throwable
 +    {
 +        createFunction(KEYSPACE, "list<text>",
 +                "CREATE OR REPLACE FUNCTION %s(listText list<text>) "                                             +
 +                        "CALLED ON NULL INPUT "                          +
 +                        "RETURNS set<text> " +
 +                        "LANGUAGE JAVA\n"                                +
 +                        "AS $$\n" +
 +                        "     Set<String> set = new HashSet<String>(); " +
 +                        "     for (String s : listtext) {" +
 +                        "            set.add(s);" +
 +                        "     }" +
 +                        "     return set;" +
 +                        "$$");
  
 -        String fNameICN = createFunction(KEYSPACE_PER_TEST, "int",
 -                                         "CREATE OR REPLACE FUNCTION %s(val int) " +
 -                                         "RETURNS NULL ON NULL INPUT " +
 -                                         "RETURNS int " +
 -                                         "LANGUAGE JAVA\n" +
 -                                         "AS 'return 0;'");
 -
 -        assertRows(execute("SELECT " + fNameSRC + "(sval) FROM %s"), row(""));
 -        assertRows(execute("SELECT " + fNameSRN + "(sval) FROM %s"), row(""));
 -        assertRows(execute("SELECT " + fNameSCC + "(sval) FROM %s"), row(""));
 -        assertRows(execute("SELECT " + fNameSCN + "(sval) FROM %s"), row(""));
 -        assertRows(execute("SELECT " + fNameSRC + "(aval) FROM %s"), row(""));
 -        assertRows(execute("SELECT " + fNameSRN + "(aval) FROM %s"), row(""));
 -        assertRows(execute("SELECT " + fNameSCC + "(aval) FROM %s"), row(""));
 -        assertRows(execute("SELECT " + fNameSCN + "(aval) FROM %s"), row(""));
 -        assertRows(execute("SELECT " + fNameBRC + "(bval) FROM %s"), row(ByteBufferUtil.EMPTY_BYTE_BUFFER));
 -        assertRows(execute("SELECT " + fNameBRN + "(bval) FROM %s"), row(ByteBufferUtil.EMPTY_BYTE_BUFFER));
 -        assertRows(execute("SELECT " + fNameBCC + "(bval) FROM %s"), row(ByteBufferUtil.EMPTY_BYTE_BUFFER));
 -        assertRows(execute("SELECT " + fNameBCN + "(bval) FROM %s"), row(ByteBufferUtil.EMPTY_BYTE_BUFFER));
 -        assertRows(execute("SELECT " + fNameIRC + "(empty_int) FROM %s"), row(new Object[]{ null }));
 -        assertRows(execute("SELECT " + fNameIRN + "(empty_int) FROM %s"), row(new Object[]{ null }));
 -        assertRows(execute("SELECT " + fNameICC + "(empty_int) FROM %s"), row(0));
 -        assertRows(execute("SELECT " + fNameICN + "(empty_int) FROM %s"), row(new Object[]{ null }));
 +    }
 +
 +    @Test
 +    public void testAnyUserTupleType() throws Throwable
 +    {
 +        createTable("CREATE TABLE %s (key int primary key, sval text)");
 +        execute("INSERT INTO %s (key, sval) VALUES (1, 'foo')");
 +
 +        String udt = createType("CREATE TYPE %s (a int, b text, c bigint)");
 +
 +        String fUdt = createFunction(KEYSPACE, "text",
 +                                     "CREATE OR REPLACE FUNCTION %s(arg text) " +
 +                                     "CALLED ON NULL INPUT " +
 +                                     "RETURNS " + udt + " " +
 +                                     "LANGUAGE JAVA\n" +
 +                                     "AS $$\n" +
 +                                     "    UDTValue udt = udfContext.newUDTValue(\"" + udt + "\");" +
 +                                     "    udt.setInt(\"a\", 42);" +
 +                                     "    udt.setString(\"b\", \"42\");" +
 +                                     "    udt.setLong(\"c\", 4242);" +
 +                                     "    return udt;" +
 +                                     "$$");
 +
 +        assertRows(execute("SELECT " + fUdt + "(sval) FROM %s"),
 +                   row(userType("a", 42, "b", "42", "c", 4242L)));
 +
 +        String fTup = createFunction(KEYSPACE, "text",
 +                                     "CREATE OR REPLACE FUNCTION %s(arg text) " +
 +                                     "CALLED ON NULL INPUT " +
 +                                     "RETURNS tuple<int, " + udt + "> " +
 +                                     "LANGUAGE JAVA\n" +
 +                                     "AS $$\n" +
 +                                     "    UDTValue udt = udfContext.newUDTValue(\"" + udt + "\");" +
 +                                     "    udt.setInt(\"a\", 42);" +
 +                                     "    udt.setString(\"b\", \"42\");" +
 +                                     "    udt.setLong(\"c\", 4242);" +
 +                                     "    TupleValue tup = udfContext.newTupleValue(\"tuple<int," + udt + ">\");" +
 +                                     "    tup.setInt(0, 88);" +
 +                                     "    tup.setUDTValue(1, udt);" +
 +                                     "    return tup;" +
 +                                     "$$");
 +
 +        assertRows(execute("SELECT " + fTup + "(sval) FROM %s"),
 +                   row(tuple(88, userType("a", 42, "b", "42", "c", 4242L))));
      }
+ 
+     @Test(expected = SyntaxException.class)
+     public void testEmptyFunctionName() throws Throwable
+     {
+         execute("CREATE FUNCTION IF NOT EXISTS " + KEYSPACE + ".\"\" (arg int)\n" +
+                 "  RETURNS NULL ON NULL INPUT\n" +
+                 "  RETURNS int\n" +
+                 "  LANGUAGE java\n" +
+                 "  AS $$\n" +
+                 "    return a;\n" +
+                 "  $$");
+     }
+ 
+     @Test(expected = SyntaxException.class)
+     public void testEmptyArgName() throws Throwable
+     {
+         execute("CREATE FUNCTION IF NOT EXISTS " + KEYSPACE + ".myfn (\"\" int)\n" +
+                 "  RETURNS NULL ON NULL INPUT\n" +
+                 "  RETURNS int\n" +
+                 "  LANGUAGE java\n" +
+                 "  AS $$\n" +
+                 "    return a;\n" +
+                 "  $$");
+     }
  }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d429cd0/test/unit/org/apache/cassandra/cql3/validation/entities/UserTypesTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/cql3/validation/entities/UserTypesTest.java
index 9ea5572,68c0b8c..646484c
--- a/test/unit/org/apache/cassandra/cql3/validation/entities/UserTypesTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/UserTypesTest.java
@@@ -872,12 -708,31 +873,31 @@@ public class UserTypesTest extends CQLT
  
          execute("ALTER TYPE " + columnType + " ADD b int");
          execute("UPDATE %s SET s = s + ?, v = ? WHERE pk = ? AND c = ?",
 -                set(userType(1, 1), userType(1, 2), userType(2, 1)), 2, 1, 1);
 +                set(userType("a", 1, "b", 1), userType("a", 1, "b", 2), userType("a", 2, "b", 1)), 2, 1, 1);
  
          assertRows(execute("SELECT * FROM %s WHERE pk = ? AND c = ?", 1, 1),
 -                       row(1, 1,set(userType(1), userType(1, 1), userType(1, 2), userType(2), userType(2, 1)), 2));
 +                       row(1, 1,set(userType("a", 1), userType("a", 1, "b", 1), userType("a", 1, "b", 2), userType("a", 2), userType("a", 2, "b", 1)), 2));
      }
  
+     @Test(expected = SyntaxException.class)
+     public void emptyTypeNameTest() throws Throwable
+     {
+         execute("CREATE TYPE \"\" (a int, b int)");
+     }
+ 
+     @Test(expected = SyntaxException.class)
+     public void emptyFieldNameTest() throws Throwable
+     {
+         execute("CREATE TYPE mytype (\"\" int, b int)");
+     }
+ 
+     @Test(expected = SyntaxException.class)
+     public void renameColumnToEmpty() throws Throwable
+     {
+         String typeName = createType("CREATE TYPE %s (a int, b int)");
+         execute(String.format("ALTER TYPE %s.%s RENAME b TO \"\"", keyspace(), typeName));
+     }
+ 
      private String typeWithKs(String type1)
      {
          return keyspace() + '.' + type1;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d429cd0/test/unit/org/apache/cassandra/cql3/validation/operations/AlterTest.java
----------------------------------------------------------------------


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


[06/25] cassandra git commit: Allow dropping COMPACT STORAGE flag

Posted by if...@apache.org.
Allow dropping COMPACT STORAGE flag

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/6c29ee84
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/6c29ee84
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/6c29ee84

Branch: refs/heads/trunk
Commit: 6c29ee84a2f62ccd05c328bbaa0c364eb1a7a821
Parents: b869744
Author: Alex Petrov <ol...@gmail.com>
Authored: Sat Sep 30 08:56:22 2017 +0200
Committer: Alex Petrov <ol...@gmail.com>
Committed: Mon Nov 6 15:44:51 2017 +0100

----------------------------------------------------------------------
 NEWS.txt                                        |  17 +
 bin/cqlsh.py                                    |   6 +-
 doc/native_protocol_v4.spec                     |   4 +
 ...dra-driver-internal-only-3.11.0-bb96859b.zip | Bin 0 -> 266661 bytes
 ...driver-internal-only-3.7.1.post0-19c1603.zip | Bin 252027 -> 0 bytes
 .../cassandra/auth/CassandraRoleManager.java    |   3 +-
 .../org/apache/cassandra/config/CFMetaData.java |  59 ++-
 src/java/org/apache/cassandra/cql3/Cql.g        |  27 +-
 .../apache/cassandra/cql3/QueryProcessor.java   |   2 +-
 .../cql3/statements/AlterTableStatement.java    |  42 +-
 .../statements/AuthenticationStatement.java     |   2 +-
 .../cql3/statements/AuthorizationStatement.java |   2 +-
 .../cql3/statements/BatchStatement.java         |   4 +-
 .../statements/CreateAggregateStatement.java    |   4 +-
 .../statements/CreateFunctionStatement.java     |   4 +-
 .../cql3/statements/CreateIndexStatement.java   |   9 +-
 .../cql3/statements/CreateTableStatement.java   |   2 +-
 .../cql3/statements/CreateViewStatement.java    |   2 +-
 .../cql3/statements/DropFunctionStatement.java  |   4 +-
 .../cql3/statements/ModificationStatement.java  |  11 +-
 .../cql3/statements/ParsedStatement.java        |   3 +-
 .../statements/SchemaAlteringStatement.java     |  32 +-
 .../cql3/statements/SelectStatement.java        |   8 +-
 .../cql3/statements/TruncateStatement.java      |   2 +-
 .../cassandra/cql3/statements/UseStatement.java |   2 +-
 src/java/org/apache/cassandra/db/view/View.java |  38 +-
 .../index/internal/keys/KeysSearcher.java       |   8 +-
 .../apache/cassandra/repair/RepairRunnable.java |   3 +-
 .../apache/cassandra/service/ClientState.java   |  16 +
 .../cassandra/thrift/ThriftValidation.java      |  10 +-
 .../transport/messages/StartupMessage.java      |   4 +
 .../org/apache/cassandra/cql3/ViewTest.java     |  15 +-
 .../cql3/validation/entities/UFTest.java        |  25 +
 .../cql3/validation/entities/UserTypesTest.java |  20 +
 .../cql3/validation/operations/AlterTest.java   |  15 +-
 .../DropCompactStorageThriftTest.java           | 525 +++++++++++++++++++
 36 files changed, 843 insertions(+), 87 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/NEWS.txt
----------------------------------------------------------------------
diff --git a/NEWS.txt b/NEWS.txt
index 60cf77c..621866b 100644
--- a/NEWS.txt
+++ b/NEWS.txt
@@ -37,6 +37,23 @@ Upgrading
    - Nothing specific to this release, but please see previous upgrading sections,
      especially if you are upgrading from 2.2.
 
+Compact Storage
+---------------
+    - Starting version 4.0, Thrift and COMPACT STORAGE is no longer supported.
+      'ALTER ... DROP COMPACT STORAGE' statement makes Compact Tables CQL-compatible,
+      exposing internal structure of Thrift/Compact Tables. You can find more details
+      on exposed internal structure under: 
+      http://cassandra.apache.org/doc/latest/cql/appendices.html#appendix-c-dropping-compact-storage
+
+      For uninterrupted cluster upgrades, drivers now support 'NO_COMPACT' startup option.
+      Supplying this flag will have same effect as 'DROP COMPACT STORAGE', but only for the
+      current connection.
+
+      In order to upgrade, clients supporting a non-compact schema view can be rolled out
+      gradually. When all the clients are updated 'ALTER ... DROP COMPACT STORAGE' can be
+      executed. After dropping compact storage, ’NO_COMPACT' option will have no effect
+      after that.
+
 Materialized Views
 -------------------
     - Cassandra will no longer allow dropping columns on tables with Materialized Views.

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/bin/cqlsh.py
----------------------------------------------------------------------
diff --git a/bin/cqlsh.py b/bin/cqlsh.py
index 8eb42a3..8d05d9d 100644
--- a/bin/cqlsh.py
+++ b/bin/cqlsh.py
@@ -205,6 +205,7 @@ parser.add_option("--browser", dest='browser', help="""The browser to use to dis
                                                     - one of the supported browsers in https://docs.python.org/2/library/webbrowser.html.
                                                     - browser path followed by %s, example: /usr/bin/google-chrome-stable %s""")
 parser.add_option('--ssl', action='store_true', help='Use SSL', default=False)
+parser.add_option('--no_compact', action='store_true', help='No Compact', default=False)
 parser.add_option("-u", "--username", help="Authenticate as user.")
 parser.add_option("-p", "--password", help="Authenticate using password.")
 parser.add_option('-k', '--keyspace', help='Authenticate to the given keyspace.')
@@ -702,6 +703,7 @@ class Shell(cmd.Cmd):
                  completekey=DEFAULT_COMPLETEKEY, browser=None, use_conn=None,
                  cqlver=DEFAULT_CQLVER, keyspace=None,
                  tracing_enabled=False, expand_enabled=False,
+                 no_compact=False,
                  display_nanotime_format=DEFAULT_NANOTIME_FORMAT,
                  display_timestamp_format=DEFAULT_TIMESTAMP_FORMAT,
                  display_date_format=DEFAULT_DATE_FORMAT,
@@ -732,7 +734,7 @@ class Shell(cmd.Cmd):
         else:
             self.conn = Cluster(contact_points=(self.hostname,), port=self.port, cql_version=cqlver,
                                 protocol_version=protocol_version,
-                                auth_provider=self.auth_provider,
+                                auth_provider=self.auth_provider, no_compact=no_compact,
                                 ssl_options=sslhandling.ssl_settings(hostname, CONFIG_FILE) if ssl else None,
                                 load_balancing_policy=WhiteListRoundRobinPolicy([self.hostname]),
                                 control_connection_timeout=connect_timeout,
@@ -2486,6 +2488,7 @@ def read_options(cmdlineargs, environment):
     optvalues.debug = False
     optvalues.file = None
     optvalues.ssl = False
+    optvalues.no_compact = False
     optvalues.encoding = option_with_default(configs.get, 'ui', 'encoding', UTF8)
 
     optvalues.tty = option_with_default(configs.getboolean, 'ui', 'tty', sys.stdin.isatty())
@@ -2643,6 +2646,7 @@ def main(options, hostname, port):
                       browser=options.browser,
                       cqlver=options.cqlversion,
                       keyspace=options.keyspace,
+                      no_compact=options.no_compact,
                       display_timestamp_format=options.time_format,
                       display_nanotime_format=options.nanotime_format,
                       display_date_format=options.date_format,

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/doc/native_protocol_v4.spec
----------------------------------------------------------------------
diff --git a/doc/native_protocol_v4.spec b/doc/native_protocol_v4.spec
index 44dac18..2188a33 100644
--- a/doc/native_protocol_v4.spec
+++ b/doc/native_protocol_v4.spec
@@ -271,6 +271,10 @@ Table of Contents
       different from the protocol version.
     - "COMPRESSION": the compression algorithm to use for frames (See section 5).
       This is optional; if not specified no compression will be used.
+    - "NO_COMPACT": whether or not connection has to be established in compatibility
+      mode. This mode will make all Thrift and Compact Tables to be exposed as if
+      they were CQL Tables. This is optional; if not specified, the option will
+      not be used.
 
 
 4.1.2. AUTH_RESPONSE

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/lib/cassandra-driver-internal-only-3.11.0-bb96859b.zip
----------------------------------------------------------------------
diff --git a/lib/cassandra-driver-internal-only-3.11.0-bb96859b.zip b/lib/cassandra-driver-internal-only-3.11.0-bb96859b.zip
new file mode 100644
index 0000000..d31abc3
Binary files /dev/null and b/lib/cassandra-driver-internal-only-3.11.0-bb96859b.zip differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/lib/cassandra-driver-internal-only-3.7.1.post0-19c1603.zip
----------------------------------------------------------------------
diff --git a/lib/cassandra-driver-internal-only-3.7.1.post0-19c1603.zip b/lib/cassandra-driver-internal-only-3.7.1.post0-19c1603.zip
deleted file mode 100644
index 900d64d..0000000
Binary files a/lib/cassandra-driver-internal-only-3.7.1.post0-19c1603.zip and /dev/null differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/src/java/org/apache/cassandra/auth/CassandraRoleManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/auth/CassandraRoleManager.java b/src/java/org/apache/cassandra/auth/CassandraRoleManager.java
index e5b98e4..c6716e5 100644
--- a/src/java/org/apache/cassandra/auth/CassandraRoleManager.java
+++ b/src/java/org/apache/cassandra/auth/CassandraRoleManager.java
@@ -39,6 +39,7 @@ import org.apache.cassandra.db.ConsistencyLevel;
 import org.apache.cassandra.db.marshal.UTF8Type;
 import org.apache.cassandra.exceptions.*;
 import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.service.ClientState;
 import org.apache.cassandra.service.QueryState;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.transport.messages.ResultMessage;
@@ -465,7 +466,7 @@ public class CassandraRoleManager implements IRoleManager
     {
         try
         {
-            return QueryProcessor.parseStatement(String.format(template, keyspace, table)).prepare().statement;
+            return QueryProcessor.parseStatement(String.format(template, keyspace, table)).prepare(ClientState.forInternalCalls()).statement;
         }
         catch (RequestValidationException e)
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/src/java/org/apache/cassandra/config/CFMetaData.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/CFMetaData.java b/src/java/org/apache/cassandra/config/CFMetaData.java
index fd1c9e5..c1b2171 100644
--- a/src/java/org/apache/cassandra/config/CFMetaData.java
+++ b/src/java/org/apache/cassandra/config/CFMetaData.java
@@ -79,10 +79,6 @@ public final class CFMetaData
     public final Pair<String, String> ksAndCFName;
     public final byte[] ksAndCFBytes;
 
-    private final ImmutableSet<Flag> flags;
-    private final boolean isDense;
-    private final boolean isCompound;
-    private final boolean isSuper;
     private final boolean isCounter;
     private final boolean isView;
     private final boolean isIndex;
@@ -94,6 +90,11 @@ public final class CFMetaData
     private final Serializers serializers;
 
     // non-final, for now
+    private volatile ImmutableSet<Flag> flags;
+    private volatile boolean isDense;
+    private volatile boolean isCompound;
+    private volatile boolean isSuper;
+
     public volatile TableParams params = TableParams.DEFAULT;
 
     private volatile Map<ByteBuffer, DroppedColumn> droppedColumns = new HashMap<>();
@@ -127,6 +128,9 @@ public final class CFMetaData
     private volatile ColumnDefinition superCfKeyColumn;
     private volatile ColumnDefinition superCfValueColumn;
 
+    /** Caches a non-compact version of the metadata for compact tables to be used with the NO_COMPACT protocol option. */
+    private volatile CFMetaData nonCompactCopy = null;
+
     public boolean isSuperColumnKeyColumn(ColumnDefinition cd)
     {
         return cd.name.equals(superCfKeyColumn.name);
@@ -330,6 +334,9 @@ public final class CFMetaData
     // are kept because they are often useful in a different format.
     private void rebuild()
     {
+        // A non-compact copy will be created lazily
+        this.nonCompactCopy = null;
+
         if (isCompactTable())
         {
             this.compactValueColumn = isSuper() ?
@@ -505,6 +512,38 @@ public final class CFMetaData
         return params(indexParams.build());
     }
 
+    /**
+     * Returns a cached non-compact version of this table. Cached version has to be invalidated
+     * every time the table is rebuilt.
+     */
+    public CFMetaData asNonCompact()
+    {
+        assert isCompactTable() : "Can't get non-compact version of a CQL table";
+
+        // Note that this is racy, but re-computing the non-compact copy a few times on first uses isn't a big deal so
+        // we don't bother.
+        if (nonCompactCopy == null)
+        {
+            nonCompactCopy = copyOpts(new CFMetaData(ksName,
+                                                     cfName,
+                                                     cfId,
+                                                     false,
+                                                     isCounter,
+                                                     false,
+                                                     true,
+                                                     isView,
+                                                     copy(partitionKeyColumns),
+                                                     copy(clusteringColumns),
+                                                     copy(partitionColumns),
+                                                     partitioner,
+                                                     superCfKeyColumn,
+                                                     superCfValueColumn),
+                                      this);
+        }
+
+        return nonCompactCopy;
+    }
+
     public CFMetaData copy()
     {
         return copy(cfId);
@@ -842,6 +881,12 @@ public final class CFMetaData
         superCfKeyColumn = cfm.superCfKeyColumn;
         superCfValueColumn = cfm.superCfValueColumn;
 
+        isDense = cfm.isDense;
+        isCompound = cfm.isCompound;
+        isSuper = cfm.isSuper;
+
+        flags = cfm.flags;
+
         rebuild();
 
         // compaction thresholds are checked by ThriftValidation. We shouldn't be doing
@@ -874,12 +919,6 @@ public final class CFMetaData
         if (!cfm.cfId.equals(cfId))
             throw new ConfigurationException(String.format("Column family ID mismatch (found %s; expected %s)",
                                                            cfm.cfId, cfId));
-
-        // Dense flag can get set, see CASSANDRA-12373 for details. We have to remove flag from both parts because
-        // there's no guaranteed call order in the call.
-
-        if (!cfm.flags.equals(flags) && (!isSuper() || !Sets.difference(cfm.flags, Sets.immutableEnumSet(Flag.DENSE)).equals(Sets.difference(flags, Sets.immutableEnumSet(Flag.DENSE)))))
-            throw new ConfigurationException("Types do not match: " + cfm.flags + " != " + flags);
     }
 
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/src/java/org/apache/cassandra/cql3/Cql.g
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/Cql.g b/src/java/org/apache/cassandra/cql3/Cql.g
index 3123877..0234327 100644
--- a/src/java/org/apache/cassandra/cql3/Cql.g
+++ b/src/java/org/apache/cassandra/cql3/Cql.g
@@ -781,7 +781,7 @@ createTriggerStatement returns [CreateTriggerStatement expr]
     @init {
         boolean ifNotExists = false;
     }
-    : K_CREATE K_TRIGGER (K_IF K_NOT K_EXISTS { ifNotExists = true; } )? (name=cident)
+    : K_CREATE K_TRIGGER (K_IF K_NOT K_EXISTS { ifNotExists = true; } )? (name=noncol_ident)
         K_ON cf=columnFamilyName K_USING cls=STRING_LITERAL
       { $expr = new CreateTriggerStatement(cf, name.toString(), $cls.text, ifNotExists); }
     ;
@@ -791,7 +791,7 @@ createTriggerStatement returns [CreateTriggerStatement expr]
  */
 dropTriggerStatement returns [DropTriggerStatement expr]
      @init { boolean ifExists = false; }
-    : K_DROP K_TRIGGER (K_IF K_EXISTS { ifExists = true; } )? (name=cident) K_ON cf=columnFamilyName
+    : K_DROP K_TRIGGER (K_IF K_EXISTS { ifExists = true; } )? (name=noncol_ident) K_ON cf=columnFamilyName
       { $expr = new DropTriggerStatement(cf, name.toString(), ifExists); }
     ;
 
@@ -816,20 +816,21 @@ alterTableStatement returns [AlterTableStatement expr]
     @init {
         AlterTableStatement.Type type = null;
         TableAttributes attrs = new TableAttributes();
-        Map<ColumnIdentifier.Raw, ColumnIdentifier.Raw> renames = new HashMap<ColumnIdentifier.Raw, ColumnIdentifier.Raw>();
+        Map<ColumnIdentifier.Raw, ColumnIdentifier> renames = new HashMap<ColumnIdentifier.Raw, ColumnIdentifier>();
         boolean isStatic = false;
         Long dropTimestamp = null;
     }
     : K_ALTER K_COLUMNFAMILY cf=columnFamilyName
-          ( K_ALTER id=cident K_TYPE v=comparatorType { type = AlterTableStatement.Type.ALTER; }
-          | K_ADD   id=cident v=comparatorType ({ isStatic=true; } K_STATIC)? { type = AlterTableStatement.Type.ADD; }
+          ( K_ALTER id=cident K_TYPE v=comparatorType { type = AlterTableStatement.Type.ALTER;  }
+          | K_ADD   aid=ident {id=new ColumnIdentifier.ColumnIdentifierValue(aid);} v=comparatorType ({ isStatic=true; } K_STATIC)? { type = AlterTableStatement.Type.ADD; }
           | K_DROP  id=cident                               { type = AlterTableStatement.Type.DROP; }
           | K_DROP  id=cident K_USING K_TIMESTAMP t=INTEGER { type = AlterTableStatement.Type.DROP;
                                                               dropTimestamp = Long.parseLong(Constants.Literal.integer($t.text).getText()); }
+          | K_DROP  K_COMPACT K_STORAGE                     { type = AlterTableStatement.Type.DROP_COMPACT_STORAGE; }
           | K_WITH  properties[attrs]                       { type = AlterTableStatement.Type.OPTS; }
           | K_RENAME                                        { type = AlterTableStatement.Type.RENAME; }
-               id1=cident K_TO toId1=cident { renames.put(id1, toId1); }
-               ( K_AND idn=cident K_TO toIdn=cident { renames.put(idn, toIdn); } )*
+               id1=cident K_TO toId1=ident { renames.put(id1, toId1); }
+               ( K_AND idn=cident K_TO toIdn=ident { renames.put(idn, toIdn); } )*
           )
     {
         $expr = new AlterTableStatement(cf, type, id, v, attrs, renames, isStatic, dropTimestamp);
@@ -1169,10 +1170,14 @@ userPassword[RoleOptions opts]
 // Column Identifiers.  These need to be treated differently from other
 // identifiers because the underlying comparator is not necessarily text. See
 // CASSANDRA-8178 for details.
+// Also, we need to support the internal of the super column map (for backward
+// compatibility) which is empty (we only want to allow this is queries, not for
+// creating table or other).
 cident returns [ColumnIdentifier.Raw id]
     : t=IDENT              { $id = new ColumnIdentifier.Literal($t.text, false); }
     | t=QUOTED_NAME        { $id = new ColumnIdentifier.Literal($t.text, true); }
     | k=unreserved_keyword { $id = new ColumnIdentifier.Literal(k, false); }
+    | EMPTY_QUOTED_NAME    { $id = new ColumnIdentifier.Literal("", false); }
     ;
 
 // Column identifiers where the comparator is known to be text
@@ -1309,7 +1314,9 @@ intValue returns [Term.Raw value]
     ;
 
 functionName returns [FunctionName s]
-    : (ks=keyspaceName '.')? f=allowedFunctionName   { $s = new FunctionName(ks, f); }
+     // antlr might try to recover and give a null for f. It will still error out in the end, but FunctionName
+     // wouldn't be happy with that so we should bypass this for now or we'll have a weird user-facing error
+    : (ks=keyspaceName '.')? f=allowedFunctionName   { $s = f == null ? null : new FunctionName(ks, f); }
     ;
 
 allowedFunctionName returns [String s]
@@ -1822,6 +1829,10 @@ STRING_LITERAL
       )
     ;
 
+EMPTY_QUOTED_NAME
+    : '\"' '\"'
+    ;
+
 QUOTED_NAME
     @init{ StringBuilder b = new StringBuilder(); }
     @after{ setText(b.toString()); }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/src/java/org/apache/cassandra/cql3/QueryProcessor.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/QueryProcessor.java b/src/java/org/apache/cassandra/cql3/QueryProcessor.java
index af94d3e..ddee6c7 100644
--- a/src/java/org/apache/cassandra/cql3/QueryProcessor.java
+++ b/src/java/org/apache/cassandra/cql3/QueryProcessor.java
@@ -517,7 +517,7 @@ public class QueryProcessor implements QueryHandler
             ((CFStatement)statement).prepareKeyspace(clientState);
 
         Tracing.trace("Preparing statement");
-        return statement.prepare();
+        return statement.prepare(clientState);
     }
 
     public static ParsedStatement parseStatement(String queryStr) throws SyntaxException

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java b/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java
index befdd25..a5fa12d 100644
--- a/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java
@@ -31,8 +31,6 @@ import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.db.marshal.CollectionType;
-import org.apache.cassandra.db.marshal.CounterColumnType;
-import org.apache.cassandra.db.marshal.ReversedType;
 import org.apache.cassandra.db.view.View;
 import org.apache.cassandra.exceptions.*;
 import org.apache.cassandra.schema.IndexMetadata;
@@ -49,14 +47,14 @@ public class AlterTableStatement extends SchemaAlteringStatement
 {
     public enum Type
     {
-        ADD, ALTER, DROP, OPTS, RENAME
+        ADD, ALTER, DROP, DROP_COMPACT_STORAGE, OPTS, RENAME
     }
 
     public final Type oType;
     public final CQL3Type.Raw validator;
     public final ColumnIdentifier.Raw rawColumnName;
     private final TableAttributes attrs;
-    private final Map<ColumnIdentifier.Raw, ColumnIdentifier.Raw> renames;
+    private final Map<ColumnIdentifier.Raw, ColumnIdentifier> renames;
     private final boolean isStatic; // Only for ALTER ADD
     private final Long deleteTimestamp;
 
@@ -65,7 +63,7 @@ public class AlterTableStatement extends SchemaAlteringStatement
                                ColumnIdentifier.Raw columnName,
                                CQL3Type.Raw validator,
                                TableAttributes attrs,
-                               Map<ColumnIdentifier.Raw, ColumnIdentifier.Raw> renames,
+                               Map<ColumnIdentifier.Raw, ColumnIdentifier> renames,
                                boolean isStatic,
                                Long deleteTimestamp)
     {
@@ -95,15 +93,15 @@ public class AlterTableStatement extends SchemaAlteringStatement
         if (meta.isView())
             throw new InvalidRequestException("Cannot use ALTER TABLE on Materialized View");
 
-        CFMetaData cfm = meta.copy();
+        CFMetaData cfm;
 
         CQL3Type validator = this.validator == null ? null : this.validator.prepare(keyspace());
         ColumnIdentifier columnName = null;
         ColumnDefinition def = null;
         if (rawColumnName != null)
         {
-            columnName = rawColumnName.prepare(cfm);
-            def = cfm.getColumnDefinition(columnName);
+            columnName = rawColumnName.prepare(meta);
+            def = meta.getColumnDefinition(columnName);
         }
 
         List<ViewDefinition> viewUpdates = null;
@@ -115,9 +113,11 @@ public class AlterTableStatement extends SchemaAlteringStatement
                 throw new InvalidRequestException("Altering of types is not allowed");
             case ADD:
                 assert columnName != null;
-                if (cfm.isDense())
+                if (meta.isDense())
                     throw new InvalidRequestException("Cannot add new column to a COMPACT STORAGE table");
 
+                cfm = meta.copy();
+
                 if (isStatic)
                 {
                     if (!cfm.isCompound())
@@ -190,11 +190,14 @@ public class AlterTableStatement extends SchemaAlteringStatement
 
             case DROP:
                 assert columnName != null;
-                if (!cfm.isCQLTable())
+                if (!meta.isCQLTable())
                     throw new InvalidRequestException("Cannot drop columns from a non-CQL3 table");
+
                 if (def == null)
                     throw new InvalidRequestException(String.format("Column %s was not found in table %s", columnName, columnFamily()));
 
+                cfm = meta.copy();
+
                 switch (def.kind)
                 {
                     case PARTITION_KEY:
@@ -238,11 +241,19 @@ public class AlterTableStatement extends SchemaAlteringStatement
                                                                     columnName.toString(),
                                                                     keyspace()));
                 break;
+            case DROP_COMPACT_STORAGE:
+                if (!meta.isCompactTable())
+                    throw new InvalidRequestException("Cannot DROP COMPACT STORAGE on table without COMPACT STORAGE");
+
+                cfm = meta.asNonCompact();
+                break;
             case OPTS:
                 if (attrs == null)
                     throw new InvalidRequestException("ALTER TABLE WITH invoked, but no parameters found");
                 attrs.validate();
 
+                cfm = meta.copy();
+
                 TableParams params = attrs.asAlteredTableParams(cfm.params);
 
                 if (!Iterables.isEmpty(views) && params.gcGraceSeconds == 0)
@@ -261,10 +272,13 @@ public class AlterTableStatement extends SchemaAlteringStatement
 
                 break;
             case RENAME:
-                for (Map.Entry<ColumnIdentifier.Raw, ColumnIdentifier.Raw> entry : renames.entrySet())
+                cfm = meta.copy();
+
+                for (Map.Entry<ColumnIdentifier.Raw, ColumnIdentifier> entry : renames.entrySet())
                 {
                     ColumnIdentifier from = entry.getKey().prepare(cfm);
-                    ColumnIdentifier to = entry.getValue().prepare(cfm);
+                    ColumnIdentifier to = entry.getValue();
+
                     cfm.renameColumn(from, to);
 
                     // If the view includes a renamed column, it must be renamed in the view table and the definition.
@@ -274,7 +288,7 @@ public class AlterTableStatement extends SchemaAlteringStatement
 
                         ViewDefinition viewCopy = view.copy();
                         ColumnIdentifier viewFrom = entry.getKey().prepare(viewCopy.metadata);
-                        ColumnIdentifier viewTo = entry.getValue().prepare(viewCopy.metadata);
+                        ColumnIdentifier viewTo = entry.getValue();
                         viewCopy.renameColumn(viewFrom, viewTo);
 
                         if (viewUpdates == null)
@@ -283,6 +297,8 @@ public class AlterTableStatement extends SchemaAlteringStatement
                     }
                 }
                 break;
+            default:
+                throw new InvalidRequestException("Can not alter table: unknown option type " + oType);
         }
 
         MigrationManager.announceColumnFamilyUpdate(cfm, viewUpdates, isLocalOnly);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/src/java/org/apache/cassandra/cql3/statements/AuthenticationStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/AuthenticationStatement.java b/src/java/org/apache/cassandra/cql3/statements/AuthenticationStatement.java
index 151e4f0..30ab6b0 100644
--- a/src/java/org/apache/cassandra/cql3/statements/AuthenticationStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/AuthenticationStatement.java
@@ -31,7 +31,7 @@ import org.apache.cassandra.transport.messages.ResultMessage;
 public abstract class AuthenticationStatement extends ParsedStatement implements CQLStatement
 {
     @Override
-    public Prepared prepare()
+    public Prepared prepare(ClientState clientState)
     {
         return new Prepared(this);
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/src/java/org/apache/cassandra/cql3/statements/AuthorizationStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/AuthorizationStatement.java b/src/java/org/apache/cassandra/cql3/statements/AuthorizationStatement.java
index 098e22c..fa2a993 100644
--- a/src/java/org/apache/cassandra/cql3/statements/AuthorizationStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/AuthorizationStatement.java
@@ -32,7 +32,7 @@ import org.apache.cassandra.transport.messages.ResultMessage;
 public abstract class AuthorizationStatement extends ParsedStatement implements CQLStatement
 {
     @Override
-    public Prepared prepare()
+    public Prepared prepare(ClientState clientState)
     {
         return new Prepared(this);
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java b/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
index cd9358c..1c3cfa6 100644
--- a/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
@@ -516,7 +516,7 @@ public class BatchStatement implements CQLStatement
                 statement.prepareKeyspace(state);
         }
 
-        public ParsedStatement.Prepared prepare() throws InvalidRequestException
+        public ParsedStatement.Prepared prepare(ClientState clientState) throws InvalidRequestException
         {
             VariableSpecifications boundNames = getBoundVariables();
 
@@ -537,7 +537,7 @@ public class BatchStatement implements CQLStatement
                     haveMultipleCFs = !firstKS.equals(parsed.keyspace()) || !firstCF.equals(parsed.columnFamily());
                 }
 
-                statements.add(parsed.prepare(boundNames));
+                statements.add(parsed.prepare(boundNames, clientState));
             }
 
             Attributes prepAttrs = attrs.prepare("[batch]", "[batch]");

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/src/java/org/apache/cassandra/cql3/statements/CreateAggregateStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/CreateAggregateStatement.java b/src/java/org/apache/cassandra/cql3/statements/CreateAggregateStatement.java
index ca0270f..9d91693 100644
--- a/src/java/org/apache/cassandra/cql3/statements/CreateAggregateStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/CreateAggregateStatement.java
@@ -78,7 +78,7 @@ public final class CreateAggregateStatement extends SchemaAlteringStatement
         this.ifNotExists = ifNotExists;
     }
 
-    public Prepared prepare()
+    public Prepared prepare(ClientState clientState)
     {
         argTypes = new ArrayList<>(argRawTypes.size());
         for (CQL3Type.Raw rawType : argRawTypes)
@@ -136,7 +136,7 @@ public final class CreateAggregateStatement extends SchemaAlteringStatement
                 throw new InvalidRequestException("INITCOND must not be empty for all types except TEXT, ASCII, BLOB");
         }
 
-        return super.prepare();
+        return super.prepare(clientState);
     }
 
     private AbstractType<?> prepareType(String typeName, CQL3Type.Raw rawType)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/src/java/org/apache/cassandra/cql3/statements/CreateFunctionStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/CreateFunctionStatement.java b/src/java/org/apache/cassandra/cql3/statements/CreateFunctionStatement.java
index c8d38f5..dfe522b 100644
--- a/src/java/org/apache/cassandra/cql3/statements/CreateFunctionStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/CreateFunctionStatement.java
@@ -76,7 +76,7 @@ public final class CreateFunctionStatement extends SchemaAlteringStatement
         this.ifNotExists = ifNotExists;
     }
 
-    public Prepared prepare() throws InvalidRequestException
+    public Prepared prepare(ClientState clientState) throws InvalidRequestException
     {
         if (new HashSet<>(argNames).size() != argNames.size())
             throw new InvalidRequestException(String.format("duplicate argument names for given function %s with argument names %s",
@@ -87,7 +87,7 @@ public final class CreateFunctionStatement extends SchemaAlteringStatement
             argTypes.add(prepareType("arguments", rawType));
 
         returnType = prepareType("return type", rawReturnType);
-        return super.prepare();
+        return super.prepare(clientState);
     }
 
     public void prepareKeyspace(ClientState state) throws InvalidRequestException

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/src/java/org/apache/cassandra/cql3/statements/CreateIndexStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/CreateIndexStatement.java b/src/java/org/apache/cassandra/cql3/statements/CreateIndexStatement.java
index c21441c..47d54fe 100644
--- a/src/java/org/apache/cassandra/cql3/statements/CreateIndexStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/CreateIndexStatement.java
@@ -103,8 +103,13 @@ public class CreateIndexStatement extends SchemaAlteringStatement
                 throw new InvalidRequestException("No column definition found for column " + target.column);
 
             // TODO: we could lift that limitation
-            if (cfm.isCompactTable() && cd.isPrimaryKeyColumn())
-                throw new InvalidRequestException("Secondary indexes are not supported on PRIMARY KEY columns in COMPACT STORAGE tables");
+            if (cfm.isCompactTable())
+            {
+                if (cd.isPrimaryKeyColumn())
+                    throw new InvalidRequestException("Secondary indexes are not supported on PRIMARY KEY columns in COMPACT STORAGE tables");
+                if (cfm.compactValueColumn().equals(cd))
+                    throw new InvalidRequestException("Secondary indexes are not supported on compact value column of COMPACT STORAGE tables");
+            }
 
             // It would be possible to support 2ndary index on static columns (but not without modifications of at least ExtendedFilter and
             // CompositesIndex) and maybe we should, but that means a query like:

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/src/java/org/apache/cassandra/cql3/statements/CreateTableStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/CreateTableStatement.java b/src/java/org/apache/cassandra/cql3/statements/CreateTableStatement.java
index ef950dc..9f14194 100644
--- a/src/java/org/apache/cassandra/cql3/statements/CreateTableStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/CreateTableStatement.java
@@ -191,7 +191,7 @@ public class CreateTableStatement extends SchemaAlteringStatement
         /**
          * Transform this raw statement into a CreateTableStatement.
          */
-        public ParsedStatement.Prepared prepare() throws RequestValidationException
+        public ParsedStatement.Prepared prepare(ClientState clientState) throws RequestValidationException
         {
             KeyspaceMetadata ksm = Schema.instance.getKSMetaData(keyspace());
             if (ksm == null)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/src/java/org/apache/cassandra/cql3/statements/CreateViewStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/CreateViewStatement.java b/src/java/org/apache/cassandra/cql3/statements/CreateViewStatement.java
index 778a3f4..cce954f 100644
--- a/src/java/org/apache/cassandra/cql3/statements/CreateViewStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/CreateViewStatement.java
@@ -214,7 +214,7 @@ public class CreateViewStatement extends SchemaAlteringStatement
         rawSelect.prepareKeyspace(state);
         rawSelect.setBoundVariables(getBoundVariables());
 
-        ParsedStatement.Prepared prepared = rawSelect.prepare(true);
+        ParsedStatement.Prepared prepared = rawSelect.prepare(true, queryState.getClientState());
         SelectStatement select = (SelectStatement) prepared.statement;
         StatementRestrictions restrictions = select.getRestrictions();
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/src/java/org/apache/cassandra/cql3/statements/DropFunctionStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/DropFunctionStatement.java b/src/java/org/apache/cassandra/cql3/statements/DropFunctionStatement.java
index 138691e..8845a82 100644
--- a/src/java/org/apache/cassandra/cql3/statements/DropFunctionStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/DropFunctionStatement.java
@@ -63,7 +63,7 @@ public final class DropFunctionStatement extends SchemaAlteringStatement
     }
 
     @Override
-    public Prepared prepare() throws InvalidRequestException
+    public Prepared prepare(ClientState clientState) throws InvalidRequestException
     {
         if (Schema.instance.getKSMetaData(functionName.keyspace) != null)
         {
@@ -82,7 +82,7 @@ public final class DropFunctionStatement extends SchemaAlteringStatement
             }
         }
 
-        return super.prepare();
+        return super.prepare(clientState);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java b/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
index 28fc90f..8ae4d64 100644
--- a/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
@@ -793,17 +793,16 @@ public abstract class ModificationStatement implements CQLStatement
             this.ifExists = ifExists;
         }
 
-        public ParsedStatement.Prepared prepare()
+        public ParsedStatement.Prepared prepare(ClientState clientState)
         {
             VariableSpecifications boundNames = getBoundVariables();
-            ModificationStatement statement = prepare(boundNames);
-            CFMetaData cfm = ThriftValidation.validateColumnFamily(keyspace(), columnFamily());
-            return new ParsedStatement.Prepared(statement, boundNames, boundNames.getPartitionKeyBindIndexes(cfm));
+            ModificationStatement statement = prepare(boundNames, clientState);
+            return new ParsedStatement.Prepared(statement, boundNames, boundNames.getPartitionKeyBindIndexes(statement.cfm));
         }
 
-        public ModificationStatement prepare(VariableSpecifications boundNames)
+        public ModificationStatement prepare(VariableSpecifications boundNames, ClientState clientState)
         {
-            CFMetaData metadata = ThriftValidation.validateColumnFamily(keyspace(), columnFamily());
+            CFMetaData metadata = ThriftValidation.validateColumnFamilyWithCompactMode(keyspace(), columnFamily(), clientState.isNoCompactMode());
 
             Attributes preparedAttributes = attrs.prepare(keyspace(), columnFamily());
             preparedAttributes.collectMarkerSpecification(boundNames);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/src/java/org/apache/cassandra/cql3/statements/ParsedStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/ParsedStatement.java b/src/java/org/apache/cassandra/cql3/statements/ParsedStatement.java
index 4c3f8a9..01a1b5e 100644
--- a/src/java/org/apache/cassandra/cql3/statements/ParsedStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/ParsedStatement.java
@@ -23,6 +23,7 @@ import java.util.List;
 import org.apache.cassandra.cql3.*;
 import org.apache.cassandra.cql3.functions.Function;
 import org.apache.cassandra.exceptions.RequestValidationException;
+import org.apache.cassandra.service.ClientState;
 
 public abstract class ParsedStatement
 {
@@ -44,7 +45,7 @@ public abstract class ParsedStatement
         this.variables = variables;
     }
 
-    public abstract Prepared prepare() throws RequestValidationException;
+    public abstract Prepared prepare(ClientState clientState) throws RequestValidationException;
 
     public static class Prepared
     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/src/java/org/apache/cassandra/cql3/statements/SchemaAlteringStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/SchemaAlteringStatement.java b/src/java/org/apache/cassandra/cql3/statements/SchemaAlteringStatement.java
index 62ba0ae..e7ecb14 100644
--- a/src/java/org/apache/cassandra/cql3/statements/SchemaAlteringStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/SchemaAlteringStatement.java
@@ -18,6 +18,8 @@
 package org.apache.cassandra.cql3.statements;
 
 import org.apache.cassandra.auth.AuthenticatedUser;
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.cql3.CFName;
 import org.apache.cassandra.cql3.CQLStatement;
 import org.apache.cassandra.cql3.QueryOptions;
@@ -25,9 +27,12 @@ import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.exceptions.RequestValidationException;
 import org.apache.cassandra.service.ClientState;
 import org.apache.cassandra.service.QueryState;
+import org.apache.cassandra.thrift.ThriftValidation;
 import org.apache.cassandra.transport.Event;
 import org.apache.cassandra.transport.messages.ResultMessage;
 
+import static org.apache.cassandra.cql3.statements.RequestValidations.invalidRequest;
+
 /**
  * Abstract class for statements that alter the schema.
  */
@@ -60,8 +65,33 @@ public abstract class SchemaAlteringStatement extends CFStatement implements CQL
     }
 
     @Override
-    public Prepared prepare()
+    public Prepared prepare(ClientState clientState)
     {
+        // We don't allow schema changes in no-compact mode on compact tables because it feels like unnecessary
+        // complication: applying the change on the non compact version of the table might be unsafe (the table is
+        // still compact in general), and applying it to the compact version in a no-compact connection feels
+        // confusing/unintuitive. If user want to alter the compact version, they can simply do so in a normal
+        // connection; if they want to alter the non-compact version, they should finish their transition and properly
+        // DROP COMPACT STORAGE on the table before doing so.
+        if (isColumnFamilyLevel && clientState.isNoCompactMode())
+        {
+            CFMetaData table = ThriftValidation.validateColumnFamily(keyspace(), columnFamily());
+            if (table.isCompactTable())
+            {
+                throw invalidRequest("Cannot alter schema of compact table %s.%s from a connection in NO-COMPACT mode",
+                                     table.ksName, table.cfName);
+            }
+            else if (table.isView())
+            {
+                CFMetaData baseTable = Schema.instance.getView(table.ksName, table.cfName).baseTableMetadata();
+                if (baseTable.isCompactTable())
+                    throw new InvalidRequestException(String.format("Cannot ALTER schema of view %s.%s on compact table %s from "
+                                                                    + "a connection in NO-COMPACT mode",
+                                                                    table.ksName, table.cfName,
+                                                                    baseTable.ksName, baseTable.cfName));
+            }
+        }
+
         return new Prepared(this);
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
index 2e090fa..1e867bc 100644
--- a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
@@ -887,14 +887,14 @@ public class SelectStatement implements CQLStatement
             this.limit = limit;
         }
 
-        public ParsedStatement.Prepared prepare() throws InvalidRequestException
+        public ParsedStatement.Prepared prepare(ClientState clientState) throws InvalidRequestException
         {
-            return prepare(false);
+            return prepare(false, clientState);
         }
 
-        public ParsedStatement.Prepared prepare(boolean forView) throws InvalidRequestException
+        public ParsedStatement.Prepared prepare(boolean forView, ClientState clientState) throws InvalidRequestException
         {
-            CFMetaData cfm = ThriftValidation.validateColumnFamily(keyspace(), columnFamily());
+            CFMetaData cfm = ThriftValidation.validateColumnFamilyWithCompactMode(keyspace(), columnFamily(), clientState.isNoCompactMode());
             VariableSpecifications boundNames = getBoundVariables();
 
             Selection selection = selectClause.isEmpty()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/src/java/org/apache/cassandra/cql3/statements/TruncateStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/TruncateStatement.java b/src/java/org/apache/cassandra/cql3/statements/TruncateStatement.java
index 66b3da0..b697910 100644
--- a/src/java/org/apache/cassandra/cql3/statements/TruncateStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/TruncateStatement.java
@@ -45,7 +45,7 @@ public class TruncateStatement extends CFStatement implements CQLStatement
         return 0;
     }
 
-    public Prepared prepare() throws InvalidRequestException
+    public Prepared prepare(ClientState clientState) throws InvalidRequestException
     {
         return new Prepared(this);
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/src/java/org/apache/cassandra/cql3/statements/UseStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/UseStatement.java b/src/java/org/apache/cassandra/cql3/statements/UseStatement.java
index fe3d518..e4685cc 100644
--- a/src/java/org/apache/cassandra/cql3/statements/UseStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/UseStatement.java
@@ -39,7 +39,7 @@ public class UseStatement extends ParsedStatement implements CQLStatement
         return 0;
     }
 
-    public Prepared prepare() throws InvalidRequestException
+    public Prepared prepare(ClientState clientState) throws InvalidRequestException
     {
         return new Prepared(this);
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/src/java/org/apache/cassandra/db/view/View.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/view/View.java b/src/java/org/apache/cassandra/db/view/View.java
index 58e2a84..9716dc4 100644
--- a/src/java/org/apache/cassandra/db/view/View.java
+++ b/src/java/org/apache/cassandra/db/view/View.java
@@ -17,32 +17,38 @@
  */
 package org.apache.cassandra.db.view;
 
-import java.nio.ByteBuffer;
-import java.util.*;
-import java.util.concurrent.TimeUnit;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.UUID;
 import java.util.stream.Collectors;
-
 import javax.annotation.Nullable;
 
 import com.google.common.collect.Iterables;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
-import org.apache.cassandra.cql3.*;
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.config.ViewDefinition;
+import org.apache.cassandra.cql3.ColumnIdentifier;
+import org.apache.cassandra.cql3.MultiColumnRelation;
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.Relation;
+import org.apache.cassandra.cql3.SingleColumnRelation;
+import org.apache.cassandra.cql3.Term;
 import org.apache.cassandra.cql3.statements.ParsedStatement;
 import org.apache.cassandra.cql3.statements.SelectStatement;
-import org.apache.cassandra.db.*;
-import org.apache.cassandra.config.*;
-import org.apache.cassandra.cql3.ColumnIdentifier;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.ReadQuery;
 import org.apache.cassandra.db.compaction.CompactionManager;
-import org.apache.cassandra.db.partitions.*;
-import org.apache.cassandra.db.rows.*;
+import org.apache.cassandra.db.rows.Row;
 import org.apache.cassandra.schema.KeyspaceMetadata;
 import org.apache.cassandra.service.ClientState;
-import org.apache.cassandra.service.pager.QueryPager;
-import org.apache.cassandra.transport.Server;
 import org.apache.cassandra.utils.FBUtilities;
-import org.apache.cassandra.utils.btree.BTreeSet;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 /**
  * A View copies data from a base table into a view table which can be queried independently from the
@@ -176,7 +182,7 @@ public class View
             ClientState state = ClientState.forInternalCalls();
             state.setKeyspace(baseCfs.keyspace.getName());
             rawSelect.prepareKeyspace(state);
-            ParsedStatement.Prepared prepared = rawSelect.prepare(true);
+            ParsedStatement.Prepared prepared = rawSelect.prepare(true, ClientState.forInternalCalls());
             select = (SelectStatement) prepared.statement;
         }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/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 c14c5a7..7cf4c51 100644
--- a/src/java/org/apache/cassandra/index/internal/keys/KeysSearcher.java
+++ b/src/java/org/apache/cassandra/index/internal/keys/KeysSearcher.java
@@ -197,7 +197,13 @@ public class KeysSearcher extends CassandraIndexSearcher
         }
         else
         {
-            assert iterator.metadata().isCompactTable();
+            if (!iterator.metadata().isCompactTable())
+            {
+                logger.warn("Non-composite index was used on the table '{}' during the query. Starting from Cassandra 4.0, only " +
+                            "composite indexes will be supported. If compact flags were dropped for this table, drop and re-create " +
+                            "the index.", iterator.metadata().cfName);
+            }
+
             Row data = iterator.staticRow();
             if (index.isStale(data, indexedValue, nowInSec))
             {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/src/java/org/apache/cassandra/repair/RepairRunnable.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/repair/RepairRunnable.java b/src/java/org/apache/cassandra/repair/RepairRunnable.java
index 213e5c5..77726d4 100644
--- a/src/java/org/apache/cassandra/repair/RepairRunnable.java
+++ b/src/java/org/apache/cassandra/repair/RepairRunnable.java
@@ -45,6 +45,7 @@ import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.repair.messages.RepairOption;
 import org.apache.cassandra.service.ActiveRepairService;
+import org.apache.cassandra.service.ClientState;
 import org.apache.cassandra.service.QueryState;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.tracing.TraceKeyspace;
@@ -385,7 +386,7 @@ public class RepairRunnable extends WrappedRunnable implements ProgressEventNoti
 
                 String format = "select event_id, source, activity from %s.%s where session_id = ? and event_id > ? and event_id < ?;";
                 String query = String.format(format, TraceKeyspace.NAME, TraceKeyspace.EVENTS);
-                SelectStatement statement = (SelectStatement) QueryProcessor.parseStatement(query).prepare().statement;
+                SelectStatement statement = (SelectStatement) QueryProcessor.parseStatement(query).prepare(ClientState.forInternalCalls()).statement;
 
                 ByteBuffer sessionIdBytes = ByteBufferUtil.bytes(sessionId);
                 InetAddress source = FBUtilities.getBroadcastAddress();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/src/java/org/apache/cassandra/service/ClientState.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/ClientState.java b/src/java/org/apache/cassandra/service/ClientState.java
index 32849bc..641d174 100644
--- a/src/java/org/apache/cassandra/service/ClientState.java
+++ b/src/java/org/apache/cassandra/service/ClientState.java
@@ -81,6 +81,12 @@ public class ClientState
     private volatile AuthenticatedUser user;
     private volatile String keyspace;
 
+    /**
+     * Force Compact Tables to be represented as CQL ones for the current client session (simulates
+     * ALTER .. DROP COMPACT STORAGE but only for this session)
+     */
+    private volatile boolean noCompactMode;
+
     private static final QueryHandler cqlQueryHandler;
     static
     {
@@ -253,6 +259,16 @@ public class ClientState
         keyspace = ks;
     }
 
+    public void setNoCompactMode()
+    {
+        this.noCompactMode = true;
+    }
+
+    public boolean isNoCompactMode()
+    {
+        return noCompactMode;
+    }
+
     /**
      * Attempts to login the given user.
      */

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/src/java/org/apache/cassandra/thrift/ThriftValidation.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/thrift/ThriftValidation.java b/src/java/org/apache/cassandra/thrift/ThriftValidation.java
index 99aed05..6ad791d 100644
--- a/src/java/org/apache/cassandra/thrift/ThriftValidation.java
+++ b/src/java/org/apache/cassandra/thrift/ThriftValidation.java
@@ -105,6 +105,11 @@ public class ThriftValidation
     // To be used when the operation should be authorized whether this is a counter CF or not
     public static CFMetaData validateColumnFamily(String keyspaceName, String cfName) throws org.apache.cassandra.exceptions.InvalidRequestException
     {
+        return validateColumnFamilyWithCompactMode(keyspaceName, cfName, false);
+    }
+
+    public static CFMetaData validateColumnFamilyWithCompactMode(String keyspaceName, String cfName, boolean noCompactMode) throws org.apache.cassandra.exceptions.InvalidRequestException
+    {
         validateKeyspace(keyspaceName);
         if (cfName.isEmpty())
             throw new org.apache.cassandra.exceptions.InvalidRequestException("non-empty table is required");
@@ -113,7 +118,10 @@ public class ThriftValidation
         if (metadata == null)
             throw new org.apache.cassandra.exceptions.InvalidRequestException("unconfigured table " + cfName);
 
-        return metadata;
+        if (metadata.isCompactTable() && noCompactMode)
+            return metadata.asNonCompact();
+        else
+            return metadata;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/src/java/org/apache/cassandra/transport/messages/StartupMessage.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/transport/messages/StartupMessage.java b/src/java/org/apache/cassandra/transport/messages/StartupMessage.java
index 04d8e62..774be6a 100644
--- a/src/java/org/apache/cassandra/transport/messages/StartupMessage.java
+++ b/src/java/org/apache/cassandra/transport/messages/StartupMessage.java
@@ -35,6 +35,7 @@ public class StartupMessage extends Message.Request
 {
     public static final String CQL_VERSION = "CQL_VERSION";
     public static final String COMPRESSION = "COMPRESSION";
+    public static final String NO_COMPACT = "NO_COMPACT";
 
     public static final Message.Codec<StartupMessage> codec = new Message.Codec<StartupMessage>()
     {
@@ -97,6 +98,9 @@ public class StartupMessage extends Message.Request
             }
         }
 
+        if (options.containsKey(NO_COMPACT) && Boolean.parseBoolean(options.get(NO_COMPACT)))
+            state.getClientState().setNoCompactMode();
+
         if (DatabaseDescriptor.getAuthenticator().requireAuthentication())
             return new AuthenticateMessage(DatabaseDescriptor.getAuthenticator().getClass().getName());
         else

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/test/unit/org/apache/cassandra/cql3/ViewTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/ViewTest.java b/test/unit/org/apache/cassandra/cql3/ViewTest.java
index 4a4fe1a..136ae1c 100644
--- a/test/unit/org/apache/cassandra/cql3/ViewTest.java
+++ b/test/unit/org/apache/cassandra/cql3/ViewTest.java
@@ -48,6 +48,8 @@ import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.SystemKeyspace;
 import org.apache.cassandra.db.compaction.CompactionManager;
 import org.apache.cassandra.db.marshal.AsciiType;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.exceptions.SyntaxException;
 import org.apache.cassandra.schema.KeyspaceParams;
 import org.apache.cassandra.utils.FBUtilities;
 
@@ -1237,7 +1239,6 @@ public class ViewTest extends CQLTester
         catch (Exception e)
         {
         }
-
     }
 
     @Test
@@ -1376,4 +1377,16 @@ public class ViewTest extends CQLTester
         assertRows(execute("SELECT k, toJson(listval) from mv"),
                    row(0, "[[\"a\", \"1\"], [\"b\", \"2\"], [\"c\", \"3\"]]"));
     }
+
+    @Test(expected = SyntaxException.class)
+    public void emptyViewNameTest() throws Throwable
+    {
+        execute("CREATE MATERIALIZED VIEW \"\" AS SELECT a, b FROM tbl WHERE b IS NOT NULL PRIMARY KEY (b, a)");
+    }
+
+     @Test(expected = SyntaxException.class)
+     public void emptyBaseTableNameTest() throws Throwable
+     {
+         execute("CREATE MATERIALIZED VIEW myview AS SELECT a, b FROM \"\" WHERE b IS NOT NULL PRIMARY KEY (b, a)");
+     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/test/unit/org/apache/cassandra/cql3/validation/entities/UFTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/UFTest.java b/test/unit/org/apache/cassandra/cql3/validation/entities/UFTest.java
index 8a743ea..6e6af19 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/entities/UFTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/UFTest.java
@@ -34,6 +34,7 @@ import org.apache.cassandra.cql3.functions.FunctionName;
 import org.apache.cassandra.cql3.functions.UDFunction;
 import org.apache.cassandra.db.marshal.CollectionType;
 import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.exceptions.SyntaxException;
 import org.apache.cassandra.schema.KeyspaceMetadata;
 import org.apache.cassandra.service.ClientState;
 import org.apache.cassandra.transport.Event;
@@ -968,4 +969,28 @@ public class UFTest extends CQLTester
         assertRows(execute("SELECT " + fNameICC + "(empty_int) FROM %s"), row(0));
         assertRows(execute("SELECT " + fNameICN + "(empty_int) FROM %s"), row(new Object[]{ null }));
     }
+
+    @Test(expected = SyntaxException.class)
+    public void testEmptyFunctionName() throws Throwable
+    {
+        execute("CREATE FUNCTION IF NOT EXISTS " + KEYSPACE + ".\"\" (arg int)\n" +
+                "  RETURNS NULL ON NULL INPUT\n" +
+                "  RETURNS int\n" +
+                "  LANGUAGE java\n" +
+                "  AS $$\n" +
+                "    return a;\n" +
+                "  $$");
+    }
+
+    @Test(expected = SyntaxException.class)
+    public void testEmptyArgName() throws Throwable
+    {
+        execute("CREATE FUNCTION IF NOT EXISTS " + KEYSPACE + ".myfn (\"\" int)\n" +
+                "  RETURNS NULL ON NULL INPUT\n" +
+                "  RETURNS int\n" +
+                "  LANGUAGE java\n" +
+                "  AS $$\n" +
+                "    return a;\n" +
+                "  $$");
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/test/unit/org/apache/cassandra/cql3/validation/entities/UserTypesTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/UserTypesTest.java b/test/unit/org/apache/cassandra/cql3/validation/entities/UserTypesTest.java
index dfc2e5e..68c0b8c 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/entities/UserTypesTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/UserTypesTest.java
@@ -24,6 +24,7 @@ import org.junit.Test;
 
 import org.apache.cassandra.cql3.CQLTester;
 import org.apache.cassandra.dht.ByteOrderedPartitioner;
+import org.apache.cassandra.exceptions.*;
 import org.apache.cassandra.service.StorageService;
 
 public class UserTypesTest extends CQLTester
@@ -713,6 +714,25 @@ public class UserTypesTest extends CQLTester
                        row(1, 1,set(userType(1), userType(1, 1), userType(1, 2), userType(2), userType(2, 1)), 2));
     }
 
+    @Test(expected = SyntaxException.class)
+    public void emptyTypeNameTest() throws Throwable
+    {
+        execute("CREATE TYPE \"\" (a int, b int)");
+    }
+
+    @Test(expected = SyntaxException.class)
+    public void emptyFieldNameTest() throws Throwable
+    {
+        execute("CREATE TYPE mytype (\"\" int, b int)");
+    }
+
+    @Test(expected = SyntaxException.class)
+    public void renameColumnToEmpty() throws Throwable
+    {
+        String typeName = createType("CREATE TYPE %s (a int, b int)");
+        execute(String.format("ALTER TYPE %s.%s RENAME b TO \"\"", keyspace(), typeName));
+    }
+
     private String typeWithKs(String type1)
     {
         return keyspace() + '.' + type1;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/test/unit/org/apache/cassandra/cql3/validation/operations/AlterTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/AlterTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/AlterTest.java
index c48ffe5..b37462f 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/AlterTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/AlterTest.java
@@ -17,6 +17,9 @@
  */
 package org.apache.cassandra.cql3.validation.operations;
 
+import org.junit.Assert;
+import org.junit.Test;
+
 import org.apache.cassandra.cql3.CQLTester;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.Keyspace;
@@ -24,11 +27,6 @@ import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.exceptions.SyntaxException;
 import org.apache.cassandra.schema.SchemaKeyspace;
-import org.apache.cassandra.transport.Server;
-import org.apache.cassandra.utils.ByteBufferUtil;
-
-import org.junit.Assert;
-import org.junit.Test;
 
 import static java.lang.String.format;
 import static org.junit.Assert.assertEquals;
@@ -266,6 +264,13 @@ public class AlterTest extends CQLTester
         execute("alter table %s add v int");
     }
 
+    @Test(expected = SyntaxException.class)
+    public void renameToEmptyTest() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int, c1 int, v int, PRIMARY KEY (k, c1))");
+        execute("ALTER TABLE %s RENAME c1 TO \"\"");
+    }
+
     @Test
     // tests CASSANDRA-9565
     public void testDoubleWith() throws Throwable


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


[13/25] cassandra git commit: Merge branch 'cassandra-3.0' into cassandra-3.11

Posted by if...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d429cd0/test/unit/org/apache/cassandra/index/sasi/SASIIndexTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/index/sasi/SASIIndexTest.java
index 03d89e1,0000000..1b2f97d
mode 100644,000000..100644
--- a/test/unit/org/apache/cassandra/index/sasi/SASIIndexTest.java
+++ b/test/unit/org/apache/cassandra/index/sasi/SASIIndexTest.java
@@@ -1,2568 -1,0 +1,2569 @@@
 +/*
 + * 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.index.sasi;
 +
 +import java.io.FileWriter;
 +import java.io.Writer;
 +import java.nio.ByteBuffer;
 +import java.nio.file.FileSystems;
 +import java.nio.file.Files;
 +import java.nio.file.Path;
 +import java.nio.file.attribute.BasicFileAttributes;
 +import java.util.*;
 +import java.util.concurrent.ExecutorService;
 +import java.util.concurrent.Executors;
 +import java.util.concurrent.ThreadLocalRandom;
 +import java.util.concurrent.TimeUnit;
 +import java.util.concurrent.atomic.AtomicInteger;
 +
 +import org.apache.cassandra.SchemaLoader;
 +import org.apache.cassandra.config.CFMetaData;
 +import org.apache.cassandra.config.ColumnDefinition;
 +import org.apache.cassandra.index.Index;
 +import org.apache.cassandra.config.DatabaseDescriptor;
 +import org.apache.cassandra.cql3.*;
 +import org.apache.cassandra.cql3.Term;
 +import org.apache.cassandra.cql3.statements.IndexTarget;
 +import org.apache.cassandra.cql3.statements.SelectStatement;
 +import org.apache.cassandra.db.*;
 +import org.apache.cassandra.db.filter.ColumnFilter;
 +import org.apache.cassandra.db.filter.DataLimits;
 +import org.apache.cassandra.db.filter.RowFilter;
 +import org.apache.cassandra.db.lifecycle.SSTableSet;
 +import org.apache.cassandra.db.marshal.*;
 +import org.apache.cassandra.db.partitions.PartitionUpdate;
 +import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator;
 +import org.apache.cassandra.db.rows.*;
 +import org.apache.cassandra.dht.IPartitioner;
 +import org.apache.cassandra.dht.Murmur3Partitioner;
 +import org.apache.cassandra.dht.Range;
 +import org.apache.cassandra.exceptions.ConfigurationException;
 +import org.apache.cassandra.exceptions.InvalidRequestException;
 +import org.apache.cassandra.index.sasi.conf.ColumnIndex;
 +import org.apache.cassandra.index.sasi.disk.OnDiskIndexBuilder;
 +import org.apache.cassandra.index.sasi.exceptions.TimeQuotaExceededException;
 +import org.apache.cassandra.index.sasi.memory.IndexMemtable;
 +import org.apache.cassandra.index.sasi.plan.QueryController;
 +import org.apache.cassandra.index.sasi.plan.QueryPlan;
 +import org.apache.cassandra.io.sstable.SSTable;
 +import org.apache.cassandra.schema.IndexMetadata;
 +import org.apache.cassandra.schema.KeyspaceMetadata;
 +import org.apache.cassandra.schema.KeyspaceParams;
 +import org.apache.cassandra.schema.Tables;
 +import org.apache.cassandra.serializers.MarshalException;
 +import org.apache.cassandra.serializers.TypeSerializer;
++import org.apache.cassandra.service.ClientState;
 +import org.apache.cassandra.service.MigrationManager;
 +import org.apache.cassandra.service.QueryState;
 +import org.apache.cassandra.thrift.CqlRow;
 +import org.apache.cassandra.transport.messages.ResultMessage;
 +import org.apache.cassandra.utils.ByteBufferUtil;
 +import org.apache.cassandra.utils.FBUtilities;
 +import org.apache.cassandra.utils.Pair;
 +
 +import com.google.common.collect.Lists;
 +import com.google.common.util.concurrent.Uninterruptibles;
 +
 +import junit.framework.Assert;
 +
 +import org.junit.*;
 +
 +public class SASIIndexTest
 +{
 +    private static final IPartitioner PARTITIONER;
 +
 +    static {
 +        System.setProperty("cassandra.config", "cassandra-murmur.yaml");
 +        PARTITIONER = Murmur3Partitioner.instance;
 +    }
 +
 +    private static final String KS_NAME = "sasi";
 +    private static final String CF_NAME = "test_cf";
 +    private static final String CLUSTERING_CF_NAME_1 = "clustering_test_cf_1";
 +    private static final String CLUSTERING_CF_NAME_2 = "clustering_test_cf_2";
 +    private static final String STATIC_CF_NAME = "static_sasi_test_cf";
 +    private static final String FTS_CF_NAME = "full_text_search_sasi_test_cf";
 +
 +    @BeforeClass
 +    public static void loadSchema() throws ConfigurationException
 +    {
 +        SchemaLoader.loadSchema();
 +        MigrationManager.announceNewKeyspace(KeyspaceMetadata.create(KS_NAME,
 +                                                                     KeyspaceParams.simpleTransient(1),
 +                                                                     Tables.of(SchemaLoader.sasiCFMD(KS_NAME, CF_NAME),
 +                                                                               SchemaLoader.clusteringSASICFMD(KS_NAME, CLUSTERING_CF_NAME_1),
 +                                                                               SchemaLoader.clusteringSASICFMD(KS_NAME, CLUSTERING_CF_NAME_2, "location"),
 +                                                                               SchemaLoader.staticSASICFMD(KS_NAME, STATIC_CF_NAME),
 +                                                                               SchemaLoader.fullTextSearchSASICFMD(KS_NAME, FTS_CF_NAME))));
 +    }
 +
 +    @Before
 +    public void cleanUp()
 +    {
 +        Keyspace.open(KS_NAME).getColumnFamilyStore(CF_NAME).truncateBlocking();
 +    }
 +
 +    @Test
 +    public void testSingleExpressionQueries() throws Exception
 +    {
 +        testSingleExpressionQueries(false);
 +        cleanupData();
 +        testSingleExpressionQueries(true);
 +    }
 +
 +    private void testSingleExpressionQueries(boolean forceFlush) throws Exception
 +    {
 +        Map<String, Pair<String, Integer>> data = new HashMap<String, Pair<String, Integer>>()
 +        {{
 +            put("key1", Pair.create("Pavel", 14));
 +            put("key2", Pair.create("Pavel", 26));
 +            put("key3", Pair.create("Pavel", 27));
 +            put("key4", Pair.create("Jason", 27));
 +        }};
 +
 +        ColumnFamilyStore store = loadData(data, forceFlush);
 +
 +        final ByteBuffer firstName = UTF8Type.instance.decompose("first_name");
 +        final ByteBuffer age = UTF8Type.instance.decompose("age");
 +
 +        Set<String> rows;
 +
 +        rows = getIndexed(store, 10, buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1", "key2", "key3", "key4" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("av")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1", "key2", "key3" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("as")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key4" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("aw")));
 +        Assert.assertEquals(rows.toString(), 0, rows.size());
 +
 +        rows = getIndexed(store, 10, buildExpression(firstName, Operator.LIKE_SUFFIX, UTF8Type.instance.decompose("avel")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1", "key2", "key3" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(firstName, Operator.LIKE_SUFFIX, UTF8Type.instance.decompose("n")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key4" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(age, Operator.EQ, Int32Type.instance.decompose(27)));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[]{"key3", "key4"}, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(age, Operator.EQ, Int32Type.instance.decompose(26)));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key2" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(age, Operator.EQ, Int32Type.instance.decompose(13)));
 +        Assert.assertEquals(rows.toString(), 0, rows.size());
 +    }
 +
 +    @Test
 +    public void testEmptyTokenizedResults() throws Exception
 +    {
 +        testEmptyTokenizedResults(false);
 +        cleanupData();
 +        testEmptyTokenizedResults(true);
 +    }
 +
 +    private void testEmptyTokenizedResults(boolean forceFlush) throws Exception
 +    {
 +        Map<String, Pair<String, Integer>> data = new HashMap<String, Pair<String, Integer>>()
 +        {{
 +                put("key1", Pair.create("  ", 14));
 +        }};
 +
 +        ColumnFamilyStore store = loadData(data, forceFlush);
 +
 +        Set<String> rows= getIndexed(store, 10, buildExpression(UTF8Type.instance.decompose("first_name"), Operator.LIKE_MATCHES, UTF8Type.instance.decompose("doesntmatter")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[]{}, rows.toArray(new String[rows.size()])));
 +    }
 +
 +    @Test
 +    public void testMultiExpressionQueries() throws Exception
 +    {
 +        testMultiExpressionQueries(false);
 +        cleanupData();
 +        testMultiExpressionQueries(true);
 +    }
 +
 +    public void testMultiExpressionQueries(boolean forceFlush) throws Exception
 +    {
 +        Map<String, Pair<String, Integer>> data = new HashMap<String, Pair<String, Integer>>()
 +        {{
 +                put("key1", Pair.create("Pavel", 14));
 +                put("key2", Pair.create("Pavel", 26));
 +                put("key3", Pair.create("Pavel", 27));
 +                put("key4", Pair.create("Jason", 27));
 +        }};
 +
 +        ColumnFamilyStore store = loadData(data, forceFlush);
 +
 +        final ByteBuffer firstName = UTF8Type.instance.decompose("first_name");
 +        final ByteBuffer age = UTF8Type.instance.decompose("age");
 +
 +        Set<String> rows;
 +        rows = getIndexed(store, 10,
 +                          buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                          buildExpression(age, Operator.GT, Int32Type.instance.decompose(14)));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key2", "key3", "key4" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10,
 +                          buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                          buildExpression(age, Operator.LT, Int32Type.instance.decompose(27)));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[]{"key1", "key2"}, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10,
 +                         buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                         buildExpression(age, Operator.GT, Int32Type.instance.decompose(14)),
 +                         buildExpression(age, Operator.LT, Int32Type.instance.decompose(27)));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key2" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10,
 +                         buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                         buildExpression(age, Operator.GT, Int32Type.instance.decompose(12)));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[]{ "key1", "key2", "key3", "key4" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10,
 +                         buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                         buildExpression(age, Operator.GTE, Int32Type.instance.decompose(13)));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1", "key2", "key3", "key4" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10,
 +                         buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                         buildExpression(age, Operator.GTE, Int32Type.instance.decompose(16)));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key2", "key3", "key4" }, rows.toArray(new String[rows.size()])));
 +
 +
 +        rows = getIndexed(store, 10,
 +                         buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                         buildExpression(age, Operator.LT, Int32Type.instance.decompose(30)));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1", "key2", "key3", "key4" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10,
 +                         buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                         buildExpression(age, Operator.LTE, Int32Type.instance.decompose(29)));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1", "key2", "key3", "key4" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10,
 +                         buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                         buildExpression(age, Operator.LTE, Int32Type.instance.decompose(25)));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[]{ "key1" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(firstName, Operator.LIKE_SUFFIX, UTF8Type.instance.decompose("avel")),
 +                                     buildExpression(age, Operator.LTE, Int32Type.instance.decompose(25)));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(firstName, Operator.LIKE_SUFFIX, UTF8Type.instance.decompose("n")),
 +                                     buildExpression(age, Operator.LTE, Int32Type.instance.decompose(25)));
 +        Assert.assertTrue(rows.isEmpty());
 +
 +    }
 +
 +    @Test
 +    public void testCrossSSTableQueries() throws Exception
 +    {
 +        testCrossSSTableQueries(false);
 +        cleanupData();
 +        testCrossSSTableQueries(true);
 +
 +    }
 +
 +    private void testCrossSSTableQueries(boolean forceFlush) throws Exception
 +    {
 +        Map<String, Pair<String, Integer>> part1 = new HashMap<String, Pair<String, Integer>>()
 +        {{
 +                put("key0", Pair.create("Maxie", 43));
 +                put("key1", Pair.create("Chelsie", 33));
 +                put("key2", Pair.create("Josephine", 43));
 +                put("key3", Pair.create("Shanna", 27));
 +                put("key4", Pair.create("Amiya", 36));
 +            }};
 +
 +        loadData(part1, forceFlush); // first sstable
 +
 +        Map<String, Pair<String, Integer>> part2 = new HashMap<String, Pair<String, Integer>>()
 +        {{
 +                put("key5", Pair.create("Americo", 20));
 +                put("key6", Pair.create("Fiona", 39));
 +                put("key7", Pair.create("Francis", 41));
 +                put("key8", Pair.create("Charley", 21));
 +                put("key9", Pair.create("Amely", 40));
 +            }};
 +
 +        loadData(part2, forceFlush);
 +
 +        Map<String, Pair<String, Integer>> part3 = new HashMap<String, Pair<String, Integer>>()
 +        {{
 +                put("key10", Pair.create("Eddie", 42));
 +                put("key11", Pair.create("Oswaldo", 35));
 +                put("key12", Pair.create("Susana", 35));
 +                put("key13", Pair.create("Alivia", 42));
 +                put("key14", Pair.create("Demario", 28));
 +            }};
 +
 +        ColumnFamilyStore store = loadData(part3, forceFlush);
 +
 +        final ByteBuffer firstName = UTF8Type.instance.decompose("first_name");
 +        final ByteBuffer age = UTF8Type.instance.decompose("age");
 +
 +        Set<String> rows;
 +        rows = getIndexed(store, 10, buildExpression(firstName, Operator.EQ, UTF8Type.instance.decompose("Fiona")),
 +                                     buildExpression(age, Operator.LT, Int32Type.instance.decompose(40)));
 +
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key6" }, rows.toArray(new String[rows.size()])));
 +
 +
 +        rows = getIndexed(store, 10,
 +                          buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")));
 +
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key0", "key11", "key12", "key13", "key14",
 +                                                                        "key3", "key4", "key6", "key7", "key8" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 5,
 +                          buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")));
 +
 +        Assert.assertEquals(rows.toString(), 5, rows.size());
 +
 +        rows = getIndexed(store, 10,
 +                          buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                          buildExpression(age, Operator.GTE, Int32Type.instance.decompose(35)));
 +
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key0", "key11", "key12", "key13", "key4", "key6", "key7" },
 +                                                         rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10,
 +                          buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                          buildExpression(age, Operator.LT, Int32Type.instance.decompose(32)));
 +
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key14", "key3", "key8" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10,
 +                          buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                          buildExpression(age, Operator.GT, Int32Type.instance.decompose(27)),
 +                          buildExpression(age, Operator.LT, Int32Type.instance.decompose(32)));
 +
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key14" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10,
 +                          buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                          buildExpression(age, Operator.GT, Int32Type.instance.decompose(10)));
 +
 +        Assert.assertEquals(rows.toString(), 10, rows.size());
 +
 +        rows = getIndexed(store, 10,
 +                          buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                          buildExpression(age, Operator.LTE, Int32Type.instance.decompose(50)));
 +
 +        Assert.assertEquals(rows.toString(), 10, rows.size());
 +
 +        rows = getIndexed(store, 10,
 +                          buildExpression(firstName, Operator.LIKE_SUFFIX, UTF8Type.instance.decompose("ie")),
 +                          buildExpression(age, Operator.LT, Int32Type.instance.decompose(43)));
 +
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1", "key10" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10,
 +                          buildExpression(firstName, Operator.LIKE_SUFFIX, UTF8Type.instance.decompose("a")));
 +
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key12", "key13", "key3", "key4", "key6" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10,
 +                          buildExpression(firstName, Operator.LIKE_SUFFIX, UTF8Type.instance.decompose("a")),
 +                          buildExpression(age, Operator.LT, Int32Type.instance.decompose(33)));
 +
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key3" }, rows.toArray(new String[rows.size()])));
 +    }
 +
 +    @Test
 +    public void testQueriesThatShouldBeTokenized() throws Exception
 +    {
 +        testQueriesThatShouldBeTokenized(false);
 +        cleanupData();
 +        testQueriesThatShouldBeTokenized(true);
 +    }
 +
 +    private void testQueriesThatShouldBeTokenized(boolean forceFlush) throws Exception
 +    {
 +        Map<String, Pair<String, Integer>> part1 = new HashMap<String, Pair<String, Integer>>()
 +        {{
 +                put("key0", Pair.create("If you can dream it, you can do it.", 43));
 +                put("key1", Pair.create("What you get by achieving your goals is not " +
 +                        "as important as what you become by achieving your goals, do it.", 33));
 +                put("key2", Pair.create("Keep your face always toward the sunshine " +
 +                        "- and shadows will fall behind you.", 43));
 +                put("key3", Pair.create("We can't help everyone, but everyone can " +
 +                        "help someone.", 27));
 +            }};
 +
 +        ColumnFamilyStore store = loadData(part1, forceFlush);
 +
 +        final ByteBuffer firstName = UTF8Type.instance.decompose("first_name");
 +        final ByteBuffer age = UTF8Type.instance.decompose("age");
 +
 +        Set<String> rows = getIndexed(store, 10,
 +                buildExpression(firstName, Operator.LIKE_CONTAINS,
 +                        UTF8Type.instance.decompose("What you get by achieving your goals")),
 +                buildExpression(age, Operator.GT, Int32Type.instance.decompose(32)));
 +
 +        Assert.assertEquals(rows.toString(), Collections.singleton("key1"), rows);
 +
 +        rows = getIndexed(store, 10,
 +                buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("do it.")));
 +
 +        Assert.assertEquals(rows.toString(), Arrays.asList("key0", "key1"), Lists.newArrayList(rows));
 +    }
 +
 +    @Test
 +    public void testPrefixSearchWithContainsMode() throws Exception
 +    {
 +        testPrefixSearchWithContainsMode(false);
 +        cleanupData();
 +        testPrefixSearchWithContainsMode(true);
 +    }
 +
 +    private void testPrefixSearchWithContainsMode(boolean forceFlush) throws Exception
 +    {
 +        ColumnFamilyStore store = Keyspace.open(KS_NAME).getColumnFamilyStore(FTS_CF_NAME);
 +
 +        executeCQL(FTS_CF_NAME, "INSERT INTO %s.%s (song_id, title, artist) VALUES(?, ?, ?)", UUID.fromString("1a4abbcd-b5de-4c69-a578-31231e01ff09"), "Poker Face", "Lady Gaga");
 +        executeCQL(FTS_CF_NAME, "INSERT INTO %s.%s (song_id, title, artist) VALUES(?, ?, ?)", UUID.fromString("9472a394-359b-4a06-b1d5-b6afce590598"), "Forgetting the Way Home", "Our Lady of Bells");
 +        executeCQL(FTS_CF_NAME, "INSERT INTO %s.%s (song_id, title, artist) VALUES(?, ?, ?)", UUID.fromString("4f8dc18e-54e6-4e16-b507-c5324b61523b"), "Zamki na piasku", "Lady Pank");
 +        executeCQL(FTS_CF_NAME, "INSERT INTO %s.%s (song_id, title, artist) VALUES(?, ?, ?)", UUID.fromString("eaf294fa-bad5-49d4-8f08-35ba3636a706"), "Koncertowa", "Lady Pank");
 +
 +
 +        if (forceFlush)
 +            store.forceBlockingFlush();
 +
 +        final UntypedResultSet results = executeCQL(FTS_CF_NAME, "SELECT * FROM %s.%s WHERE artist LIKE 'lady%%'");
 +        Assert.assertNotNull(results);
 +        Assert.assertEquals(3, results.size());
 +    }
 +
 +    @Test
 +    public void testMultiExpressionQueriesWhereRowSplitBetweenSSTables() throws Exception
 +    {
 +        testMultiExpressionQueriesWhereRowSplitBetweenSSTables(false);
 +        cleanupData();
 +        testMultiExpressionQueriesWhereRowSplitBetweenSSTables(true);
 +    }
 +
 +    private void testMultiExpressionQueriesWhereRowSplitBetweenSSTables(boolean forceFlush) throws Exception
 +    {
 +        Map<String, Pair<String, Integer>> part1 = new HashMap<String, Pair<String, Integer>>()
 +        {{
 +                put("key0", Pair.create("Maxie", -1));
 +                put("key1", Pair.create("Chelsie", 33));
 +                put("key2", Pair.create((String)null, 43));
 +                put("key3", Pair.create("Shanna", 27));
 +                put("key4", Pair.create("Amiya", 36));
 +        }};
 +
 +        loadData(part1, forceFlush); // first sstable
 +
 +        Map<String, Pair<String, Integer>> part2 = new HashMap<String, Pair<String, Integer>>()
 +        {{
 +                put("key5", Pair.create("Americo", 20));
 +                put("key6", Pair.create("Fiona", 39));
 +                put("key7", Pair.create("Francis", 41));
 +                put("key8", Pair.create("Charley", 21));
 +                put("key9", Pair.create("Amely", 40));
 +                put("key14", Pair.create((String)null, 28));
 +        }};
 +
 +        loadData(part2, forceFlush);
 +
 +        Map<String, Pair<String, Integer>> part3 = new HashMap<String, Pair<String, Integer>>()
 +        {{
 +                put("key0", Pair.create((String)null, 43));
 +                put("key10", Pair.create("Eddie", 42));
 +                put("key11", Pair.create("Oswaldo", 35));
 +                put("key12", Pair.create("Susana", 35));
 +                put("key13", Pair.create("Alivia", 42));
 +                put("key14", Pair.create("Demario", -1));
 +                put("key2", Pair.create("Josephine", -1));
 +        }};
 +
 +        ColumnFamilyStore store = loadData(part3, forceFlush);
 +
 +        final ByteBuffer firstName = UTF8Type.instance.decompose("first_name");
 +        final ByteBuffer age = UTF8Type.instance.decompose("age");
 +
 +        Set<String> rows = getIndexed(store, 10,
 +                                      buildExpression(firstName, Operator.EQ, UTF8Type.instance.decompose("Fiona")),
 +                                      buildExpression(age, Operator.LT, Int32Type.instance.decompose(40)));
 +
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key6" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10,
 +                          buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")));
 +
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key0", "key11", "key12", "key13", "key14",
 +                                                                        "key3", "key4", "key6", "key7", "key8" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 5,
 +                          buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")));
 +
 +        Assert.assertEquals(rows.toString(), 5, rows.size());
 +
 +        rows = getIndexed(store, 10,
 +                          buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                          buildExpression(age, Operator.GTE, Int32Type.instance.decompose(35)));
 +
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key0", "key11", "key12", "key13", "key4", "key6", "key7" },
 +                                                         rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10,
 +                          buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                          buildExpression(age, Operator.LT, Int32Type.instance.decompose(32)));
 +
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key14", "key3", "key8" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10,
 +                          buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                          buildExpression(age, Operator.GT, Int32Type.instance.decompose(27)),
 +                          buildExpression(age, Operator.LT, Int32Type.instance.decompose(32)));
 +
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key14" }, rows.toArray(new String[rows.size()])));
 +
 +        Map<String, Pair<String, Integer>> part4 = new HashMap<String, Pair<String, Integer>>()
 +        {{
 +                put("key12", Pair.create((String)null, 12));
 +                put("key14", Pair.create("Demario", 42));
 +                put("key2", Pair.create("Frank", -1));
 +        }};
 +
 +        store = loadData(part4, forceFlush);
 +
 +        rows = getIndexed(store, 10,
 +                          buildExpression(firstName, Operator.LIKE_MATCHES, UTF8Type.instance.decompose("Susana")),
 +                          buildExpression(age, Operator.LTE, Int32Type.instance.decompose(13)),
 +                          buildExpression(age, Operator.GT, Int32Type.instance.decompose(10)));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key12" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10,
 +                          buildExpression(firstName, Operator.LIKE_MATCHES, UTF8Type.instance.decompose("Demario")),
 +                          buildExpression(age, Operator.LTE, Int32Type.instance.decompose(30)));
 +        Assert.assertTrue(rows.toString(), rows.size() == 0);
 +
 +        rows = getIndexed(store, 10,
 +                          buildExpression(firstName, Operator.LIKE_MATCHES, UTF8Type.instance.decompose("Josephine")));
 +        Assert.assertTrue(rows.toString(), rows.size() == 0);
 +
 +        rows = getIndexed(store, 10,
 +                          buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                          buildExpression(age, Operator.GT, Int32Type.instance.decompose(10)));
 +
 +        Assert.assertEquals(rows.toString(), 10, rows.size());
 +
 +        rows = getIndexed(store, 10,
 +                          buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                          buildExpression(age, Operator.LTE, Int32Type.instance.decompose(50)));
 +
 +        Assert.assertEquals(rows.toString(), 10, rows.size());
 +
 +        rows = getIndexed(store, 10,
 +                          buildExpression(firstName, Operator.LIKE_SUFFIX, UTF8Type.instance.decompose("ie")),
 +                          buildExpression(age, Operator.LTE, Int32Type.instance.decompose(43)));
 +
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key0", "key1", "key10" }, rows.toArray(new String[rows.size()])));
 +    }
 +
 +    @Test
 +    public void testPagination() throws Exception
 +    {
 +        testPagination(false);
 +        cleanupData();
 +        testPagination(true);
 +    }
 +
 +    private void testPagination(boolean forceFlush) throws Exception
 +    {
 +        // split data into 3 distinct SSTables to test paging with overlapping token intervals.
 +
 +        Map<String, Pair<String, Integer>> part1 = new HashMap<String, Pair<String, Integer>>()
 +        {{
 +                put("key01", Pair.create("Ali", 33));
 +                put("key02", Pair.create("Jeremy", 41));
 +                put("key03", Pair.create("Elvera", 22));
 +                put("key04", Pair.create("Bailey", 45));
 +                put("key05", Pair.create("Emerson", 32));
 +                put("key06", Pair.create("Kadin", 38));
 +                put("key07", Pair.create("Maggie", 36));
 +                put("key08", Pair.create("Kailey", 36));
 +                put("key09", Pair.create("Armand", 21));
 +                put("key10", Pair.create("Arnold", 35));
 +        }};
 +
 +        Map<String, Pair<String, Integer>> part2 = new HashMap<String, Pair<String, Integer>>()
 +        {{
 +                put("key11", Pair.create("Ken", 38));
 +                put("key12", Pair.create("Penelope", 43));
 +                put("key13", Pair.create("Wyatt", 34));
 +                put("key14", Pair.create("Johnpaul", 34));
 +                put("key15", Pair.create("Trycia", 43));
 +                put("key16", Pair.create("Aida", 21));
 +                put("key17", Pair.create("Devon", 42));
 +        }};
 +
 +        Map<String, Pair<String, Integer>> part3 = new HashMap<String, Pair<String, Integer>>()
 +        {{
 +                put("key18", Pair.create("Christina", 20));
 +                put("key19", Pair.create("Rick", 19));
 +                put("key20", Pair.create("Fannie", 22));
 +                put("key21", Pair.create("Keegan", 29));
 +                put("key22", Pair.create("Ignatius", 36));
 +                put("key23", Pair.create("Ellis", 26));
 +                put("key24", Pair.create("Annamarie", 29));
 +                put("key25", Pair.create("Tianna", 31));
 +                put("key26", Pair.create("Dennis", 32));
 +        }};
 +
 +        ColumnFamilyStore store = loadData(part1, forceFlush);
 +
 +        loadData(part2, forceFlush);
 +        loadData(part3, forceFlush);
 +
 +        final ByteBuffer firstName = UTF8Type.instance.decompose("first_name");
 +        final ByteBuffer age = UTF8Type.instance.decompose("age");
 +
 +        Set<DecoratedKey> uniqueKeys = getPaged(store, 4,
 +                buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                buildExpression(age, Operator.GTE, Int32Type.instance.decompose(21)));
 +
 +
 +        List<String> expected = new ArrayList<String>()
 +        {{
 +                add("key25");
 +                add("key20");
 +                add("key13");
 +                add("key22");
 +                add("key09");
 +                add("key14");
 +                add("key16");
 +                add("key24");
 +                add("key03");
 +                add("key04");
 +                add("key08");
 +                add("key07");
 +                add("key15");
 +                add("key06");
 +                add("key21");
 +        }};
 +
 +        Assert.assertEquals(expected, convert(uniqueKeys));
 +
 +        // now let's test a single equals condition
 +
 +        uniqueKeys = getPaged(store, 4, buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")));
 +
 +        expected = new ArrayList<String>()
 +        {{
 +                add("key25");
 +                add("key20");
 +                add("key13");
 +                add("key22");
 +                add("key09");
 +                add("key14");
 +                add("key16");
 +                add("key24");
 +                add("key03");
 +                add("key04");
 +                add("key18");
 +                add("key08");
 +                add("key07");
 +                add("key15");
 +                add("key06");
 +                add("key21");
 +        }};
 +
 +        Assert.assertEquals(expected, convert(uniqueKeys));
 +
 +        // now let's test something which is smaller than a single page
 +        uniqueKeys = getPaged(store, 4,
 +                              buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                              buildExpression(age, Operator.EQ, Int32Type.instance.decompose(36)));
 +
 +        expected = new ArrayList<String>()
 +        {{
 +                add("key22");
 +                add("key08");
 +                add("key07");
 +        }};
 +
 +        Assert.assertEquals(expected, convert(uniqueKeys));
 +
 +        // the same but with the page size of 2 to test minimal pagination windows
 +
 +        uniqueKeys = getPaged(store, 2,
 +                              buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                              buildExpression(age, Operator.EQ, Int32Type.instance.decompose(36)));
 +
 +        Assert.assertEquals(expected, convert(uniqueKeys));
 +
 +        // and last but not least, test age range query with pagination
 +        uniqueKeys = getPaged(store, 4,
 +                buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                buildExpression(age, Operator.GT, Int32Type.instance.decompose(20)),
 +                buildExpression(age, Operator.LTE, Int32Type.instance.decompose(36)));
 +
 +        expected = new ArrayList<String>()
 +        {{
 +                add("key25");
 +                add("key20");
 +                add("key13");
 +                add("key22");
 +                add("key09");
 +                add("key14");
 +                add("key16");
 +                add("key24");
 +                add("key03");
 +                add("key08");
 +                add("key07");
 +                add("key21");
 +        }};
 +
 +        Assert.assertEquals(expected, convert(uniqueKeys));
 +
 +        Set<String> rows;
 +
 +        rows = executeCQLWithKeys(String.format("SELECT * FROM %s.%s WHERE first_name LIKE '%%a%%' limit 10 ALLOW FILTERING;", KS_NAME, CF_NAME));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key03", "key04", "key09", "key13", "key14", "key16", "key20", "key22", "key24", "key25" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = executeCQLWithKeys(String.format("SELECT * FROM %s.%s WHERE first_name LIKE '%%a%%' and token(id) >= token('key14') limit 5 ALLOW FILTERING;", KS_NAME, CF_NAME));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key03", "key04", "key14", "key16", "key24" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = executeCQLWithKeys(String.format("SELECT * FROM %s.%s WHERE first_name LIKE '%%a%%' and token(id) >= token('key14') and token(id) <= token('key24') limit 5 ALLOW FILTERING;", KS_NAME, CF_NAME));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key14", "key16", "key24" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = executeCQLWithKeys(String.format("SELECT * FROM %s.%s WHERE first_name LIKE '%%a%%' and age > 30 and token(id) >= token('key14') and token(id) <= token('key24') limit 5 ALLOW FILTERING;", KS_NAME, CF_NAME));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key14" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = executeCQLWithKeys(String.format("SELECT * FROM %s.%s WHERE first_name like '%%ie' limit 5 ALLOW FILTERING;", KS_NAME, CF_NAME));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key07", "key20", "key24" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = executeCQLWithKeys(String.format("SELECT * FROM %s.%s WHERE first_name like '%%ie' AND token(id) > token('key24') limit 5 ALLOW FILTERING;", KS_NAME, CF_NAME));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key07", "key24" }, rows.toArray(new String[rows.size()])));
 +    }
 +
 +    @Test
 +    public void testColumnNamesWithSlashes() throws Exception
 +    {
 +        testColumnNamesWithSlashes(false);
 +        cleanupData();
 +        testColumnNamesWithSlashes(true);
 +    }
 +
 +    private void testColumnNamesWithSlashes(boolean forceFlush) throws Exception
 +    {
 +        ColumnFamilyStore store = Keyspace.open(KS_NAME).getColumnFamilyStore(CF_NAME);
 +
 +        Mutation rm1 = new Mutation(KS_NAME, decoratedKey(AsciiType.instance.decompose("key1")));
 +        rm1.add(PartitionUpdate.singleRowUpdate(store.metadata,
 +                                                rm1.key(),
 +                                                buildRow(buildCell(store.metadata,
 +                                                                   UTF8Type.instance.decompose("/data/output/id"),
 +                                                                   AsciiType.instance.decompose("jason"),
 +                                                                   System.currentTimeMillis()))));
 +
 +        Mutation rm2 = new Mutation(KS_NAME, decoratedKey(AsciiType.instance.decompose("key2")));
 +        rm2.add(PartitionUpdate.singleRowUpdate(store.metadata,
 +                                                rm2.key(),
 +                                                buildRow(buildCell(store.metadata,
 +                                                                   UTF8Type.instance.decompose("/data/output/id"),
 +                                                                   AsciiType.instance.decompose("pavel"),
 +                                                                   System.currentTimeMillis()))));
 +
 +        Mutation rm3 = new Mutation(KS_NAME, decoratedKey(AsciiType.instance.decompose("key3")));
 +        rm3.add(PartitionUpdate.singleRowUpdate(store.metadata,
 +                                                rm3.key(),
 +                                                buildRow(buildCell(store.metadata,
 +                                                                   UTF8Type.instance.decompose("/data/output/id"),
 +                                                                   AsciiType.instance.decompose("Aleksey"),
 +                                                                   System.currentTimeMillis()))));
 +
 +        rm1.apply();
 +        rm2.apply();
 +        rm3.apply();
 +
 +        if (forceFlush)
 +            store.forceBlockingFlush();
 +
 +        final ByteBuffer dataOutputId = UTF8Type.instance.decompose("/data/output/id");
 +
 +        Set<String> rows = getIndexed(store, 10, buildExpression(dataOutputId, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1", "key2" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(dataOutputId, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("A")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[]{ "key3" }, rows.toArray(new String[rows.size()])));
 +
 +        // doesn't really make sense to rebuild index for in-memory data
 +        if (!forceFlush)
 +            return;
 +
 +        store.indexManager.invalidateAllIndexesBlocking();
 +
 +        rows = getIndexed(store, 10, buildExpression(dataOutputId, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")));
 +        Assert.assertTrue(rows.toString(), rows.isEmpty());
 +
 +        rows = getIndexed(store, 10, buildExpression(dataOutputId, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("A")));
 +        Assert.assertTrue(rows.toString(), rows.isEmpty());
 +
 +        // now let's trigger index rebuild and check if we got the data back
 +        store.indexManager.buildIndexBlocking(store.indexManager.getIndexByName("data_output_id"));
 +
 +        rows = getIndexed(store, 10, buildExpression(dataOutputId, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1", "key2" }, rows.toArray(new String[rows.size()])));
 +
 +        // also let's try to build an index for column which has no data to make sure that doesn't fail
 +        store.indexManager.buildIndexBlocking(store.indexManager.getIndexByName("first_name"));
 +        store.indexManager.buildIndexBlocking(store.indexManager.getIndexByName("data_output_id"));
 +
 +        rows = getIndexed(store, 10, buildExpression(dataOutputId, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1", "key2" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(dataOutputId, Operator.LIKE_SUFFIX, UTF8Type.instance.decompose("el")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key2" }, rows.toArray(new String[rows.size()])));
 +    }
 +
 +    @Test
 +    public void testInvalidate() throws Exception
 +    {
 +        testInvalidate(false);
 +        cleanupData();
 +        testInvalidate(true);
 +    }
 +
 +    private void testInvalidate(boolean forceFlush) throws Exception
 +    {
 +        Map<String, Pair<String, Integer>> part1 = new HashMap<String, Pair<String, Integer>>()
 +        {{
 +                put("key0", Pair.create("Maxie", -1));
 +                put("key1", Pair.create("Chelsie", 33));
 +                put("key2", Pair.create((String) null, 43));
 +                put("key3", Pair.create("Shanna", 27));
 +                put("key4", Pair.create("Amiya", 36));
 +        }};
 +
 +        ColumnFamilyStore store = loadData(part1, forceFlush);
 +
 +        final ByteBuffer firstName = UTF8Type.instance.decompose("first_name");
 +        final ByteBuffer age = UTF8Type.instance.decompose("age");
 +
 +        Set<String> rows = getIndexed(store, 10, buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[]{ "key0", "key3", "key4" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(age, Operator.EQ, Int32Type.instance.decompose(33)));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[]{ "key1" }, rows.toArray(new String[rows.size()])));
 +
 +        store.indexManager.invalidateAllIndexesBlocking();
 +
 +        rows = getIndexed(store, 10, buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")));
 +        Assert.assertTrue(rows.toString(), rows.isEmpty());
 +
 +        rows = getIndexed(store, 10, buildExpression(age, Operator.EQ, Int32Type.instance.decompose(33)));
 +        Assert.assertTrue(rows.toString(), rows.isEmpty());
 +
 +
 +        Map<String, Pair<String, Integer>> part2 = new HashMap<String, Pair<String, Integer>>()
 +        {{
 +                put("key5", Pair.create("Americo", 20));
 +                put("key6", Pair.create("Fiona", 39));
 +                put("key7", Pair.create("Francis", 41));
 +                put("key8", Pair.create("Fred", 21));
 +                put("key9", Pair.create("Amely", 40));
 +                put("key14", Pair.create("Dino", 28));
 +        }};
 +
 +        loadData(part2, forceFlush);
 +
 +        rows = getIndexed(store, 10, buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[]{ "key6", "key7" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(age, Operator.EQ, Int32Type.instance.decompose(40)));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[]{ "key9" }, rows.toArray(new String[rows.size()])));
 +    }
 +
 +    @Test
 +    public void testTruncate()
 +    {
 +        Map<String, Pair<String, Integer>> part1 = new HashMap<String, Pair<String, Integer>>()
 +        {{
 +                put("key01", Pair.create("Ali", 33));
 +                put("key02", Pair.create("Jeremy", 41));
 +                put("key03", Pair.create("Elvera", 22));
 +                put("key04", Pair.create("Bailey", 45));
 +                put("key05", Pair.create("Emerson", 32));
 +                put("key06", Pair.create("Kadin", 38));
 +                put("key07", Pair.create("Maggie", 36));
 +                put("key08", Pair.create("Kailey", 36));
 +                put("key09", Pair.create("Armand", 21));
 +                put("key10", Pair.create("Arnold", 35));
 +        }};
 +
 +        Map<String, Pair<String, Integer>> part2 = new HashMap<String, Pair<String, Integer>>()
 +        {{
 +                put("key11", Pair.create("Ken", 38));
 +                put("key12", Pair.create("Penelope", 43));
 +                put("key13", Pair.create("Wyatt", 34));
 +                put("key14", Pair.create("Johnpaul", 34));
 +                put("key15", Pair.create("Trycia", 43));
 +                put("key16", Pair.create("Aida", 21));
 +                put("key17", Pair.create("Devon", 42));
 +        }};
 +
 +        Map<String, Pair<String, Integer>> part3 = new HashMap<String, Pair<String, Integer>>()
 +        {{
 +                put("key18", Pair.create("Christina", 20));
 +                put("key19", Pair.create("Rick", 19));
 +                put("key20", Pair.create("Fannie", 22));
 +                put("key21", Pair.create("Keegan", 29));
 +                put("key22", Pair.create("Ignatius", 36));
 +                put("key23", Pair.create("Ellis", 26));
 +                put("key24", Pair.create("Annamarie", 29));
 +                put("key25", Pair.create("Tianna", 31));
 +                put("key26", Pair.create("Dennis", 32));
 +        }};
 +
 +        ColumnFamilyStore store = loadData(part1, 1000, true);
 +
 +        loadData(part2, 2000, true);
 +        loadData(part3, 3000, true);
 +
 +        final ByteBuffer firstName = UTF8Type.instance.decompose("first_name");
 +
 +        Set<String> rows = getIndexed(store, 100, buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")));
 +        Assert.assertEquals(rows.toString(), 16, rows.size());
 +
 +        // make sure we don't prematurely delete anything
 +        store.indexManager.truncateAllIndexesBlocking(500);
 +
 +        rows = getIndexed(store, 100, buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")));
 +        Assert.assertEquals(rows.toString(), 16, rows.size());
 +
 +        store.indexManager.truncateAllIndexesBlocking(1500);
 +
 +        rows = getIndexed(store, 100, buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")));
 +        Assert.assertEquals(rows.toString(), 10, rows.size());
 +
 +        store.indexManager.truncateAllIndexesBlocking(2500);
 +
 +        rows = getIndexed(store, 100, buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")));
 +        Assert.assertEquals(rows.toString(), 6, rows.size());
 +
 +        store.indexManager.truncateAllIndexesBlocking(3500);
 +
 +        rows = getIndexed(store, 100, buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")));
 +        Assert.assertEquals(rows.toString(), 0, rows.size());
 +
 +        // add back in some data just to make sure it all still works
 +        Map<String, Pair<String, Integer>> part4 = new HashMap<String, Pair<String, Integer>>()
 +        {{
 +                put("key40", Pair.create("Tianna", 31));
 +                put("key41", Pair.create("Dennis", 32));
 +        }};
 +
 +        loadData(part4, 4000, true);
 +
 +        rows = getIndexed(store, 100, buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")));
 +        Assert.assertEquals(rows.toString(), 1, rows.size());
 +    }
 +
 +
 +    @Test
 +    public void testConcurrentMemtableReadsAndWrites() throws Exception
 +    {
 +        final ColumnFamilyStore store = Keyspace.open(KS_NAME).getColumnFamilyStore(CF_NAME);
 +
 +        ExecutorService scheduler = Executors.newFixedThreadPool(Runtime.getRuntime().availableProcessors());
 +
 +        final int writeCount = 10000;
 +        final AtomicInteger updates = new AtomicInteger(0);
 +
 +        for (int i = 0; i < writeCount; i++)
 +        {
 +            final String key = "key" + i;
 +            final String firstName = "first_name#" + i;
 +            final String lastName = "last_name#" + i;
 +
 +            scheduler.submit((Runnable) () -> {
 +                try
 +                {
 +                    newMutation(key, firstName, lastName, 26, System.currentTimeMillis()).apply();
 +                    Uninterruptibles.sleepUninterruptibly(5, TimeUnit.MILLISECONDS); // back up a bit to do more reads
 +                }
 +                finally
 +                {
 +                    updates.incrementAndGet();
 +                }
 +            });
 +        }
 +
 +        final ByteBuffer firstName = UTF8Type.instance.decompose("first_name");
 +        final ByteBuffer age = UTF8Type.instance.decompose("age");
 +
 +        int previousCount = 0;
 +
 +        do
 +        {
 +            // this loop figures out if number of search results monotonically increasing
 +            // to make sure that concurrent updates don't interfere with reads, uses first_name and age
 +            // indexes to test correctness of both Trie and SkipList ColumnIndex implementations.
 +
 +            Set<DecoratedKey> rows = getPaged(store, 100, buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                                                          buildExpression(age, Operator.EQ, Int32Type.instance.decompose(26)));
 +
 +            Assert.assertTrue(previousCount <= rows.size());
 +            previousCount = rows.size();
 +        }
 +        while (updates.get() < writeCount);
 +
 +        // to make sure that after all of the right are done we can read all "count" worth of rows
 +        Set<DecoratedKey> rows = getPaged(store, 100, buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                                                      buildExpression(age, Operator.EQ, Int32Type.instance.decompose(26)));
 +
 +        Assert.assertEquals(writeCount, rows.size());
 +    }
 +
 +    @Test
 +    public void testSameKeyInMemtableAndSSTables()
 +    {
 +        final ByteBuffer firstName = UTF8Type.instance.decompose("first_name");
 +        final ByteBuffer age = UTF8Type.instance.decompose("age");
 +
 +        Map<String, Pair<String, Integer>> data1 = new HashMap<String, Pair<String, Integer>>()
 +        {{
 +                put("key1", Pair.create("Pavel", 14));
 +                put("key2", Pair.create("Pavel", 26));
 +                put("key3", Pair.create("Pavel", 27));
 +                put("key4", Pair.create("Jason", 27));
 +        }};
 +
 +        ColumnFamilyStore store = loadData(data1, true);
 +
 +        Map<String, Pair<String, Integer>> data2 = new HashMap<String, Pair<String, Integer>>()
 +        {{
 +                put("key1", Pair.create("Pavel", 14));
 +                put("key2", Pair.create("Pavel", 27));
 +                put("key4", Pair.create("Jason", 28));
 +        }};
 +
 +        loadData(data2, true);
 +
 +        Map<String, Pair<String, Integer>> data3 = new HashMap<String, Pair<String, Integer>>()
 +        {{
 +                put("key1", Pair.create("Pavel", 15));
 +                put("key4", Pair.create("Jason", 29));
 +        }};
 +
 +        loadData(data3, false);
 +
 +        Set<String> rows = getIndexed(store, 100, buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1", "key2", "key3", "key4" }, rows.toArray(new String[rows.size()])));
 +
 +
 +        rows = getIndexed(store, 100, buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                                      buildExpression(age, Operator.EQ, Int32Type.instance.decompose(15)));
 +
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 100, buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                                      buildExpression(age, Operator.EQ, Int32Type.instance.decompose(29)));
 +
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key4" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 100, buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                                      buildExpression(age, Operator.EQ, Int32Type.instance.decompose(27)));
 +
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[]{"key2", "key3"}, rows.toArray(new String[rows.size()])));
 +    }
 +
 +    @Test
 +    public void testInsertingIncorrectValuesIntoAgeIndex()
 +    {
 +        ColumnFamilyStore store = Keyspace.open(KS_NAME).getColumnFamilyStore(CF_NAME);
 +
 +        final ByteBuffer firstName = UTF8Type.instance.decompose("first_name");
 +        final ByteBuffer age = UTF8Type.instance.decompose("age");
 +
 +        Mutation rm = new Mutation(KS_NAME, decoratedKey(AsciiType.instance.decompose("key1")));
 +        update(rm, new ArrayList<Cell>()
 +        {{
 +            add(buildCell(age, LongType.instance.decompose(26L), System.currentTimeMillis()));
 +            add(buildCell(firstName, AsciiType.instance.decompose("pavel"), System.currentTimeMillis()));
 +        }});
 +        rm.apply();
 +
 +        store.forceBlockingFlush();
 +
 +        Set<String> rows = getIndexed(store, 10, buildExpression(firstName, Operator.EQ, UTF8Type.instance.decompose("a")),
 +                                                 buildExpression(age, Operator.GTE, Int32Type.instance.decompose(26)));
 +
 +        // index is expected to have 0 results because age value was of wrong type
 +        Assert.assertEquals(0, rows.size());
 +    }
 +
 +
 +    @Test
 +    public void testUnicodeSupport()
 +    {
 +        testUnicodeSupport(false);
 +        cleanupData();
 +        testUnicodeSupport(true);
 +    }
 +
 +    private void testUnicodeSupport(boolean forceFlush)
 +    {
 +        ColumnFamilyStore store = Keyspace.open(KS_NAME).getColumnFamilyStore(CF_NAME);
 +
 +        final ByteBuffer comment = UTF8Type.instance.decompose("comment");
 +
 +        Mutation rm = new Mutation(KS_NAME, decoratedKey("key1"));
 +        update(rm, comment, UTF8Type.instance.decompose("ⓈⓅⒺⒸⒾⒶⓁ ⒞⒣⒜⒭⒮ and normal ones"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        rm = new Mutation(KS_NAME, decoratedKey("key2"));
 +        update(rm, comment, UTF8Type.instance.decompose("龍馭鬱"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        rm = new Mutation(KS_NAME, decoratedKey("key3"));
 +        update(rm, comment, UTF8Type.instance.decompose("インディアナ"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        rm = new Mutation(KS_NAME, decoratedKey("key4"));
 +        update(rm, comment, UTF8Type.instance.decompose("レストラン"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        rm = new Mutation(KS_NAME, decoratedKey("key5"));
 +        update(rm, comment, UTF8Type.instance.decompose("ベンジャミン ウエスト"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        if (forceFlush)
 +            store.forceBlockingFlush();
 +
 +        Set<String> rows;
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("ⓈⓅⒺⒸⒾ")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("normal")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("龍")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key2" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("鬱")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key2" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("馭鬱")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key2" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("龍馭鬱")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key2" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("ベンジャミン")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key5" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("レストラ")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key4" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("インディ")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key3" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("ベンジャミ")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key5" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_SUFFIX, UTF8Type.instance.decompose("ン")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key4", "key5" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_MATCHES, UTF8Type.instance.decompose("レストラン")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key4" }, rows.toArray(new String[rows.size()])));
 +    }
 +
 +    @Test
 +    public void testUnicodeSuffixModeNoSplits()
 +    {
 +        testUnicodeSuffixModeNoSplits(false);
 +        cleanupData();
 +        testUnicodeSuffixModeNoSplits(true);
 +    }
 +
 +    private void testUnicodeSuffixModeNoSplits(boolean forceFlush)
 +    {
 +        ColumnFamilyStore store = Keyspace.open(KS_NAME).getColumnFamilyStore(CF_NAME);
 +
 +        final ByteBuffer comment = UTF8Type.instance.decompose("comment_suffix_split");
 +
 +        Mutation rm = new Mutation(KS_NAME, decoratedKey("key1"));
 +        update(rm, comment, UTF8Type.instance.decompose("龍馭鬱"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        rm = new Mutation(KS_NAME, decoratedKey("key2"));
 +        update(rm, comment, UTF8Type.instance.decompose("インディアナ"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        rm = new Mutation(KS_NAME, decoratedKey("key3"));
 +        update(rm, comment, UTF8Type.instance.decompose("レストラン"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        rm = new Mutation(KS_NAME, decoratedKey("key4"));
 +        update(rm, comment, UTF8Type.instance.decompose("ベンジャミン ウエスト"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        if (forceFlush)
 +            store.forceBlockingFlush();
 +
 +        Set<String> rows;
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("龍")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("鬱")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("馭鬱")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("龍馭鬱")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("ベンジャミン")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key4" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("トラン")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key3" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("ディア")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key2" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("ジャミン")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key4" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("ン")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key2", "key3", "key4" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_SUFFIX, UTF8Type.instance.decompose("ン")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key3" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_MATCHES, UTF8Type.instance.decompose("ベンジャミン ウエスト")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key4" }, rows.toArray(new String[rows.size()])));
 +    }
 +
 +    @Test
 +    public void testThatTooBigValueIsRejected()
 +    {
 +        ColumnFamilyStore store = Keyspace.open(KS_NAME).getColumnFamilyStore(CF_NAME);
 +
 +        final ByteBuffer comment = UTF8Type.instance.decompose("comment_suffix_split");
 +
 +        for (int i = 0; i < 10; i++)
 +        {
 +            byte[] randomBytes = new byte[ThreadLocalRandom.current().nextInt(OnDiskIndexBuilder.MAX_TERM_SIZE, 5 * OnDiskIndexBuilder.MAX_TERM_SIZE)];
 +            ThreadLocalRandom.current().nextBytes(randomBytes);
 +
 +            final ByteBuffer bigValue = UTF8Type.instance.decompose(new String(randomBytes));
 +
 +            Mutation rm = new Mutation(KS_NAME, decoratedKey("key1"));
 +            update(rm, comment, bigValue, System.currentTimeMillis());
 +            rm.apply();
 +
 +            Set<String> rows;
 +
 +            rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_MATCHES, bigValue.duplicate()));
 +            Assert.assertEquals(0, rows.size());
 +
 +            store.forceBlockingFlush();
 +
 +            rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_MATCHES, bigValue.duplicate()));
 +            Assert.assertEquals(0, rows.size());
 +        }
 +    }
 +
 +    @Test
 +    public void testSearchTimeouts() throws Exception
 +    {
 +        final ByteBuffer firstName = UTF8Type.instance.decompose("first_name");
 +
 +        Map<String, Pair<String, Integer>> data1 = new HashMap<String, Pair<String, Integer>>()
 +        {{
 +                put("key1", Pair.create("Pavel", 14));
 +                put("key2", Pair.create("Pavel", 26));
 +                put("key3", Pair.create("Pavel", 27));
 +                put("key4", Pair.create("Jason", 27));
 +        }};
 +
 +        ColumnFamilyStore store = loadData(data1, true);
 +
 +        RowFilter filter = RowFilter.create();
 +        filter.add(store.metadata.getColumnDefinition(firstName), Operator.LIKE_CONTAINS, AsciiType.instance.fromString("a"));
 +
 +        ReadCommand command =
 +            PartitionRangeReadCommand.create(false,
 +                                             store.metadata,
 +                                             FBUtilities.nowInSeconds(),
 +                                             ColumnFilter.all(store.metadata),
 +                                             filter,
 +                                             DataLimits.NONE,
 +                                             DataRange.allData(store.metadata.partitioner));
 +        try
 +        {
 +            new QueryPlan(store, command, 0).execute(ReadExecutionController.empty());
 +            Assert.fail();
 +        }
 +        catch (TimeQuotaExceededException e)
 +        {
 +            // correct behavior
 +        }
 +        catch (Exception e)
 +        {
 +            Assert.fail();
 +            e.printStackTrace();
 +        }
 +
 +        // to make sure that query doesn't fail in normal conditions
 +
 +        try (ReadExecutionController controller = command.executionController())
 +        {
 +            Set<String> rows = getKeys(new QueryPlan(store, command, DatabaseDescriptor.getRangeRpcTimeout()).execute(controller));
 +            Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1", "key2", "key3", "key4" }, rows.toArray(new String[rows.size()])));
 +        }
 +    }
 +
 +    @Test
 +    public void testLowerCaseAnalyzer()
 +    {
 +        testLowerCaseAnalyzer(false);
 +        cleanupData();
 +        testLowerCaseAnalyzer(true);
 +    }
 +
 +    @Test
 +    public void testChinesePrefixSearch()
 +    {
 +        ColumnFamilyStore store = Keyspace.open(KS_NAME).getColumnFamilyStore(CF_NAME);
 +
 +        final ByteBuffer fullName = UTF8Type.instance.decompose("/output/full-name/");
 +
 +        Mutation rm = new Mutation(KS_NAME, decoratedKey("key1"));
 +        update(rm, fullName, UTF8Type.instance.decompose("美加 八田"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        rm = new Mutation(KS_NAME, decoratedKey("key2"));
 +        update(rm, fullName, UTF8Type.instance.decompose("仁美 瀧澤"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        rm = new Mutation(KS_NAME, decoratedKey("key3"));
 +        update(rm, fullName, UTF8Type.instance.decompose("晃宏 高須"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        rm = new Mutation(KS_NAME, decoratedKey("key4"));
 +        update(rm, fullName, UTF8Type.instance.decompose("弘孝 大竹"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        rm = new Mutation(KS_NAME, decoratedKey("key5"));
 +        update(rm, fullName, UTF8Type.instance.decompose("満枝 榎本"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        rm = new Mutation(KS_NAME, decoratedKey("key6"));
 +        update(rm, fullName, UTF8Type.instance.decompose("飛鳥 上原"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        rm = new Mutation(KS_NAME, decoratedKey("key7"));
 +        update(rm, fullName, UTF8Type.instance.decompose("大輝 鎌田"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        rm = new Mutation(KS_NAME, decoratedKey("key8"));
 +        update(rm, fullName, UTF8Type.instance.decompose("利久 寺地"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        store.forceBlockingFlush();
 +
 +
 +        Set<String> rows;
 +
 +        rows = getIndexed(store, 10, buildExpression(fullName, Operator.EQ, UTF8Type.instance.decompose("美加 八田")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(fullName, Operator.LIKE_PREFIX, UTF8Type.instance.decompose("美加")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(fullName, Operator.EQ, UTF8Type.instance.decompose("晃宏 高須")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key3" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(fullName, Operator.LIKE_PREFIX, UTF8Type.instance.decompose("大輝")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key7" }, rows.toArray(new String[rows.size()])));
 +    }
 +
 +    public void testLowerCaseAnalyzer(boolean forceFlush)
 +    {
 +        ColumnFamilyStore store = Keyspace.open(KS_NAME).getColumnFamilyStore(CF_NAME);
 +
 +        final ByteBuffer comment = UTF8Type.instance.decompose("address");
 +
 +        Mutation rm = new Mutation(KS_NAME, decoratedKey("key1"));
 +        update(rm, comment, UTF8Type.instance.decompose("577 Rogahn Valleys Apt. 178"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        rm = new Mutation(KS_NAME, decoratedKey("key2"));
 +        update(rm, comment, UTF8Type.instance.decompose("89809 Beverly Course Suite 089"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        rm = new Mutation(KS_NAME, decoratedKey("key3"));
 +        update(rm, comment, UTF8Type.instance.decompose("165 clydie oval apt. 399"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        if (forceFlush)
 +            store.forceBlockingFlush();
 +
 +        Set<String> rows;
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_PREFIX, UTF8Type.instance.decompose("577 Rogahn Valleys")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_PREFIX, UTF8Type.instance.decompose("577 ROgAhn VallEYs")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_PREFIX, UTF8Type.instance.decompose("577 rogahn valleys")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_PREFIX, UTF8Type.instance.decompose("577 rogahn")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_PREFIX, UTF8Type.instance.decompose("57")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_PREFIX, UTF8Type.instance.decompose("89809 Beverly Course")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key2" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_PREFIX, UTF8Type.instance.decompose("89809 BEVERly COURSE")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key2" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_PREFIX, UTF8Type.instance.decompose("89809 beverly course")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key2" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_PREFIX, UTF8Type.instance.decompose("89809 Beverly")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key2" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_PREFIX, UTF8Type.instance.decompose("8980")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key2" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_PREFIX, UTF8Type.instance.decompose("165 ClYdie OvAl APT. 399")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key3" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_PREFIX, UTF8Type.instance.decompose("165 Clydie Oval Apt. 399")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key3" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_PREFIX, UTF8Type.instance.decompose("165 clydie oval apt. 399")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key3" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_PREFIX, UTF8Type.instance.decompose("165 ClYdie OvA")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key3" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_PREFIX, UTF8Type.instance.decompose("165 ClYdi")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key3" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_PREFIX, UTF8Type.instance.decompose("165")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key3" }, rows.toArray(new String[rows.size()])));
 +    }
 +
 +    @Test
 +    public void testPrefixSSTableLookup()
 +    {
 +        // This test coverts particular case which interval lookup can return invalid results
 +        // when queried on the prefix e.g. "j".
 +        ColumnFamilyStore store = Keyspace.open(KS_NAME).getColumnFamilyStore(CF_NAME);
 +
 +        final ByteBuffer name = UTF8Type.instance.decompose("first_name_prefix");
 +
 +        Mutation rm;
 +
 +        rm = new Mutation(KS_NAME, decoratedKey("key1"));
 +        update(rm, name, UTF8Type.instance.decompose("Pavel"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        rm = new Mutation(KS_NAME, decoratedKey("key2"));
 +        update(rm, name, UTF8Type.instance.decompose("Jordan"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        rm = new Mutation(KS_NAME, decoratedKey("key3"));
 +        update(rm, name, UTF8Type.instance.decompose("Mikhail"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        rm = new Mutation(KS_NAME, decoratedKey("key4"));
 +        update(rm, name, UTF8Type.instance.decompose("Michael"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        rm = new Mutation(KS_NAME, decoratedKey("key5"));
 +        update(rm, name, UTF8Type.instance.decompose("Johnny"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        // first flush would make interval for name - 'johnny' -> 'pavel'
 +        store.forceBlockingFlush();
 +
 +        rm = new Mutation(KS_NAME, decoratedKey("key6"));
 +        update(rm, name, UTF8Type.instance.decompose("Jason"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        rm = new Mutation(KS_NAME, decoratedKey("key7"));
 +        update(rm, name, UTF8Type.instance.decompose("Vijay"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        rm = new Mutation(KS_NAME, decoratedKey("key8")); // this name is going to be tokenized
 +        update(rm, name, UTF8Type.instance.decompose("Jean-Claude"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        // this flush is going to produce range - 'jason' -> 'vijay'
 +        store.forceBlockingFlush();
 +
 +        // make sure that overlap of the prefixes is properly handled across sstables
 +        // since simple interval tree lookup is not going to cover it, prefix lookup actually required.
 +
 +        Set<String> rows;
 +
 +        rows = getIndexed(store, 10, buildExpression(name, Operator.LIKE_PREFIX, UTF8Type.instance.decompose("J")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key2", "key5", "key6", "key8"}, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(name, Operator.LIKE_PREFIX, UTF8Type.instance.decompose("j")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key2", "key5", "key6", "key8" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(name, Operator.LIKE_PREFIX, UTF8Type.instance.decompose("m")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key3", "key4" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(name, Operator.LIKE_PREFIX, UTF8Type.instance.decompose("v")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key7" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(name, Operator.LIKE_PREFIX, UTF8Type.instance.decompose("p")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(name, Operator.LIKE_PREFIX, UTF8Type.instance.decompose("j")),
 +                                     buildExpression(name, Operator.NEQ, UTF8Type.instance.decompose("joh")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key2", "key6", "key8" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(name, Operator.LIKE_MATCHES, UTF8Type.instance.decompose("pavel")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(name, Operator.EQ, UTF8Type.instance.decompose("Pave")));
 +        Assert.assertTrue(rows.isEmpty());
 +
 +        rows = getIndexed(store, 10, buildExpression(name, Operator.EQ, UTF8Type.instance.decompose("Pavel")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(name, Operator.LIKE_MATCHES, UTF8Type.instance.decompose("JeAn")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key8" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(name, Operator.LIKE_MATCHES, UTF8Type.instance.decompose("claUde")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key8" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(name, Operator.EQ, UTF8Type.instance.decompose("Jean")));
 +        Assert.assertTrue(rows.isEmpty());
 +
 +        rows = getIndexed(store, 10, buildExpression(name, Operator.EQ, UTF8Type.instance.decompose("Jean-Claude")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key8" }, rows.toArray(new String[rows.size()])));
 +    }
 +
 +    @Test
 +    public void testSettingIsLiteralOption()
 +    {
 +
 +        // special type which is UTF-8 but is only on the inside
 +        AbstractType<?> stringType = new AbstractType<String>(AbstractType.ComparisonType.CUSTOM)
 +        {
 +            public ByteBuffer fromString(String source) throws MarshalException
 +            {
 +                return UTF8Type.instance.fromString(source);
 +            }
 +
 +            public Term fromJSONObject(Object parsed) throws MarshalException
 +            {
 +                throw new UnsupportedOperationException();
 +            }
 +
 +            public TypeSerializer<String> getSerializer()
 +            {
 +                return UTF8Type.instance.getSerializer();
 +            }
 +
 +            public int compareCustom(ByteBuffer a, ByteBuffer b)
 +            {
 +                return UTF8Type.instance.compare(a, b);
 +            }
 +        };
 +
 +        // first let's check that we get 'false' for 'isLiteral' if we don't set the option with special comparator
 +        ColumnDefinition columnA = ColumnDefinition.regularDef(KS_NAME, CF_NAME, "special-A", stringType);
 +
 +        ColumnIndex indexA = new ColumnIndex(UTF8Type.instance, columnA, IndexMetadata.fromSchemaMetadata("special-index-A", IndexMetadata.Kind.CUSTOM, new HashMap<String, String>()
 +        {{
 +            put(IndexTarget.CUSTOM_INDEX_OPTION_NAME, SASIIndex.class.getName());
 +        }}));
 +
 +        Assert.assertEquals(true,  indexA.isIndexed());
 +        Assert.assertEquals(false, indexA.isLiteral());
 +
 +        // now let's double-check that we do get 'true' when we set it
 +        ColumnDefinition columnB = ColumnDefinition.regularDef(KS_NAME, CF_NAME, "special-B", stringType);
 +
 +        ColumnIndex indexB = new ColumnIndex(UTF8Type.instance, columnB, IndexMetadata.fromSchemaMetadata("special-index-B", IndexMetadata.Kind.CUSTOM, new HashMap<String, String>()
 +        {{
 +            put(IndexTarget.CUSTOM_INDEX_OPTION_NAME, SASIIndex.class.getName());
 +            put("is_literal", "true");
 +        }}));
 +
 +        Assert.assertEquals(true, indexB.isIndexed());
 +        Assert.assertEquals(true, indexB.isLiteral());
 +
 +        // and finally we should also get a 'true' if it's built-in UTF-8/ASCII comparator
 +        ColumnDefinition columnC = ColumnDefinition.regularDef(KS_NAME, CF_NAME, "special-C", UTF8Type.instance);
 +
 +        ColumnIndex indexC = new ColumnIndex(UTF8Type.instance, columnC, IndexMetadata.fromSchemaMetadata("special-index-C", IndexMetadata.Kind.CUSTOM, new HashMap<String, String>()
 +        {{
 +            put(IndexTarget.CUSTOM_INDEX_OPTION_NAME, SASIIndex.class.getName());
 +        }}));
 +
 +        Assert.assertEquals(true, indexC.isIndexed());
 +        Assert.assertEquals(true, indexC.isLiteral());
 +
 +        ColumnDefinition columnD = ColumnDefinition.regularDef(KS_NAME, CF_NAME, "special-D", AsciiType.instance);
 +
 +        ColumnIndex indexD = new ColumnIndex(UTF8Type.instance, columnD, IndexMetadata.fromSchemaMetadata("special-index-D", IndexMetadata.Kind.CUSTOM, new HashMap<String, String>()
 +        {{
 +            put(IndexTarget.CUSTOM_INDEX_OPTION_NAME, SASIIndex.class.getName());
 +        }}));
 +
 +        Assert.assertEquals(true, indexD.isIndexed());
 +        Assert.assertEquals(true, indexD.isLiteral());
 +
 +        // and option should supersedes the comparator type
 +        ColumnDefinition columnE = ColumnDefinition.regularDef(KS_NAME, CF_NAME, "special-E", UTF8Type.instance);
 +
 +        ColumnIndex indexE = new ColumnIndex(UTF8Type.instance, columnE, IndexMetadata.fromSchemaMetadata("special-index-E", IndexMetadata.Kind.CUSTOM, new HashMap<String, String>()
 +        {{
 +            put(IndexTarget.CUSTOM_INDEX_OPTION_NAME, SASIIndex.class.getName());
 +            put("is_literal", "false");
 +        }}));
 +
 +        Assert.assertEquals(true,  indexE.isIndexed());
 +        Assert.assertEquals(false, indexE.isLiteral());
 +
 +        // test frozen-collection
 +        ColumnDefinition columnF = ColumnDefinition.regularDef(KS_NAME,
 +                                                               CF_NAME,
 +                                                               "special-F",
 +                                                               ListType.getInstance(UTF8Type.instance, false));
 +
 +        ColumnIndex indexF = new ColumnIndex(UTF8Type.instance, columnF, IndexMetadata.fromSchemaMetadata("special-index-F", IndexMetadata.Kind.CUSTOM, new HashMap<String, String>()
 +        {{
 +            put(IndexTarget.CUSTOM_INDEX_OPTION_NAME, SASIIndex.class.getName());
 +        }}));
 +
 +        Assert.assertEquals(true,  indexF.isIndexed());
 +        Assert.assertEquals(false, indexF.isLiteral());
 +    }
 +
 +    @Test
 +    public void testClusteringIndexes() throws Exception
 +    {
 +        testClusteringIndexes(false);
 +        cleanupData();
 +        testClusteringIndexes(true);
 +    }
 +
 +    public void testClusteringIndexes(boolean forceFlush) throws Exception
 +    {
 +        ColumnFamilyStore store = Keyspace.open(KS_NAME).getColumnFamilyStore(CLUSTERING_CF_NAME_1);
 +
 +        executeCQL(CLUSTERING_CF_NAME_1, "INSERT INTO %s.%s (name, nick

<TRUNCATED>

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


[04/25] cassandra git commit: Allow dropping COMPACT STORAGE flag

Posted by if...@apache.org.
Allow dropping COMPACT STORAGE flag

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/6c29ee84
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/6c29ee84
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/6c29ee84

Branch: refs/heads/cassandra-3.11
Commit: 6c29ee84a2f62ccd05c328bbaa0c364eb1a7a821
Parents: b869744
Author: Alex Petrov <ol...@gmail.com>
Authored: Sat Sep 30 08:56:22 2017 +0200
Committer: Alex Petrov <ol...@gmail.com>
Committed: Mon Nov 6 15:44:51 2017 +0100

----------------------------------------------------------------------
 NEWS.txt                                        |  17 +
 bin/cqlsh.py                                    |   6 +-
 doc/native_protocol_v4.spec                     |   4 +
 ...dra-driver-internal-only-3.11.0-bb96859b.zip | Bin 0 -> 266661 bytes
 ...driver-internal-only-3.7.1.post0-19c1603.zip | Bin 252027 -> 0 bytes
 .../cassandra/auth/CassandraRoleManager.java    |   3 +-
 .../org/apache/cassandra/config/CFMetaData.java |  59 ++-
 src/java/org/apache/cassandra/cql3/Cql.g        |  27 +-
 .../apache/cassandra/cql3/QueryProcessor.java   |   2 +-
 .../cql3/statements/AlterTableStatement.java    |  42 +-
 .../statements/AuthenticationStatement.java     |   2 +-
 .../cql3/statements/AuthorizationStatement.java |   2 +-
 .../cql3/statements/BatchStatement.java         |   4 +-
 .../statements/CreateAggregateStatement.java    |   4 +-
 .../statements/CreateFunctionStatement.java     |   4 +-
 .../cql3/statements/CreateIndexStatement.java   |   9 +-
 .../cql3/statements/CreateTableStatement.java   |   2 +-
 .../cql3/statements/CreateViewStatement.java    |   2 +-
 .../cql3/statements/DropFunctionStatement.java  |   4 +-
 .../cql3/statements/ModificationStatement.java  |  11 +-
 .../cql3/statements/ParsedStatement.java        |   3 +-
 .../statements/SchemaAlteringStatement.java     |  32 +-
 .../cql3/statements/SelectStatement.java        |   8 +-
 .../cql3/statements/TruncateStatement.java      |   2 +-
 .../cassandra/cql3/statements/UseStatement.java |   2 +-
 src/java/org/apache/cassandra/db/view/View.java |  38 +-
 .../index/internal/keys/KeysSearcher.java       |   8 +-
 .../apache/cassandra/repair/RepairRunnable.java |   3 +-
 .../apache/cassandra/service/ClientState.java   |  16 +
 .../cassandra/thrift/ThriftValidation.java      |  10 +-
 .../transport/messages/StartupMessage.java      |   4 +
 .../org/apache/cassandra/cql3/ViewTest.java     |  15 +-
 .../cql3/validation/entities/UFTest.java        |  25 +
 .../cql3/validation/entities/UserTypesTest.java |  20 +
 .../cql3/validation/operations/AlterTest.java   |  15 +-
 .../DropCompactStorageThriftTest.java           | 525 +++++++++++++++++++
 36 files changed, 843 insertions(+), 87 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/NEWS.txt
----------------------------------------------------------------------
diff --git a/NEWS.txt b/NEWS.txt
index 60cf77c..621866b 100644
--- a/NEWS.txt
+++ b/NEWS.txt
@@ -37,6 +37,23 @@ Upgrading
    - Nothing specific to this release, but please see previous upgrading sections,
      especially if you are upgrading from 2.2.
 
+Compact Storage
+---------------
+    - Starting version 4.0, Thrift and COMPACT STORAGE is no longer supported.
+      'ALTER ... DROP COMPACT STORAGE' statement makes Compact Tables CQL-compatible,
+      exposing internal structure of Thrift/Compact Tables. You can find more details
+      on exposed internal structure under: 
+      http://cassandra.apache.org/doc/latest/cql/appendices.html#appendix-c-dropping-compact-storage
+
+      For uninterrupted cluster upgrades, drivers now support 'NO_COMPACT' startup option.
+      Supplying this flag will have same effect as 'DROP COMPACT STORAGE', but only for the
+      current connection.
+
+      In order to upgrade, clients supporting a non-compact schema view can be rolled out
+      gradually. When all the clients are updated 'ALTER ... DROP COMPACT STORAGE' can be
+      executed. After dropping compact storage, ’NO_COMPACT' option will have no effect
+      after that.
+
 Materialized Views
 -------------------
     - Cassandra will no longer allow dropping columns on tables with Materialized Views.

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/bin/cqlsh.py
----------------------------------------------------------------------
diff --git a/bin/cqlsh.py b/bin/cqlsh.py
index 8eb42a3..8d05d9d 100644
--- a/bin/cqlsh.py
+++ b/bin/cqlsh.py
@@ -205,6 +205,7 @@ parser.add_option("--browser", dest='browser', help="""The browser to use to dis
                                                     - one of the supported browsers in https://docs.python.org/2/library/webbrowser.html.
                                                     - browser path followed by %s, example: /usr/bin/google-chrome-stable %s""")
 parser.add_option('--ssl', action='store_true', help='Use SSL', default=False)
+parser.add_option('--no_compact', action='store_true', help='No Compact', default=False)
 parser.add_option("-u", "--username", help="Authenticate as user.")
 parser.add_option("-p", "--password", help="Authenticate using password.")
 parser.add_option('-k', '--keyspace', help='Authenticate to the given keyspace.')
@@ -702,6 +703,7 @@ class Shell(cmd.Cmd):
                  completekey=DEFAULT_COMPLETEKEY, browser=None, use_conn=None,
                  cqlver=DEFAULT_CQLVER, keyspace=None,
                  tracing_enabled=False, expand_enabled=False,
+                 no_compact=False,
                  display_nanotime_format=DEFAULT_NANOTIME_FORMAT,
                  display_timestamp_format=DEFAULT_TIMESTAMP_FORMAT,
                  display_date_format=DEFAULT_DATE_FORMAT,
@@ -732,7 +734,7 @@ class Shell(cmd.Cmd):
         else:
             self.conn = Cluster(contact_points=(self.hostname,), port=self.port, cql_version=cqlver,
                                 protocol_version=protocol_version,
-                                auth_provider=self.auth_provider,
+                                auth_provider=self.auth_provider, no_compact=no_compact,
                                 ssl_options=sslhandling.ssl_settings(hostname, CONFIG_FILE) if ssl else None,
                                 load_balancing_policy=WhiteListRoundRobinPolicy([self.hostname]),
                                 control_connection_timeout=connect_timeout,
@@ -2486,6 +2488,7 @@ def read_options(cmdlineargs, environment):
     optvalues.debug = False
     optvalues.file = None
     optvalues.ssl = False
+    optvalues.no_compact = False
     optvalues.encoding = option_with_default(configs.get, 'ui', 'encoding', UTF8)
 
     optvalues.tty = option_with_default(configs.getboolean, 'ui', 'tty', sys.stdin.isatty())
@@ -2643,6 +2646,7 @@ def main(options, hostname, port):
                       browser=options.browser,
                       cqlver=options.cqlversion,
                       keyspace=options.keyspace,
+                      no_compact=options.no_compact,
                       display_timestamp_format=options.time_format,
                       display_nanotime_format=options.nanotime_format,
                       display_date_format=options.date_format,

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/doc/native_protocol_v4.spec
----------------------------------------------------------------------
diff --git a/doc/native_protocol_v4.spec b/doc/native_protocol_v4.spec
index 44dac18..2188a33 100644
--- a/doc/native_protocol_v4.spec
+++ b/doc/native_protocol_v4.spec
@@ -271,6 +271,10 @@ Table of Contents
       different from the protocol version.
     - "COMPRESSION": the compression algorithm to use for frames (See section 5).
       This is optional; if not specified no compression will be used.
+    - "NO_COMPACT": whether or not connection has to be established in compatibility
+      mode. This mode will make all Thrift and Compact Tables to be exposed as if
+      they were CQL Tables. This is optional; if not specified, the option will
+      not be used.
 
 
 4.1.2. AUTH_RESPONSE

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/lib/cassandra-driver-internal-only-3.11.0-bb96859b.zip
----------------------------------------------------------------------
diff --git a/lib/cassandra-driver-internal-only-3.11.0-bb96859b.zip b/lib/cassandra-driver-internal-only-3.11.0-bb96859b.zip
new file mode 100644
index 0000000..d31abc3
Binary files /dev/null and b/lib/cassandra-driver-internal-only-3.11.0-bb96859b.zip differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/lib/cassandra-driver-internal-only-3.7.1.post0-19c1603.zip
----------------------------------------------------------------------
diff --git a/lib/cassandra-driver-internal-only-3.7.1.post0-19c1603.zip b/lib/cassandra-driver-internal-only-3.7.1.post0-19c1603.zip
deleted file mode 100644
index 900d64d..0000000
Binary files a/lib/cassandra-driver-internal-only-3.7.1.post0-19c1603.zip and /dev/null differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/src/java/org/apache/cassandra/auth/CassandraRoleManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/auth/CassandraRoleManager.java b/src/java/org/apache/cassandra/auth/CassandraRoleManager.java
index e5b98e4..c6716e5 100644
--- a/src/java/org/apache/cassandra/auth/CassandraRoleManager.java
+++ b/src/java/org/apache/cassandra/auth/CassandraRoleManager.java
@@ -39,6 +39,7 @@ import org.apache.cassandra.db.ConsistencyLevel;
 import org.apache.cassandra.db.marshal.UTF8Type;
 import org.apache.cassandra.exceptions.*;
 import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.service.ClientState;
 import org.apache.cassandra.service.QueryState;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.transport.messages.ResultMessage;
@@ -465,7 +466,7 @@ public class CassandraRoleManager implements IRoleManager
     {
         try
         {
-            return QueryProcessor.parseStatement(String.format(template, keyspace, table)).prepare().statement;
+            return QueryProcessor.parseStatement(String.format(template, keyspace, table)).prepare(ClientState.forInternalCalls()).statement;
         }
         catch (RequestValidationException e)
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/src/java/org/apache/cassandra/config/CFMetaData.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/CFMetaData.java b/src/java/org/apache/cassandra/config/CFMetaData.java
index fd1c9e5..c1b2171 100644
--- a/src/java/org/apache/cassandra/config/CFMetaData.java
+++ b/src/java/org/apache/cassandra/config/CFMetaData.java
@@ -79,10 +79,6 @@ public final class CFMetaData
     public final Pair<String, String> ksAndCFName;
     public final byte[] ksAndCFBytes;
 
-    private final ImmutableSet<Flag> flags;
-    private final boolean isDense;
-    private final boolean isCompound;
-    private final boolean isSuper;
     private final boolean isCounter;
     private final boolean isView;
     private final boolean isIndex;
@@ -94,6 +90,11 @@ public final class CFMetaData
     private final Serializers serializers;
 
     // non-final, for now
+    private volatile ImmutableSet<Flag> flags;
+    private volatile boolean isDense;
+    private volatile boolean isCompound;
+    private volatile boolean isSuper;
+
     public volatile TableParams params = TableParams.DEFAULT;
 
     private volatile Map<ByteBuffer, DroppedColumn> droppedColumns = new HashMap<>();
@@ -127,6 +128,9 @@ public final class CFMetaData
     private volatile ColumnDefinition superCfKeyColumn;
     private volatile ColumnDefinition superCfValueColumn;
 
+    /** Caches a non-compact version of the metadata for compact tables to be used with the NO_COMPACT protocol option. */
+    private volatile CFMetaData nonCompactCopy = null;
+
     public boolean isSuperColumnKeyColumn(ColumnDefinition cd)
     {
         return cd.name.equals(superCfKeyColumn.name);
@@ -330,6 +334,9 @@ public final class CFMetaData
     // are kept because they are often useful in a different format.
     private void rebuild()
     {
+        // A non-compact copy will be created lazily
+        this.nonCompactCopy = null;
+
         if (isCompactTable())
         {
             this.compactValueColumn = isSuper() ?
@@ -505,6 +512,38 @@ public final class CFMetaData
         return params(indexParams.build());
     }
 
+    /**
+     * Returns a cached non-compact version of this table. Cached version has to be invalidated
+     * every time the table is rebuilt.
+     */
+    public CFMetaData asNonCompact()
+    {
+        assert isCompactTable() : "Can't get non-compact version of a CQL table";
+
+        // Note that this is racy, but re-computing the non-compact copy a few times on first uses isn't a big deal so
+        // we don't bother.
+        if (nonCompactCopy == null)
+        {
+            nonCompactCopy = copyOpts(new CFMetaData(ksName,
+                                                     cfName,
+                                                     cfId,
+                                                     false,
+                                                     isCounter,
+                                                     false,
+                                                     true,
+                                                     isView,
+                                                     copy(partitionKeyColumns),
+                                                     copy(clusteringColumns),
+                                                     copy(partitionColumns),
+                                                     partitioner,
+                                                     superCfKeyColumn,
+                                                     superCfValueColumn),
+                                      this);
+        }
+
+        return nonCompactCopy;
+    }
+
     public CFMetaData copy()
     {
         return copy(cfId);
@@ -842,6 +881,12 @@ public final class CFMetaData
         superCfKeyColumn = cfm.superCfKeyColumn;
         superCfValueColumn = cfm.superCfValueColumn;
 
+        isDense = cfm.isDense;
+        isCompound = cfm.isCompound;
+        isSuper = cfm.isSuper;
+
+        flags = cfm.flags;
+
         rebuild();
 
         // compaction thresholds are checked by ThriftValidation. We shouldn't be doing
@@ -874,12 +919,6 @@ public final class CFMetaData
         if (!cfm.cfId.equals(cfId))
             throw new ConfigurationException(String.format("Column family ID mismatch (found %s; expected %s)",
                                                            cfm.cfId, cfId));
-
-        // Dense flag can get set, see CASSANDRA-12373 for details. We have to remove flag from both parts because
-        // there's no guaranteed call order in the call.
-
-        if (!cfm.flags.equals(flags) && (!isSuper() || !Sets.difference(cfm.flags, Sets.immutableEnumSet(Flag.DENSE)).equals(Sets.difference(flags, Sets.immutableEnumSet(Flag.DENSE)))))
-            throw new ConfigurationException("Types do not match: " + cfm.flags + " != " + flags);
     }
 
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/src/java/org/apache/cassandra/cql3/Cql.g
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/Cql.g b/src/java/org/apache/cassandra/cql3/Cql.g
index 3123877..0234327 100644
--- a/src/java/org/apache/cassandra/cql3/Cql.g
+++ b/src/java/org/apache/cassandra/cql3/Cql.g
@@ -781,7 +781,7 @@ createTriggerStatement returns [CreateTriggerStatement expr]
     @init {
         boolean ifNotExists = false;
     }
-    : K_CREATE K_TRIGGER (K_IF K_NOT K_EXISTS { ifNotExists = true; } )? (name=cident)
+    : K_CREATE K_TRIGGER (K_IF K_NOT K_EXISTS { ifNotExists = true; } )? (name=noncol_ident)
         K_ON cf=columnFamilyName K_USING cls=STRING_LITERAL
       { $expr = new CreateTriggerStatement(cf, name.toString(), $cls.text, ifNotExists); }
     ;
@@ -791,7 +791,7 @@ createTriggerStatement returns [CreateTriggerStatement expr]
  */
 dropTriggerStatement returns [DropTriggerStatement expr]
      @init { boolean ifExists = false; }
-    : K_DROP K_TRIGGER (K_IF K_EXISTS { ifExists = true; } )? (name=cident) K_ON cf=columnFamilyName
+    : K_DROP K_TRIGGER (K_IF K_EXISTS { ifExists = true; } )? (name=noncol_ident) K_ON cf=columnFamilyName
       { $expr = new DropTriggerStatement(cf, name.toString(), ifExists); }
     ;
 
@@ -816,20 +816,21 @@ alterTableStatement returns [AlterTableStatement expr]
     @init {
         AlterTableStatement.Type type = null;
         TableAttributes attrs = new TableAttributes();
-        Map<ColumnIdentifier.Raw, ColumnIdentifier.Raw> renames = new HashMap<ColumnIdentifier.Raw, ColumnIdentifier.Raw>();
+        Map<ColumnIdentifier.Raw, ColumnIdentifier> renames = new HashMap<ColumnIdentifier.Raw, ColumnIdentifier>();
         boolean isStatic = false;
         Long dropTimestamp = null;
     }
     : K_ALTER K_COLUMNFAMILY cf=columnFamilyName
-          ( K_ALTER id=cident K_TYPE v=comparatorType { type = AlterTableStatement.Type.ALTER; }
-          | K_ADD   id=cident v=comparatorType ({ isStatic=true; } K_STATIC)? { type = AlterTableStatement.Type.ADD; }
+          ( K_ALTER id=cident K_TYPE v=comparatorType { type = AlterTableStatement.Type.ALTER;  }
+          | K_ADD   aid=ident {id=new ColumnIdentifier.ColumnIdentifierValue(aid);} v=comparatorType ({ isStatic=true; } K_STATIC)? { type = AlterTableStatement.Type.ADD; }
           | K_DROP  id=cident                               { type = AlterTableStatement.Type.DROP; }
           | K_DROP  id=cident K_USING K_TIMESTAMP t=INTEGER { type = AlterTableStatement.Type.DROP;
                                                               dropTimestamp = Long.parseLong(Constants.Literal.integer($t.text).getText()); }
+          | K_DROP  K_COMPACT K_STORAGE                     { type = AlterTableStatement.Type.DROP_COMPACT_STORAGE; }
           | K_WITH  properties[attrs]                       { type = AlterTableStatement.Type.OPTS; }
           | K_RENAME                                        { type = AlterTableStatement.Type.RENAME; }
-               id1=cident K_TO toId1=cident { renames.put(id1, toId1); }
-               ( K_AND idn=cident K_TO toIdn=cident { renames.put(idn, toIdn); } )*
+               id1=cident K_TO toId1=ident { renames.put(id1, toId1); }
+               ( K_AND idn=cident K_TO toIdn=ident { renames.put(idn, toIdn); } )*
           )
     {
         $expr = new AlterTableStatement(cf, type, id, v, attrs, renames, isStatic, dropTimestamp);
@@ -1169,10 +1170,14 @@ userPassword[RoleOptions opts]
 // Column Identifiers.  These need to be treated differently from other
 // identifiers because the underlying comparator is not necessarily text. See
 // CASSANDRA-8178 for details.
+// Also, we need to support the internal of the super column map (for backward
+// compatibility) which is empty (we only want to allow this is queries, not for
+// creating table or other).
 cident returns [ColumnIdentifier.Raw id]
     : t=IDENT              { $id = new ColumnIdentifier.Literal($t.text, false); }
     | t=QUOTED_NAME        { $id = new ColumnIdentifier.Literal($t.text, true); }
     | k=unreserved_keyword { $id = new ColumnIdentifier.Literal(k, false); }
+    | EMPTY_QUOTED_NAME    { $id = new ColumnIdentifier.Literal("", false); }
     ;
 
 // Column identifiers where the comparator is known to be text
@@ -1309,7 +1314,9 @@ intValue returns [Term.Raw value]
     ;
 
 functionName returns [FunctionName s]
-    : (ks=keyspaceName '.')? f=allowedFunctionName   { $s = new FunctionName(ks, f); }
+     // antlr might try to recover and give a null for f. It will still error out in the end, but FunctionName
+     // wouldn't be happy with that so we should bypass this for now or we'll have a weird user-facing error
+    : (ks=keyspaceName '.')? f=allowedFunctionName   { $s = f == null ? null : new FunctionName(ks, f); }
     ;
 
 allowedFunctionName returns [String s]
@@ -1822,6 +1829,10 @@ STRING_LITERAL
       )
     ;
 
+EMPTY_QUOTED_NAME
+    : '\"' '\"'
+    ;
+
 QUOTED_NAME
     @init{ StringBuilder b = new StringBuilder(); }
     @after{ setText(b.toString()); }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/src/java/org/apache/cassandra/cql3/QueryProcessor.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/QueryProcessor.java b/src/java/org/apache/cassandra/cql3/QueryProcessor.java
index af94d3e..ddee6c7 100644
--- a/src/java/org/apache/cassandra/cql3/QueryProcessor.java
+++ b/src/java/org/apache/cassandra/cql3/QueryProcessor.java
@@ -517,7 +517,7 @@ public class QueryProcessor implements QueryHandler
             ((CFStatement)statement).prepareKeyspace(clientState);
 
         Tracing.trace("Preparing statement");
-        return statement.prepare();
+        return statement.prepare(clientState);
     }
 
     public static ParsedStatement parseStatement(String queryStr) throws SyntaxException

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java b/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java
index befdd25..a5fa12d 100644
--- a/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java
@@ -31,8 +31,6 @@ import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.db.marshal.CollectionType;
-import org.apache.cassandra.db.marshal.CounterColumnType;
-import org.apache.cassandra.db.marshal.ReversedType;
 import org.apache.cassandra.db.view.View;
 import org.apache.cassandra.exceptions.*;
 import org.apache.cassandra.schema.IndexMetadata;
@@ -49,14 +47,14 @@ public class AlterTableStatement extends SchemaAlteringStatement
 {
     public enum Type
     {
-        ADD, ALTER, DROP, OPTS, RENAME
+        ADD, ALTER, DROP, DROP_COMPACT_STORAGE, OPTS, RENAME
     }
 
     public final Type oType;
     public final CQL3Type.Raw validator;
     public final ColumnIdentifier.Raw rawColumnName;
     private final TableAttributes attrs;
-    private final Map<ColumnIdentifier.Raw, ColumnIdentifier.Raw> renames;
+    private final Map<ColumnIdentifier.Raw, ColumnIdentifier> renames;
     private final boolean isStatic; // Only for ALTER ADD
     private final Long deleteTimestamp;
 
@@ -65,7 +63,7 @@ public class AlterTableStatement extends SchemaAlteringStatement
                                ColumnIdentifier.Raw columnName,
                                CQL3Type.Raw validator,
                                TableAttributes attrs,
-                               Map<ColumnIdentifier.Raw, ColumnIdentifier.Raw> renames,
+                               Map<ColumnIdentifier.Raw, ColumnIdentifier> renames,
                                boolean isStatic,
                                Long deleteTimestamp)
     {
@@ -95,15 +93,15 @@ public class AlterTableStatement extends SchemaAlteringStatement
         if (meta.isView())
             throw new InvalidRequestException("Cannot use ALTER TABLE on Materialized View");
 
-        CFMetaData cfm = meta.copy();
+        CFMetaData cfm;
 
         CQL3Type validator = this.validator == null ? null : this.validator.prepare(keyspace());
         ColumnIdentifier columnName = null;
         ColumnDefinition def = null;
         if (rawColumnName != null)
         {
-            columnName = rawColumnName.prepare(cfm);
-            def = cfm.getColumnDefinition(columnName);
+            columnName = rawColumnName.prepare(meta);
+            def = meta.getColumnDefinition(columnName);
         }
 
         List<ViewDefinition> viewUpdates = null;
@@ -115,9 +113,11 @@ public class AlterTableStatement extends SchemaAlteringStatement
                 throw new InvalidRequestException("Altering of types is not allowed");
             case ADD:
                 assert columnName != null;
-                if (cfm.isDense())
+                if (meta.isDense())
                     throw new InvalidRequestException("Cannot add new column to a COMPACT STORAGE table");
 
+                cfm = meta.copy();
+
                 if (isStatic)
                 {
                     if (!cfm.isCompound())
@@ -190,11 +190,14 @@ public class AlterTableStatement extends SchemaAlteringStatement
 
             case DROP:
                 assert columnName != null;
-                if (!cfm.isCQLTable())
+                if (!meta.isCQLTable())
                     throw new InvalidRequestException("Cannot drop columns from a non-CQL3 table");
+
                 if (def == null)
                     throw new InvalidRequestException(String.format("Column %s was not found in table %s", columnName, columnFamily()));
 
+                cfm = meta.copy();
+
                 switch (def.kind)
                 {
                     case PARTITION_KEY:
@@ -238,11 +241,19 @@ public class AlterTableStatement extends SchemaAlteringStatement
                                                                     columnName.toString(),
                                                                     keyspace()));
                 break;
+            case DROP_COMPACT_STORAGE:
+                if (!meta.isCompactTable())
+                    throw new InvalidRequestException("Cannot DROP COMPACT STORAGE on table without COMPACT STORAGE");
+
+                cfm = meta.asNonCompact();
+                break;
             case OPTS:
                 if (attrs == null)
                     throw new InvalidRequestException("ALTER TABLE WITH invoked, but no parameters found");
                 attrs.validate();
 
+                cfm = meta.copy();
+
                 TableParams params = attrs.asAlteredTableParams(cfm.params);
 
                 if (!Iterables.isEmpty(views) && params.gcGraceSeconds == 0)
@@ -261,10 +272,13 @@ public class AlterTableStatement extends SchemaAlteringStatement
 
                 break;
             case RENAME:
-                for (Map.Entry<ColumnIdentifier.Raw, ColumnIdentifier.Raw> entry : renames.entrySet())
+                cfm = meta.copy();
+
+                for (Map.Entry<ColumnIdentifier.Raw, ColumnIdentifier> entry : renames.entrySet())
                 {
                     ColumnIdentifier from = entry.getKey().prepare(cfm);
-                    ColumnIdentifier to = entry.getValue().prepare(cfm);
+                    ColumnIdentifier to = entry.getValue();
+
                     cfm.renameColumn(from, to);
 
                     // If the view includes a renamed column, it must be renamed in the view table and the definition.
@@ -274,7 +288,7 @@ public class AlterTableStatement extends SchemaAlteringStatement
 
                         ViewDefinition viewCopy = view.copy();
                         ColumnIdentifier viewFrom = entry.getKey().prepare(viewCopy.metadata);
-                        ColumnIdentifier viewTo = entry.getValue().prepare(viewCopy.metadata);
+                        ColumnIdentifier viewTo = entry.getValue();
                         viewCopy.renameColumn(viewFrom, viewTo);
 
                         if (viewUpdates == null)
@@ -283,6 +297,8 @@ public class AlterTableStatement extends SchemaAlteringStatement
                     }
                 }
                 break;
+            default:
+                throw new InvalidRequestException("Can not alter table: unknown option type " + oType);
         }
 
         MigrationManager.announceColumnFamilyUpdate(cfm, viewUpdates, isLocalOnly);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/src/java/org/apache/cassandra/cql3/statements/AuthenticationStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/AuthenticationStatement.java b/src/java/org/apache/cassandra/cql3/statements/AuthenticationStatement.java
index 151e4f0..30ab6b0 100644
--- a/src/java/org/apache/cassandra/cql3/statements/AuthenticationStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/AuthenticationStatement.java
@@ -31,7 +31,7 @@ import org.apache.cassandra.transport.messages.ResultMessage;
 public abstract class AuthenticationStatement extends ParsedStatement implements CQLStatement
 {
     @Override
-    public Prepared prepare()
+    public Prepared prepare(ClientState clientState)
     {
         return new Prepared(this);
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/src/java/org/apache/cassandra/cql3/statements/AuthorizationStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/AuthorizationStatement.java b/src/java/org/apache/cassandra/cql3/statements/AuthorizationStatement.java
index 098e22c..fa2a993 100644
--- a/src/java/org/apache/cassandra/cql3/statements/AuthorizationStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/AuthorizationStatement.java
@@ -32,7 +32,7 @@ import org.apache.cassandra.transport.messages.ResultMessage;
 public abstract class AuthorizationStatement extends ParsedStatement implements CQLStatement
 {
     @Override
-    public Prepared prepare()
+    public Prepared prepare(ClientState clientState)
     {
         return new Prepared(this);
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java b/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
index cd9358c..1c3cfa6 100644
--- a/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
@@ -516,7 +516,7 @@ public class BatchStatement implements CQLStatement
                 statement.prepareKeyspace(state);
         }
 
-        public ParsedStatement.Prepared prepare() throws InvalidRequestException
+        public ParsedStatement.Prepared prepare(ClientState clientState) throws InvalidRequestException
         {
             VariableSpecifications boundNames = getBoundVariables();
 
@@ -537,7 +537,7 @@ public class BatchStatement implements CQLStatement
                     haveMultipleCFs = !firstKS.equals(parsed.keyspace()) || !firstCF.equals(parsed.columnFamily());
                 }
 
-                statements.add(parsed.prepare(boundNames));
+                statements.add(parsed.prepare(boundNames, clientState));
             }
 
             Attributes prepAttrs = attrs.prepare("[batch]", "[batch]");

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/src/java/org/apache/cassandra/cql3/statements/CreateAggregateStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/CreateAggregateStatement.java b/src/java/org/apache/cassandra/cql3/statements/CreateAggregateStatement.java
index ca0270f..9d91693 100644
--- a/src/java/org/apache/cassandra/cql3/statements/CreateAggregateStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/CreateAggregateStatement.java
@@ -78,7 +78,7 @@ public final class CreateAggregateStatement extends SchemaAlteringStatement
         this.ifNotExists = ifNotExists;
     }
 
-    public Prepared prepare()
+    public Prepared prepare(ClientState clientState)
     {
         argTypes = new ArrayList<>(argRawTypes.size());
         for (CQL3Type.Raw rawType : argRawTypes)
@@ -136,7 +136,7 @@ public final class CreateAggregateStatement extends SchemaAlteringStatement
                 throw new InvalidRequestException("INITCOND must not be empty for all types except TEXT, ASCII, BLOB");
         }
 
-        return super.prepare();
+        return super.prepare(clientState);
     }
 
     private AbstractType<?> prepareType(String typeName, CQL3Type.Raw rawType)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/src/java/org/apache/cassandra/cql3/statements/CreateFunctionStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/CreateFunctionStatement.java b/src/java/org/apache/cassandra/cql3/statements/CreateFunctionStatement.java
index c8d38f5..dfe522b 100644
--- a/src/java/org/apache/cassandra/cql3/statements/CreateFunctionStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/CreateFunctionStatement.java
@@ -76,7 +76,7 @@ public final class CreateFunctionStatement extends SchemaAlteringStatement
         this.ifNotExists = ifNotExists;
     }
 
-    public Prepared prepare() throws InvalidRequestException
+    public Prepared prepare(ClientState clientState) throws InvalidRequestException
     {
         if (new HashSet<>(argNames).size() != argNames.size())
             throw new InvalidRequestException(String.format("duplicate argument names for given function %s with argument names %s",
@@ -87,7 +87,7 @@ public final class CreateFunctionStatement extends SchemaAlteringStatement
             argTypes.add(prepareType("arguments", rawType));
 
         returnType = prepareType("return type", rawReturnType);
-        return super.prepare();
+        return super.prepare(clientState);
     }
 
     public void prepareKeyspace(ClientState state) throws InvalidRequestException

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/src/java/org/apache/cassandra/cql3/statements/CreateIndexStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/CreateIndexStatement.java b/src/java/org/apache/cassandra/cql3/statements/CreateIndexStatement.java
index c21441c..47d54fe 100644
--- a/src/java/org/apache/cassandra/cql3/statements/CreateIndexStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/CreateIndexStatement.java
@@ -103,8 +103,13 @@ public class CreateIndexStatement extends SchemaAlteringStatement
                 throw new InvalidRequestException("No column definition found for column " + target.column);
 
             // TODO: we could lift that limitation
-            if (cfm.isCompactTable() && cd.isPrimaryKeyColumn())
-                throw new InvalidRequestException("Secondary indexes are not supported on PRIMARY KEY columns in COMPACT STORAGE tables");
+            if (cfm.isCompactTable())
+            {
+                if (cd.isPrimaryKeyColumn())
+                    throw new InvalidRequestException("Secondary indexes are not supported on PRIMARY KEY columns in COMPACT STORAGE tables");
+                if (cfm.compactValueColumn().equals(cd))
+                    throw new InvalidRequestException("Secondary indexes are not supported on compact value column of COMPACT STORAGE tables");
+            }
 
             // It would be possible to support 2ndary index on static columns (but not without modifications of at least ExtendedFilter and
             // CompositesIndex) and maybe we should, but that means a query like:

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/src/java/org/apache/cassandra/cql3/statements/CreateTableStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/CreateTableStatement.java b/src/java/org/apache/cassandra/cql3/statements/CreateTableStatement.java
index ef950dc..9f14194 100644
--- a/src/java/org/apache/cassandra/cql3/statements/CreateTableStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/CreateTableStatement.java
@@ -191,7 +191,7 @@ public class CreateTableStatement extends SchemaAlteringStatement
         /**
          * Transform this raw statement into a CreateTableStatement.
          */
-        public ParsedStatement.Prepared prepare() throws RequestValidationException
+        public ParsedStatement.Prepared prepare(ClientState clientState) throws RequestValidationException
         {
             KeyspaceMetadata ksm = Schema.instance.getKSMetaData(keyspace());
             if (ksm == null)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/src/java/org/apache/cassandra/cql3/statements/CreateViewStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/CreateViewStatement.java b/src/java/org/apache/cassandra/cql3/statements/CreateViewStatement.java
index 778a3f4..cce954f 100644
--- a/src/java/org/apache/cassandra/cql3/statements/CreateViewStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/CreateViewStatement.java
@@ -214,7 +214,7 @@ public class CreateViewStatement extends SchemaAlteringStatement
         rawSelect.prepareKeyspace(state);
         rawSelect.setBoundVariables(getBoundVariables());
 
-        ParsedStatement.Prepared prepared = rawSelect.prepare(true);
+        ParsedStatement.Prepared prepared = rawSelect.prepare(true, queryState.getClientState());
         SelectStatement select = (SelectStatement) prepared.statement;
         StatementRestrictions restrictions = select.getRestrictions();
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/src/java/org/apache/cassandra/cql3/statements/DropFunctionStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/DropFunctionStatement.java b/src/java/org/apache/cassandra/cql3/statements/DropFunctionStatement.java
index 138691e..8845a82 100644
--- a/src/java/org/apache/cassandra/cql3/statements/DropFunctionStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/DropFunctionStatement.java
@@ -63,7 +63,7 @@ public final class DropFunctionStatement extends SchemaAlteringStatement
     }
 
     @Override
-    public Prepared prepare() throws InvalidRequestException
+    public Prepared prepare(ClientState clientState) throws InvalidRequestException
     {
         if (Schema.instance.getKSMetaData(functionName.keyspace) != null)
         {
@@ -82,7 +82,7 @@ public final class DropFunctionStatement extends SchemaAlteringStatement
             }
         }
 
-        return super.prepare();
+        return super.prepare(clientState);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java b/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
index 28fc90f..8ae4d64 100644
--- a/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
@@ -793,17 +793,16 @@ public abstract class ModificationStatement implements CQLStatement
             this.ifExists = ifExists;
         }
 
-        public ParsedStatement.Prepared prepare()
+        public ParsedStatement.Prepared prepare(ClientState clientState)
         {
             VariableSpecifications boundNames = getBoundVariables();
-            ModificationStatement statement = prepare(boundNames);
-            CFMetaData cfm = ThriftValidation.validateColumnFamily(keyspace(), columnFamily());
-            return new ParsedStatement.Prepared(statement, boundNames, boundNames.getPartitionKeyBindIndexes(cfm));
+            ModificationStatement statement = prepare(boundNames, clientState);
+            return new ParsedStatement.Prepared(statement, boundNames, boundNames.getPartitionKeyBindIndexes(statement.cfm));
         }
 
-        public ModificationStatement prepare(VariableSpecifications boundNames)
+        public ModificationStatement prepare(VariableSpecifications boundNames, ClientState clientState)
         {
-            CFMetaData metadata = ThriftValidation.validateColumnFamily(keyspace(), columnFamily());
+            CFMetaData metadata = ThriftValidation.validateColumnFamilyWithCompactMode(keyspace(), columnFamily(), clientState.isNoCompactMode());
 
             Attributes preparedAttributes = attrs.prepare(keyspace(), columnFamily());
             preparedAttributes.collectMarkerSpecification(boundNames);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/src/java/org/apache/cassandra/cql3/statements/ParsedStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/ParsedStatement.java b/src/java/org/apache/cassandra/cql3/statements/ParsedStatement.java
index 4c3f8a9..01a1b5e 100644
--- a/src/java/org/apache/cassandra/cql3/statements/ParsedStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/ParsedStatement.java
@@ -23,6 +23,7 @@ import java.util.List;
 import org.apache.cassandra.cql3.*;
 import org.apache.cassandra.cql3.functions.Function;
 import org.apache.cassandra.exceptions.RequestValidationException;
+import org.apache.cassandra.service.ClientState;
 
 public abstract class ParsedStatement
 {
@@ -44,7 +45,7 @@ public abstract class ParsedStatement
         this.variables = variables;
     }
 
-    public abstract Prepared prepare() throws RequestValidationException;
+    public abstract Prepared prepare(ClientState clientState) throws RequestValidationException;
 
     public static class Prepared
     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/src/java/org/apache/cassandra/cql3/statements/SchemaAlteringStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/SchemaAlteringStatement.java b/src/java/org/apache/cassandra/cql3/statements/SchemaAlteringStatement.java
index 62ba0ae..e7ecb14 100644
--- a/src/java/org/apache/cassandra/cql3/statements/SchemaAlteringStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/SchemaAlteringStatement.java
@@ -18,6 +18,8 @@
 package org.apache.cassandra.cql3.statements;
 
 import org.apache.cassandra.auth.AuthenticatedUser;
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.cql3.CFName;
 import org.apache.cassandra.cql3.CQLStatement;
 import org.apache.cassandra.cql3.QueryOptions;
@@ -25,9 +27,12 @@ import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.exceptions.RequestValidationException;
 import org.apache.cassandra.service.ClientState;
 import org.apache.cassandra.service.QueryState;
+import org.apache.cassandra.thrift.ThriftValidation;
 import org.apache.cassandra.transport.Event;
 import org.apache.cassandra.transport.messages.ResultMessage;
 
+import static org.apache.cassandra.cql3.statements.RequestValidations.invalidRequest;
+
 /**
  * Abstract class for statements that alter the schema.
  */
@@ -60,8 +65,33 @@ public abstract class SchemaAlteringStatement extends CFStatement implements CQL
     }
 
     @Override
-    public Prepared prepare()
+    public Prepared prepare(ClientState clientState)
     {
+        // We don't allow schema changes in no-compact mode on compact tables because it feels like unnecessary
+        // complication: applying the change on the non compact version of the table might be unsafe (the table is
+        // still compact in general), and applying it to the compact version in a no-compact connection feels
+        // confusing/unintuitive. If user want to alter the compact version, they can simply do so in a normal
+        // connection; if they want to alter the non-compact version, they should finish their transition and properly
+        // DROP COMPACT STORAGE on the table before doing so.
+        if (isColumnFamilyLevel && clientState.isNoCompactMode())
+        {
+            CFMetaData table = ThriftValidation.validateColumnFamily(keyspace(), columnFamily());
+            if (table.isCompactTable())
+            {
+                throw invalidRequest("Cannot alter schema of compact table %s.%s from a connection in NO-COMPACT mode",
+                                     table.ksName, table.cfName);
+            }
+            else if (table.isView())
+            {
+                CFMetaData baseTable = Schema.instance.getView(table.ksName, table.cfName).baseTableMetadata();
+                if (baseTable.isCompactTable())
+                    throw new InvalidRequestException(String.format("Cannot ALTER schema of view %s.%s on compact table %s from "
+                                                                    + "a connection in NO-COMPACT mode",
+                                                                    table.ksName, table.cfName,
+                                                                    baseTable.ksName, baseTable.cfName));
+            }
+        }
+
         return new Prepared(this);
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
index 2e090fa..1e867bc 100644
--- a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
@@ -887,14 +887,14 @@ public class SelectStatement implements CQLStatement
             this.limit = limit;
         }
 
-        public ParsedStatement.Prepared prepare() throws InvalidRequestException
+        public ParsedStatement.Prepared prepare(ClientState clientState) throws InvalidRequestException
         {
-            return prepare(false);
+            return prepare(false, clientState);
         }
 
-        public ParsedStatement.Prepared prepare(boolean forView) throws InvalidRequestException
+        public ParsedStatement.Prepared prepare(boolean forView, ClientState clientState) throws InvalidRequestException
         {
-            CFMetaData cfm = ThriftValidation.validateColumnFamily(keyspace(), columnFamily());
+            CFMetaData cfm = ThriftValidation.validateColumnFamilyWithCompactMode(keyspace(), columnFamily(), clientState.isNoCompactMode());
             VariableSpecifications boundNames = getBoundVariables();
 
             Selection selection = selectClause.isEmpty()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/src/java/org/apache/cassandra/cql3/statements/TruncateStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/TruncateStatement.java b/src/java/org/apache/cassandra/cql3/statements/TruncateStatement.java
index 66b3da0..b697910 100644
--- a/src/java/org/apache/cassandra/cql3/statements/TruncateStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/TruncateStatement.java
@@ -45,7 +45,7 @@ public class TruncateStatement extends CFStatement implements CQLStatement
         return 0;
     }
 
-    public Prepared prepare() throws InvalidRequestException
+    public Prepared prepare(ClientState clientState) throws InvalidRequestException
     {
         return new Prepared(this);
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/src/java/org/apache/cassandra/cql3/statements/UseStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/UseStatement.java b/src/java/org/apache/cassandra/cql3/statements/UseStatement.java
index fe3d518..e4685cc 100644
--- a/src/java/org/apache/cassandra/cql3/statements/UseStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/UseStatement.java
@@ -39,7 +39,7 @@ public class UseStatement extends ParsedStatement implements CQLStatement
         return 0;
     }
 
-    public Prepared prepare() throws InvalidRequestException
+    public Prepared prepare(ClientState clientState) throws InvalidRequestException
     {
         return new Prepared(this);
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/src/java/org/apache/cassandra/db/view/View.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/view/View.java b/src/java/org/apache/cassandra/db/view/View.java
index 58e2a84..9716dc4 100644
--- a/src/java/org/apache/cassandra/db/view/View.java
+++ b/src/java/org/apache/cassandra/db/view/View.java
@@ -17,32 +17,38 @@
  */
 package org.apache.cassandra.db.view;
 
-import java.nio.ByteBuffer;
-import java.util.*;
-import java.util.concurrent.TimeUnit;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.UUID;
 import java.util.stream.Collectors;
-
 import javax.annotation.Nullable;
 
 import com.google.common.collect.Iterables;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
-import org.apache.cassandra.cql3.*;
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.config.ViewDefinition;
+import org.apache.cassandra.cql3.ColumnIdentifier;
+import org.apache.cassandra.cql3.MultiColumnRelation;
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.Relation;
+import org.apache.cassandra.cql3.SingleColumnRelation;
+import org.apache.cassandra.cql3.Term;
 import org.apache.cassandra.cql3.statements.ParsedStatement;
 import org.apache.cassandra.cql3.statements.SelectStatement;
-import org.apache.cassandra.db.*;
-import org.apache.cassandra.config.*;
-import org.apache.cassandra.cql3.ColumnIdentifier;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.ReadQuery;
 import org.apache.cassandra.db.compaction.CompactionManager;
-import org.apache.cassandra.db.partitions.*;
-import org.apache.cassandra.db.rows.*;
+import org.apache.cassandra.db.rows.Row;
 import org.apache.cassandra.schema.KeyspaceMetadata;
 import org.apache.cassandra.service.ClientState;
-import org.apache.cassandra.service.pager.QueryPager;
-import org.apache.cassandra.transport.Server;
 import org.apache.cassandra.utils.FBUtilities;
-import org.apache.cassandra.utils.btree.BTreeSet;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 /**
  * A View copies data from a base table into a view table which can be queried independently from the
@@ -176,7 +182,7 @@ public class View
             ClientState state = ClientState.forInternalCalls();
             state.setKeyspace(baseCfs.keyspace.getName());
             rawSelect.prepareKeyspace(state);
-            ParsedStatement.Prepared prepared = rawSelect.prepare(true);
+            ParsedStatement.Prepared prepared = rawSelect.prepare(true, ClientState.forInternalCalls());
             select = (SelectStatement) prepared.statement;
         }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/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 c14c5a7..7cf4c51 100644
--- a/src/java/org/apache/cassandra/index/internal/keys/KeysSearcher.java
+++ b/src/java/org/apache/cassandra/index/internal/keys/KeysSearcher.java
@@ -197,7 +197,13 @@ public class KeysSearcher extends CassandraIndexSearcher
         }
         else
         {
-            assert iterator.metadata().isCompactTable();
+            if (!iterator.metadata().isCompactTable())
+            {
+                logger.warn("Non-composite index was used on the table '{}' during the query. Starting from Cassandra 4.0, only " +
+                            "composite indexes will be supported. If compact flags were dropped for this table, drop and re-create " +
+                            "the index.", iterator.metadata().cfName);
+            }
+
             Row data = iterator.staticRow();
             if (index.isStale(data, indexedValue, nowInSec))
             {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/src/java/org/apache/cassandra/repair/RepairRunnable.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/repair/RepairRunnable.java b/src/java/org/apache/cassandra/repair/RepairRunnable.java
index 213e5c5..77726d4 100644
--- a/src/java/org/apache/cassandra/repair/RepairRunnable.java
+++ b/src/java/org/apache/cassandra/repair/RepairRunnable.java
@@ -45,6 +45,7 @@ import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.repair.messages.RepairOption;
 import org.apache.cassandra.service.ActiveRepairService;
+import org.apache.cassandra.service.ClientState;
 import org.apache.cassandra.service.QueryState;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.tracing.TraceKeyspace;
@@ -385,7 +386,7 @@ public class RepairRunnable extends WrappedRunnable implements ProgressEventNoti
 
                 String format = "select event_id, source, activity from %s.%s where session_id = ? and event_id > ? and event_id < ?;";
                 String query = String.format(format, TraceKeyspace.NAME, TraceKeyspace.EVENTS);
-                SelectStatement statement = (SelectStatement) QueryProcessor.parseStatement(query).prepare().statement;
+                SelectStatement statement = (SelectStatement) QueryProcessor.parseStatement(query).prepare(ClientState.forInternalCalls()).statement;
 
                 ByteBuffer sessionIdBytes = ByteBufferUtil.bytes(sessionId);
                 InetAddress source = FBUtilities.getBroadcastAddress();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/src/java/org/apache/cassandra/service/ClientState.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/ClientState.java b/src/java/org/apache/cassandra/service/ClientState.java
index 32849bc..641d174 100644
--- a/src/java/org/apache/cassandra/service/ClientState.java
+++ b/src/java/org/apache/cassandra/service/ClientState.java
@@ -81,6 +81,12 @@ public class ClientState
     private volatile AuthenticatedUser user;
     private volatile String keyspace;
 
+    /**
+     * Force Compact Tables to be represented as CQL ones for the current client session (simulates
+     * ALTER .. DROP COMPACT STORAGE but only for this session)
+     */
+    private volatile boolean noCompactMode;
+
     private static final QueryHandler cqlQueryHandler;
     static
     {
@@ -253,6 +259,16 @@ public class ClientState
         keyspace = ks;
     }
 
+    public void setNoCompactMode()
+    {
+        this.noCompactMode = true;
+    }
+
+    public boolean isNoCompactMode()
+    {
+        return noCompactMode;
+    }
+
     /**
      * Attempts to login the given user.
      */

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/src/java/org/apache/cassandra/thrift/ThriftValidation.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/thrift/ThriftValidation.java b/src/java/org/apache/cassandra/thrift/ThriftValidation.java
index 99aed05..6ad791d 100644
--- a/src/java/org/apache/cassandra/thrift/ThriftValidation.java
+++ b/src/java/org/apache/cassandra/thrift/ThriftValidation.java
@@ -105,6 +105,11 @@ public class ThriftValidation
     // To be used when the operation should be authorized whether this is a counter CF or not
     public static CFMetaData validateColumnFamily(String keyspaceName, String cfName) throws org.apache.cassandra.exceptions.InvalidRequestException
     {
+        return validateColumnFamilyWithCompactMode(keyspaceName, cfName, false);
+    }
+
+    public static CFMetaData validateColumnFamilyWithCompactMode(String keyspaceName, String cfName, boolean noCompactMode) throws org.apache.cassandra.exceptions.InvalidRequestException
+    {
         validateKeyspace(keyspaceName);
         if (cfName.isEmpty())
             throw new org.apache.cassandra.exceptions.InvalidRequestException("non-empty table is required");
@@ -113,7 +118,10 @@ public class ThriftValidation
         if (metadata == null)
             throw new org.apache.cassandra.exceptions.InvalidRequestException("unconfigured table " + cfName);
 
-        return metadata;
+        if (metadata.isCompactTable() && noCompactMode)
+            return metadata.asNonCompact();
+        else
+            return metadata;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/src/java/org/apache/cassandra/transport/messages/StartupMessage.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/transport/messages/StartupMessage.java b/src/java/org/apache/cassandra/transport/messages/StartupMessage.java
index 04d8e62..774be6a 100644
--- a/src/java/org/apache/cassandra/transport/messages/StartupMessage.java
+++ b/src/java/org/apache/cassandra/transport/messages/StartupMessage.java
@@ -35,6 +35,7 @@ public class StartupMessage extends Message.Request
 {
     public static final String CQL_VERSION = "CQL_VERSION";
     public static final String COMPRESSION = "COMPRESSION";
+    public static final String NO_COMPACT = "NO_COMPACT";
 
     public static final Message.Codec<StartupMessage> codec = new Message.Codec<StartupMessage>()
     {
@@ -97,6 +98,9 @@ public class StartupMessage extends Message.Request
             }
         }
 
+        if (options.containsKey(NO_COMPACT) && Boolean.parseBoolean(options.get(NO_COMPACT)))
+            state.getClientState().setNoCompactMode();
+
         if (DatabaseDescriptor.getAuthenticator().requireAuthentication())
             return new AuthenticateMessage(DatabaseDescriptor.getAuthenticator().getClass().getName());
         else

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/test/unit/org/apache/cassandra/cql3/ViewTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/ViewTest.java b/test/unit/org/apache/cassandra/cql3/ViewTest.java
index 4a4fe1a..136ae1c 100644
--- a/test/unit/org/apache/cassandra/cql3/ViewTest.java
+++ b/test/unit/org/apache/cassandra/cql3/ViewTest.java
@@ -48,6 +48,8 @@ import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.SystemKeyspace;
 import org.apache.cassandra.db.compaction.CompactionManager;
 import org.apache.cassandra.db.marshal.AsciiType;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.exceptions.SyntaxException;
 import org.apache.cassandra.schema.KeyspaceParams;
 import org.apache.cassandra.utils.FBUtilities;
 
@@ -1237,7 +1239,6 @@ public class ViewTest extends CQLTester
         catch (Exception e)
         {
         }
-
     }
 
     @Test
@@ -1376,4 +1377,16 @@ public class ViewTest extends CQLTester
         assertRows(execute("SELECT k, toJson(listval) from mv"),
                    row(0, "[[\"a\", \"1\"], [\"b\", \"2\"], [\"c\", \"3\"]]"));
     }
+
+    @Test(expected = SyntaxException.class)
+    public void emptyViewNameTest() throws Throwable
+    {
+        execute("CREATE MATERIALIZED VIEW \"\" AS SELECT a, b FROM tbl WHERE b IS NOT NULL PRIMARY KEY (b, a)");
+    }
+
+     @Test(expected = SyntaxException.class)
+     public void emptyBaseTableNameTest() throws Throwable
+     {
+         execute("CREATE MATERIALIZED VIEW myview AS SELECT a, b FROM \"\" WHERE b IS NOT NULL PRIMARY KEY (b, a)");
+     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/test/unit/org/apache/cassandra/cql3/validation/entities/UFTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/UFTest.java b/test/unit/org/apache/cassandra/cql3/validation/entities/UFTest.java
index 8a743ea..6e6af19 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/entities/UFTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/UFTest.java
@@ -34,6 +34,7 @@ import org.apache.cassandra.cql3.functions.FunctionName;
 import org.apache.cassandra.cql3.functions.UDFunction;
 import org.apache.cassandra.db.marshal.CollectionType;
 import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.exceptions.SyntaxException;
 import org.apache.cassandra.schema.KeyspaceMetadata;
 import org.apache.cassandra.service.ClientState;
 import org.apache.cassandra.transport.Event;
@@ -968,4 +969,28 @@ public class UFTest extends CQLTester
         assertRows(execute("SELECT " + fNameICC + "(empty_int) FROM %s"), row(0));
         assertRows(execute("SELECT " + fNameICN + "(empty_int) FROM %s"), row(new Object[]{ null }));
     }
+
+    @Test(expected = SyntaxException.class)
+    public void testEmptyFunctionName() throws Throwable
+    {
+        execute("CREATE FUNCTION IF NOT EXISTS " + KEYSPACE + ".\"\" (arg int)\n" +
+                "  RETURNS NULL ON NULL INPUT\n" +
+                "  RETURNS int\n" +
+                "  LANGUAGE java\n" +
+                "  AS $$\n" +
+                "    return a;\n" +
+                "  $$");
+    }
+
+    @Test(expected = SyntaxException.class)
+    public void testEmptyArgName() throws Throwable
+    {
+        execute("CREATE FUNCTION IF NOT EXISTS " + KEYSPACE + ".myfn (\"\" int)\n" +
+                "  RETURNS NULL ON NULL INPUT\n" +
+                "  RETURNS int\n" +
+                "  LANGUAGE java\n" +
+                "  AS $$\n" +
+                "    return a;\n" +
+                "  $$");
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/test/unit/org/apache/cassandra/cql3/validation/entities/UserTypesTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/UserTypesTest.java b/test/unit/org/apache/cassandra/cql3/validation/entities/UserTypesTest.java
index dfc2e5e..68c0b8c 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/entities/UserTypesTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/UserTypesTest.java
@@ -24,6 +24,7 @@ import org.junit.Test;
 
 import org.apache.cassandra.cql3.CQLTester;
 import org.apache.cassandra.dht.ByteOrderedPartitioner;
+import org.apache.cassandra.exceptions.*;
 import org.apache.cassandra.service.StorageService;
 
 public class UserTypesTest extends CQLTester
@@ -713,6 +714,25 @@ public class UserTypesTest extends CQLTester
                        row(1, 1,set(userType(1), userType(1, 1), userType(1, 2), userType(2), userType(2, 1)), 2));
     }
 
+    @Test(expected = SyntaxException.class)
+    public void emptyTypeNameTest() throws Throwable
+    {
+        execute("CREATE TYPE \"\" (a int, b int)");
+    }
+
+    @Test(expected = SyntaxException.class)
+    public void emptyFieldNameTest() throws Throwable
+    {
+        execute("CREATE TYPE mytype (\"\" int, b int)");
+    }
+
+    @Test(expected = SyntaxException.class)
+    public void renameColumnToEmpty() throws Throwable
+    {
+        String typeName = createType("CREATE TYPE %s (a int, b int)");
+        execute(String.format("ALTER TYPE %s.%s RENAME b TO \"\"", keyspace(), typeName));
+    }
+
     private String typeWithKs(String type1)
     {
         return keyspace() + '.' + type1;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c29ee84/test/unit/org/apache/cassandra/cql3/validation/operations/AlterTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/AlterTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/AlterTest.java
index c48ffe5..b37462f 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/AlterTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/AlterTest.java
@@ -17,6 +17,9 @@
  */
 package org.apache.cassandra.cql3.validation.operations;
 
+import org.junit.Assert;
+import org.junit.Test;
+
 import org.apache.cassandra.cql3.CQLTester;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.Keyspace;
@@ -24,11 +27,6 @@ import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.exceptions.SyntaxException;
 import org.apache.cassandra.schema.SchemaKeyspace;
-import org.apache.cassandra.transport.Server;
-import org.apache.cassandra.utils.ByteBufferUtil;
-
-import org.junit.Assert;
-import org.junit.Test;
 
 import static java.lang.String.format;
 import static org.junit.Assert.assertEquals;
@@ -266,6 +264,13 @@ public class AlterTest extends CQLTester
         execute("alter table %s add v int");
     }
 
+    @Test(expected = SyntaxException.class)
+    public void renameToEmptyTest() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int, c1 int, v int, PRIMARY KEY (k, c1))");
+        execute("ALTER TABLE %s RENAME c1 TO \"\"");
+    }
+
     @Test
     // tests CASSANDRA-9565
     public void testDoubleWith() throws Throwable


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


[08/25] cassandra git commit: Merge branch 'cassandra-3.0' into cassandra-3.11

Posted by if...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d429cd0/test/unit/org/apache/cassandra/index/sasi/SASIIndexTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/index/sasi/SASIIndexTest.java
index 03d89e1,0000000..1b2f97d
mode 100644,000000..100644
--- a/test/unit/org/apache/cassandra/index/sasi/SASIIndexTest.java
+++ b/test/unit/org/apache/cassandra/index/sasi/SASIIndexTest.java
@@@ -1,2568 -1,0 +1,2569 @@@
 +/*
 + * 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.index.sasi;
 +
 +import java.io.FileWriter;
 +import java.io.Writer;
 +import java.nio.ByteBuffer;
 +import java.nio.file.FileSystems;
 +import java.nio.file.Files;
 +import java.nio.file.Path;
 +import java.nio.file.attribute.BasicFileAttributes;
 +import java.util.*;
 +import java.util.concurrent.ExecutorService;
 +import java.util.concurrent.Executors;
 +import java.util.concurrent.ThreadLocalRandom;
 +import java.util.concurrent.TimeUnit;
 +import java.util.concurrent.atomic.AtomicInteger;
 +
 +import org.apache.cassandra.SchemaLoader;
 +import org.apache.cassandra.config.CFMetaData;
 +import org.apache.cassandra.config.ColumnDefinition;
 +import org.apache.cassandra.index.Index;
 +import org.apache.cassandra.config.DatabaseDescriptor;
 +import org.apache.cassandra.cql3.*;
 +import org.apache.cassandra.cql3.Term;
 +import org.apache.cassandra.cql3.statements.IndexTarget;
 +import org.apache.cassandra.cql3.statements.SelectStatement;
 +import org.apache.cassandra.db.*;
 +import org.apache.cassandra.db.filter.ColumnFilter;
 +import org.apache.cassandra.db.filter.DataLimits;
 +import org.apache.cassandra.db.filter.RowFilter;
 +import org.apache.cassandra.db.lifecycle.SSTableSet;
 +import org.apache.cassandra.db.marshal.*;
 +import org.apache.cassandra.db.partitions.PartitionUpdate;
 +import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator;
 +import org.apache.cassandra.db.rows.*;
 +import org.apache.cassandra.dht.IPartitioner;
 +import org.apache.cassandra.dht.Murmur3Partitioner;
 +import org.apache.cassandra.dht.Range;
 +import org.apache.cassandra.exceptions.ConfigurationException;
 +import org.apache.cassandra.exceptions.InvalidRequestException;
 +import org.apache.cassandra.index.sasi.conf.ColumnIndex;
 +import org.apache.cassandra.index.sasi.disk.OnDiskIndexBuilder;
 +import org.apache.cassandra.index.sasi.exceptions.TimeQuotaExceededException;
 +import org.apache.cassandra.index.sasi.memory.IndexMemtable;
 +import org.apache.cassandra.index.sasi.plan.QueryController;
 +import org.apache.cassandra.index.sasi.plan.QueryPlan;
 +import org.apache.cassandra.io.sstable.SSTable;
 +import org.apache.cassandra.schema.IndexMetadata;
 +import org.apache.cassandra.schema.KeyspaceMetadata;
 +import org.apache.cassandra.schema.KeyspaceParams;
 +import org.apache.cassandra.schema.Tables;
 +import org.apache.cassandra.serializers.MarshalException;
 +import org.apache.cassandra.serializers.TypeSerializer;
++import org.apache.cassandra.service.ClientState;
 +import org.apache.cassandra.service.MigrationManager;
 +import org.apache.cassandra.service.QueryState;
 +import org.apache.cassandra.thrift.CqlRow;
 +import org.apache.cassandra.transport.messages.ResultMessage;
 +import org.apache.cassandra.utils.ByteBufferUtil;
 +import org.apache.cassandra.utils.FBUtilities;
 +import org.apache.cassandra.utils.Pair;
 +
 +import com.google.common.collect.Lists;
 +import com.google.common.util.concurrent.Uninterruptibles;
 +
 +import junit.framework.Assert;
 +
 +import org.junit.*;
 +
 +public class SASIIndexTest
 +{
 +    private static final IPartitioner PARTITIONER;
 +
 +    static {
 +        System.setProperty("cassandra.config", "cassandra-murmur.yaml");
 +        PARTITIONER = Murmur3Partitioner.instance;
 +    }
 +
 +    private static final String KS_NAME = "sasi";
 +    private static final String CF_NAME = "test_cf";
 +    private static final String CLUSTERING_CF_NAME_1 = "clustering_test_cf_1";
 +    private static final String CLUSTERING_CF_NAME_2 = "clustering_test_cf_2";
 +    private static final String STATIC_CF_NAME = "static_sasi_test_cf";
 +    private static final String FTS_CF_NAME = "full_text_search_sasi_test_cf";
 +
 +    @BeforeClass
 +    public static void loadSchema() throws ConfigurationException
 +    {
 +        SchemaLoader.loadSchema();
 +        MigrationManager.announceNewKeyspace(KeyspaceMetadata.create(KS_NAME,
 +                                                                     KeyspaceParams.simpleTransient(1),
 +                                                                     Tables.of(SchemaLoader.sasiCFMD(KS_NAME, CF_NAME),
 +                                                                               SchemaLoader.clusteringSASICFMD(KS_NAME, CLUSTERING_CF_NAME_1),
 +                                                                               SchemaLoader.clusteringSASICFMD(KS_NAME, CLUSTERING_CF_NAME_2, "location"),
 +                                                                               SchemaLoader.staticSASICFMD(KS_NAME, STATIC_CF_NAME),
 +                                                                               SchemaLoader.fullTextSearchSASICFMD(KS_NAME, FTS_CF_NAME))));
 +    }
 +
 +    @Before
 +    public void cleanUp()
 +    {
 +        Keyspace.open(KS_NAME).getColumnFamilyStore(CF_NAME).truncateBlocking();
 +    }
 +
 +    @Test
 +    public void testSingleExpressionQueries() throws Exception
 +    {
 +        testSingleExpressionQueries(false);
 +        cleanupData();
 +        testSingleExpressionQueries(true);
 +    }
 +
 +    private void testSingleExpressionQueries(boolean forceFlush) throws Exception
 +    {
 +        Map<String, Pair<String, Integer>> data = new HashMap<String, Pair<String, Integer>>()
 +        {{
 +            put("key1", Pair.create("Pavel", 14));
 +            put("key2", Pair.create("Pavel", 26));
 +            put("key3", Pair.create("Pavel", 27));
 +            put("key4", Pair.create("Jason", 27));
 +        }};
 +
 +        ColumnFamilyStore store = loadData(data, forceFlush);
 +
 +        final ByteBuffer firstName = UTF8Type.instance.decompose("first_name");
 +        final ByteBuffer age = UTF8Type.instance.decompose("age");
 +
 +        Set<String> rows;
 +
 +        rows = getIndexed(store, 10, buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1", "key2", "key3", "key4" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("av")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1", "key2", "key3" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("as")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key4" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("aw")));
 +        Assert.assertEquals(rows.toString(), 0, rows.size());
 +
 +        rows = getIndexed(store, 10, buildExpression(firstName, Operator.LIKE_SUFFIX, UTF8Type.instance.decompose("avel")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1", "key2", "key3" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(firstName, Operator.LIKE_SUFFIX, UTF8Type.instance.decompose("n")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key4" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(age, Operator.EQ, Int32Type.instance.decompose(27)));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[]{"key3", "key4"}, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(age, Operator.EQ, Int32Type.instance.decompose(26)));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key2" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(age, Operator.EQ, Int32Type.instance.decompose(13)));
 +        Assert.assertEquals(rows.toString(), 0, rows.size());
 +    }
 +
 +    @Test
 +    public void testEmptyTokenizedResults() throws Exception
 +    {
 +        testEmptyTokenizedResults(false);
 +        cleanupData();
 +        testEmptyTokenizedResults(true);
 +    }
 +
 +    private void testEmptyTokenizedResults(boolean forceFlush) throws Exception
 +    {
 +        Map<String, Pair<String, Integer>> data = new HashMap<String, Pair<String, Integer>>()
 +        {{
 +                put("key1", Pair.create("  ", 14));
 +        }};
 +
 +        ColumnFamilyStore store = loadData(data, forceFlush);
 +
 +        Set<String> rows= getIndexed(store, 10, buildExpression(UTF8Type.instance.decompose("first_name"), Operator.LIKE_MATCHES, UTF8Type.instance.decompose("doesntmatter")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[]{}, rows.toArray(new String[rows.size()])));
 +    }
 +
 +    @Test
 +    public void testMultiExpressionQueries() throws Exception
 +    {
 +        testMultiExpressionQueries(false);
 +        cleanupData();
 +        testMultiExpressionQueries(true);
 +    }
 +
 +    public void testMultiExpressionQueries(boolean forceFlush) throws Exception
 +    {
 +        Map<String, Pair<String, Integer>> data = new HashMap<String, Pair<String, Integer>>()
 +        {{
 +                put("key1", Pair.create("Pavel", 14));
 +                put("key2", Pair.create("Pavel", 26));
 +                put("key3", Pair.create("Pavel", 27));
 +                put("key4", Pair.create("Jason", 27));
 +        }};
 +
 +        ColumnFamilyStore store = loadData(data, forceFlush);
 +
 +        final ByteBuffer firstName = UTF8Type.instance.decompose("first_name");
 +        final ByteBuffer age = UTF8Type.instance.decompose("age");
 +
 +        Set<String> rows;
 +        rows = getIndexed(store, 10,
 +                          buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                          buildExpression(age, Operator.GT, Int32Type.instance.decompose(14)));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key2", "key3", "key4" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10,
 +                          buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                          buildExpression(age, Operator.LT, Int32Type.instance.decompose(27)));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[]{"key1", "key2"}, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10,
 +                         buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                         buildExpression(age, Operator.GT, Int32Type.instance.decompose(14)),
 +                         buildExpression(age, Operator.LT, Int32Type.instance.decompose(27)));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key2" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10,
 +                         buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                         buildExpression(age, Operator.GT, Int32Type.instance.decompose(12)));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[]{ "key1", "key2", "key3", "key4" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10,
 +                         buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                         buildExpression(age, Operator.GTE, Int32Type.instance.decompose(13)));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1", "key2", "key3", "key4" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10,
 +                         buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                         buildExpression(age, Operator.GTE, Int32Type.instance.decompose(16)));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key2", "key3", "key4" }, rows.toArray(new String[rows.size()])));
 +
 +
 +        rows = getIndexed(store, 10,
 +                         buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                         buildExpression(age, Operator.LT, Int32Type.instance.decompose(30)));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1", "key2", "key3", "key4" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10,
 +                         buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                         buildExpression(age, Operator.LTE, Int32Type.instance.decompose(29)));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1", "key2", "key3", "key4" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10,
 +                         buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                         buildExpression(age, Operator.LTE, Int32Type.instance.decompose(25)));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[]{ "key1" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(firstName, Operator.LIKE_SUFFIX, UTF8Type.instance.decompose("avel")),
 +                                     buildExpression(age, Operator.LTE, Int32Type.instance.decompose(25)));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(firstName, Operator.LIKE_SUFFIX, UTF8Type.instance.decompose("n")),
 +                                     buildExpression(age, Operator.LTE, Int32Type.instance.decompose(25)));
 +        Assert.assertTrue(rows.isEmpty());
 +
 +    }
 +
 +    @Test
 +    public void testCrossSSTableQueries() throws Exception
 +    {
 +        testCrossSSTableQueries(false);
 +        cleanupData();
 +        testCrossSSTableQueries(true);
 +
 +    }
 +
 +    private void testCrossSSTableQueries(boolean forceFlush) throws Exception
 +    {
 +        Map<String, Pair<String, Integer>> part1 = new HashMap<String, Pair<String, Integer>>()
 +        {{
 +                put("key0", Pair.create("Maxie", 43));
 +                put("key1", Pair.create("Chelsie", 33));
 +                put("key2", Pair.create("Josephine", 43));
 +                put("key3", Pair.create("Shanna", 27));
 +                put("key4", Pair.create("Amiya", 36));
 +            }};
 +
 +        loadData(part1, forceFlush); // first sstable
 +
 +        Map<String, Pair<String, Integer>> part2 = new HashMap<String, Pair<String, Integer>>()
 +        {{
 +                put("key5", Pair.create("Americo", 20));
 +                put("key6", Pair.create("Fiona", 39));
 +                put("key7", Pair.create("Francis", 41));
 +                put("key8", Pair.create("Charley", 21));
 +                put("key9", Pair.create("Amely", 40));
 +            }};
 +
 +        loadData(part2, forceFlush);
 +
 +        Map<String, Pair<String, Integer>> part3 = new HashMap<String, Pair<String, Integer>>()
 +        {{
 +                put("key10", Pair.create("Eddie", 42));
 +                put("key11", Pair.create("Oswaldo", 35));
 +                put("key12", Pair.create("Susana", 35));
 +                put("key13", Pair.create("Alivia", 42));
 +                put("key14", Pair.create("Demario", 28));
 +            }};
 +
 +        ColumnFamilyStore store = loadData(part3, forceFlush);
 +
 +        final ByteBuffer firstName = UTF8Type.instance.decompose("first_name");
 +        final ByteBuffer age = UTF8Type.instance.decompose("age");
 +
 +        Set<String> rows;
 +        rows = getIndexed(store, 10, buildExpression(firstName, Operator.EQ, UTF8Type.instance.decompose("Fiona")),
 +                                     buildExpression(age, Operator.LT, Int32Type.instance.decompose(40)));
 +
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key6" }, rows.toArray(new String[rows.size()])));
 +
 +
 +        rows = getIndexed(store, 10,
 +                          buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")));
 +
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key0", "key11", "key12", "key13", "key14",
 +                                                                        "key3", "key4", "key6", "key7", "key8" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 5,
 +                          buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")));
 +
 +        Assert.assertEquals(rows.toString(), 5, rows.size());
 +
 +        rows = getIndexed(store, 10,
 +                          buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                          buildExpression(age, Operator.GTE, Int32Type.instance.decompose(35)));
 +
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key0", "key11", "key12", "key13", "key4", "key6", "key7" },
 +                                                         rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10,
 +                          buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                          buildExpression(age, Operator.LT, Int32Type.instance.decompose(32)));
 +
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key14", "key3", "key8" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10,
 +                          buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                          buildExpression(age, Operator.GT, Int32Type.instance.decompose(27)),
 +                          buildExpression(age, Operator.LT, Int32Type.instance.decompose(32)));
 +
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key14" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10,
 +                          buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                          buildExpression(age, Operator.GT, Int32Type.instance.decompose(10)));
 +
 +        Assert.assertEquals(rows.toString(), 10, rows.size());
 +
 +        rows = getIndexed(store, 10,
 +                          buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                          buildExpression(age, Operator.LTE, Int32Type.instance.decompose(50)));
 +
 +        Assert.assertEquals(rows.toString(), 10, rows.size());
 +
 +        rows = getIndexed(store, 10,
 +                          buildExpression(firstName, Operator.LIKE_SUFFIX, UTF8Type.instance.decompose("ie")),
 +                          buildExpression(age, Operator.LT, Int32Type.instance.decompose(43)));
 +
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1", "key10" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10,
 +                          buildExpression(firstName, Operator.LIKE_SUFFIX, UTF8Type.instance.decompose("a")));
 +
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key12", "key13", "key3", "key4", "key6" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10,
 +                          buildExpression(firstName, Operator.LIKE_SUFFIX, UTF8Type.instance.decompose("a")),
 +                          buildExpression(age, Operator.LT, Int32Type.instance.decompose(33)));
 +
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key3" }, rows.toArray(new String[rows.size()])));
 +    }
 +
 +    @Test
 +    public void testQueriesThatShouldBeTokenized() throws Exception
 +    {
 +        testQueriesThatShouldBeTokenized(false);
 +        cleanupData();
 +        testQueriesThatShouldBeTokenized(true);
 +    }
 +
 +    private void testQueriesThatShouldBeTokenized(boolean forceFlush) throws Exception
 +    {
 +        Map<String, Pair<String, Integer>> part1 = new HashMap<String, Pair<String, Integer>>()
 +        {{
 +                put("key0", Pair.create("If you can dream it, you can do it.", 43));
 +                put("key1", Pair.create("What you get by achieving your goals is not " +
 +                        "as important as what you become by achieving your goals, do it.", 33));
 +                put("key2", Pair.create("Keep your face always toward the sunshine " +
 +                        "- and shadows will fall behind you.", 43));
 +                put("key3", Pair.create("We can't help everyone, but everyone can " +
 +                        "help someone.", 27));
 +            }};
 +
 +        ColumnFamilyStore store = loadData(part1, forceFlush);
 +
 +        final ByteBuffer firstName = UTF8Type.instance.decompose("first_name");
 +        final ByteBuffer age = UTF8Type.instance.decompose("age");
 +
 +        Set<String> rows = getIndexed(store, 10,
 +                buildExpression(firstName, Operator.LIKE_CONTAINS,
 +                        UTF8Type.instance.decompose("What you get by achieving your goals")),
 +                buildExpression(age, Operator.GT, Int32Type.instance.decompose(32)));
 +
 +        Assert.assertEquals(rows.toString(), Collections.singleton("key1"), rows);
 +
 +        rows = getIndexed(store, 10,
 +                buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("do it.")));
 +
 +        Assert.assertEquals(rows.toString(), Arrays.asList("key0", "key1"), Lists.newArrayList(rows));
 +    }
 +
 +    @Test
 +    public void testPrefixSearchWithContainsMode() throws Exception
 +    {
 +        testPrefixSearchWithContainsMode(false);
 +        cleanupData();
 +        testPrefixSearchWithContainsMode(true);
 +    }
 +
 +    private void testPrefixSearchWithContainsMode(boolean forceFlush) throws Exception
 +    {
 +        ColumnFamilyStore store = Keyspace.open(KS_NAME).getColumnFamilyStore(FTS_CF_NAME);
 +
 +        executeCQL(FTS_CF_NAME, "INSERT INTO %s.%s (song_id, title, artist) VALUES(?, ?, ?)", UUID.fromString("1a4abbcd-b5de-4c69-a578-31231e01ff09"), "Poker Face", "Lady Gaga");
 +        executeCQL(FTS_CF_NAME, "INSERT INTO %s.%s (song_id, title, artist) VALUES(?, ?, ?)", UUID.fromString("9472a394-359b-4a06-b1d5-b6afce590598"), "Forgetting the Way Home", "Our Lady of Bells");
 +        executeCQL(FTS_CF_NAME, "INSERT INTO %s.%s (song_id, title, artist) VALUES(?, ?, ?)", UUID.fromString("4f8dc18e-54e6-4e16-b507-c5324b61523b"), "Zamki na piasku", "Lady Pank");
 +        executeCQL(FTS_CF_NAME, "INSERT INTO %s.%s (song_id, title, artist) VALUES(?, ?, ?)", UUID.fromString("eaf294fa-bad5-49d4-8f08-35ba3636a706"), "Koncertowa", "Lady Pank");
 +
 +
 +        if (forceFlush)
 +            store.forceBlockingFlush();
 +
 +        final UntypedResultSet results = executeCQL(FTS_CF_NAME, "SELECT * FROM %s.%s WHERE artist LIKE 'lady%%'");
 +        Assert.assertNotNull(results);
 +        Assert.assertEquals(3, results.size());
 +    }
 +
 +    @Test
 +    public void testMultiExpressionQueriesWhereRowSplitBetweenSSTables() throws Exception
 +    {
 +        testMultiExpressionQueriesWhereRowSplitBetweenSSTables(false);
 +        cleanupData();
 +        testMultiExpressionQueriesWhereRowSplitBetweenSSTables(true);
 +    }
 +
 +    private void testMultiExpressionQueriesWhereRowSplitBetweenSSTables(boolean forceFlush) throws Exception
 +    {
 +        Map<String, Pair<String, Integer>> part1 = new HashMap<String, Pair<String, Integer>>()
 +        {{
 +                put("key0", Pair.create("Maxie", -1));
 +                put("key1", Pair.create("Chelsie", 33));
 +                put("key2", Pair.create((String)null, 43));
 +                put("key3", Pair.create("Shanna", 27));
 +                put("key4", Pair.create("Amiya", 36));
 +        }};
 +
 +        loadData(part1, forceFlush); // first sstable
 +
 +        Map<String, Pair<String, Integer>> part2 = new HashMap<String, Pair<String, Integer>>()
 +        {{
 +                put("key5", Pair.create("Americo", 20));
 +                put("key6", Pair.create("Fiona", 39));
 +                put("key7", Pair.create("Francis", 41));
 +                put("key8", Pair.create("Charley", 21));
 +                put("key9", Pair.create("Amely", 40));
 +                put("key14", Pair.create((String)null, 28));
 +        }};
 +
 +        loadData(part2, forceFlush);
 +
 +        Map<String, Pair<String, Integer>> part3 = new HashMap<String, Pair<String, Integer>>()
 +        {{
 +                put("key0", Pair.create((String)null, 43));
 +                put("key10", Pair.create("Eddie", 42));
 +                put("key11", Pair.create("Oswaldo", 35));
 +                put("key12", Pair.create("Susana", 35));
 +                put("key13", Pair.create("Alivia", 42));
 +                put("key14", Pair.create("Demario", -1));
 +                put("key2", Pair.create("Josephine", -1));
 +        }};
 +
 +        ColumnFamilyStore store = loadData(part3, forceFlush);
 +
 +        final ByteBuffer firstName = UTF8Type.instance.decompose("first_name");
 +        final ByteBuffer age = UTF8Type.instance.decompose("age");
 +
 +        Set<String> rows = getIndexed(store, 10,
 +                                      buildExpression(firstName, Operator.EQ, UTF8Type.instance.decompose("Fiona")),
 +                                      buildExpression(age, Operator.LT, Int32Type.instance.decompose(40)));
 +
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key6" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10,
 +                          buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")));
 +
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key0", "key11", "key12", "key13", "key14",
 +                                                                        "key3", "key4", "key6", "key7", "key8" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 5,
 +                          buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")));
 +
 +        Assert.assertEquals(rows.toString(), 5, rows.size());
 +
 +        rows = getIndexed(store, 10,
 +                          buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                          buildExpression(age, Operator.GTE, Int32Type.instance.decompose(35)));
 +
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key0", "key11", "key12", "key13", "key4", "key6", "key7" },
 +                                                         rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10,
 +                          buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                          buildExpression(age, Operator.LT, Int32Type.instance.decompose(32)));
 +
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key14", "key3", "key8" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10,
 +                          buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                          buildExpression(age, Operator.GT, Int32Type.instance.decompose(27)),
 +                          buildExpression(age, Operator.LT, Int32Type.instance.decompose(32)));
 +
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key14" }, rows.toArray(new String[rows.size()])));
 +
 +        Map<String, Pair<String, Integer>> part4 = new HashMap<String, Pair<String, Integer>>()
 +        {{
 +                put("key12", Pair.create((String)null, 12));
 +                put("key14", Pair.create("Demario", 42));
 +                put("key2", Pair.create("Frank", -1));
 +        }};
 +
 +        store = loadData(part4, forceFlush);
 +
 +        rows = getIndexed(store, 10,
 +                          buildExpression(firstName, Operator.LIKE_MATCHES, UTF8Type.instance.decompose("Susana")),
 +                          buildExpression(age, Operator.LTE, Int32Type.instance.decompose(13)),
 +                          buildExpression(age, Operator.GT, Int32Type.instance.decompose(10)));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key12" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10,
 +                          buildExpression(firstName, Operator.LIKE_MATCHES, UTF8Type.instance.decompose("Demario")),
 +                          buildExpression(age, Operator.LTE, Int32Type.instance.decompose(30)));
 +        Assert.assertTrue(rows.toString(), rows.size() == 0);
 +
 +        rows = getIndexed(store, 10,
 +                          buildExpression(firstName, Operator.LIKE_MATCHES, UTF8Type.instance.decompose("Josephine")));
 +        Assert.assertTrue(rows.toString(), rows.size() == 0);
 +
 +        rows = getIndexed(store, 10,
 +                          buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                          buildExpression(age, Operator.GT, Int32Type.instance.decompose(10)));
 +
 +        Assert.assertEquals(rows.toString(), 10, rows.size());
 +
 +        rows = getIndexed(store, 10,
 +                          buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                          buildExpression(age, Operator.LTE, Int32Type.instance.decompose(50)));
 +
 +        Assert.assertEquals(rows.toString(), 10, rows.size());
 +
 +        rows = getIndexed(store, 10,
 +                          buildExpression(firstName, Operator.LIKE_SUFFIX, UTF8Type.instance.decompose("ie")),
 +                          buildExpression(age, Operator.LTE, Int32Type.instance.decompose(43)));
 +
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key0", "key1", "key10" }, rows.toArray(new String[rows.size()])));
 +    }
 +
 +    @Test
 +    public void testPagination() throws Exception
 +    {
 +        testPagination(false);
 +        cleanupData();
 +        testPagination(true);
 +    }
 +
 +    private void testPagination(boolean forceFlush) throws Exception
 +    {
 +        // split data into 3 distinct SSTables to test paging with overlapping token intervals.
 +
 +        Map<String, Pair<String, Integer>> part1 = new HashMap<String, Pair<String, Integer>>()
 +        {{
 +                put("key01", Pair.create("Ali", 33));
 +                put("key02", Pair.create("Jeremy", 41));
 +                put("key03", Pair.create("Elvera", 22));
 +                put("key04", Pair.create("Bailey", 45));
 +                put("key05", Pair.create("Emerson", 32));
 +                put("key06", Pair.create("Kadin", 38));
 +                put("key07", Pair.create("Maggie", 36));
 +                put("key08", Pair.create("Kailey", 36));
 +                put("key09", Pair.create("Armand", 21));
 +                put("key10", Pair.create("Arnold", 35));
 +        }};
 +
 +        Map<String, Pair<String, Integer>> part2 = new HashMap<String, Pair<String, Integer>>()
 +        {{
 +                put("key11", Pair.create("Ken", 38));
 +                put("key12", Pair.create("Penelope", 43));
 +                put("key13", Pair.create("Wyatt", 34));
 +                put("key14", Pair.create("Johnpaul", 34));
 +                put("key15", Pair.create("Trycia", 43));
 +                put("key16", Pair.create("Aida", 21));
 +                put("key17", Pair.create("Devon", 42));
 +        }};
 +
 +        Map<String, Pair<String, Integer>> part3 = new HashMap<String, Pair<String, Integer>>()
 +        {{
 +                put("key18", Pair.create("Christina", 20));
 +                put("key19", Pair.create("Rick", 19));
 +                put("key20", Pair.create("Fannie", 22));
 +                put("key21", Pair.create("Keegan", 29));
 +                put("key22", Pair.create("Ignatius", 36));
 +                put("key23", Pair.create("Ellis", 26));
 +                put("key24", Pair.create("Annamarie", 29));
 +                put("key25", Pair.create("Tianna", 31));
 +                put("key26", Pair.create("Dennis", 32));
 +        }};
 +
 +        ColumnFamilyStore store = loadData(part1, forceFlush);
 +
 +        loadData(part2, forceFlush);
 +        loadData(part3, forceFlush);
 +
 +        final ByteBuffer firstName = UTF8Type.instance.decompose("first_name");
 +        final ByteBuffer age = UTF8Type.instance.decompose("age");
 +
 +        Set<DecoratedKey> uniqueKeys = getPaged(store, 4,
 +                buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                buildExpression(age, Operator.GTE, Int32Type.instance.decompose(21)));
 +
 +
 +        List<String> expected = new ArrayList<String>()
 +        {{
 +                add("key25");
 +                add("key20");
 +                add("key13");
 +                add("key22");
 +                add("key09");
 +                add("key14");
 +                add("key16");
 +                add("key24");
 +                add("key03");
 +                add("key04");
 +                add("key08");
 +                add("key07");
 +                add("key15");
 +                add("key06");
 +                add("key21");
 +        }};
 +
 +        Assert.assertEquals(expected, convert(uniqueKeys));
 +
 +        // now let's test a single equals condition
 +
 +        uniqueKeys = getPaged(store, 4, buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")));
 +
 +        expected = new ArrayList<String>()
 +        {{
 +                add("key25");
 +                add("key20");
 +                add("key13");
 +                add("key22");
 +                add("key09");
 +                add("key14");
 +                add("key16");
 +                add("key24");
 +                add("key03");
 +                add("key04");
 +                add("key18");
 +                add("key08");
 +                add("key07");
 +                add("key15");
 +                add("key06");
 +                add("key21");
 +        }};
 +
 +        Assert.assertEquals(expected, convert(uniqueKeys));
 +
 +        // now let's test something which is smaller than a single page
 +        uniqueKeys = getPaged(store, 4,
 +                              buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                              buildExpression(age, Operator.EQ, Int32Type.instance.decompose(36)));
 +
 +        expected = new ArrayList<String>()
 +        {{
 +                add("key22");
 +                add("key08");
 +                add("key07");
 +        }};
 +
 +        Assert.assertEquals(expected, convert(uniqueKeys));
 +
 +        // the same but with the page size of 2 to test minimal pagination windows
 +
 +        uniqueKeys = getPaged(store, 2,
 +                              buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                              buildExpression(age, Operator.EQ, Int32Type.instance.decompose(36)));
 +
 +        Assert.assertEquals(expected, convert(uniqueKeys));
 +
 +        // and last but not least, test age range query with pagination
 +        uniqueKeys = getPaged(store, 4,
 +                buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                buildExpression(age, Operator.GT, Int32Type.instance.decompose(20)),
 +                buildExpression(age, Operator.LTE, Int32Type.instance.decompose(36)));
 +
 +        expected = new ArrayList<String>()
 +        {{
 +                add("key25");
 +                add("key20");
 +                add("key13");
 +                add("key22");
 +                add("key09");
 +                add("key14");
 +                add("key16");
 +                add("key24");
 +                add("key03");
 +                add("key08");
 +                add("key07");
 +                add("key21");
 +        }};
 +
 +        Assert.assertEquals(expected, convert(uniqueKeys));
 +
 +        Set<String> rows;
 +
 +        rows = executeCQLWithKeys(String.format("SELECT * FROM %s.%s WHERE first_name LIKE '%%a%%' limit 10 ALLOW FILTERING;", KS_NAME, CF_NAME));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key03", "key04", "key09", "key13", "key14", "key16", "key20", "key22", "key24", "key25" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = executeCQLWithKeys(String.format("SELECT * FROM %s.%s WHERE first_name LIKE '%%a%%' and token(id) >= token('key14') limit 5 ALLOW FILTERING;", KS_NAME, CF_NAME));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key03", "key04", "key14", "key16", "key24" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = executeCQLWithKeys(String.format("SELECT * FROM %s.%s WHERE first_name LIKE '%%a%%' and token(id) >= token('key14') and token(id) <= token('key24') limit 5 ALLOW FILTERING;", KS_NAME, CF_NAME));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key14", "key16", "key24" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = executeCQLWithKeys(String.format("SELECT * FROM %s.%s WHERE first_name LIKE '%%a%%' and age > 30 and token(id) >= token('key14') and token(id) <= token('key24') limit 5 ALLOW FILTERING;", KS_NAME, CF_NAME));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key14" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = executeCQLWithKeys(String.format("SELECT * FROM %s.%s WHERE first_name like '%%ie' limit 5 ALLOW FILTERING;", KS_NAME, CF_NAME));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key07", "key20", "key24" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = executeCQLWithKeys(String.format("SELECT * FROM %s.%s WHERE first_name like '%%ie' AND token(id) > token('key24') limit 5 ALLOW FILTERING;", KS_NAME, CF_NAME));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key07", "key24" }, rows.toArray(new String[rows.size()])));
 +    }
 +
 +    @Test
 +    public void testColumnNamesWithSlashes() throws Exception
 +    {
 +        testColumnNamesWithSlashes(false);
 +        cleanupData();
 +        testColumnNamesWithSlashes(true);
 +    }
 +
 +    private void testColumnNamesWithSlashes(boolean forceFlush) throws Exception
 +    {
 +        ColumnFamilyStore store = Keyspace.open(KS_NAME).getColumnFamilyStore(CF_NAME);
 +
 +        Mutation rm1 = new Mutation(KS_NAME, decoratedKey(AsciiType.instance.decompose("key1")));
 +        rm1.add(PartitionUpdate.singleRowUpdate(store.metadata,
 +                                                rm1.key(),
 +                                                buildRow(buildCell(store.metadata,
 +                                                                   UTF8Type.instance.decompose("/data/output/id"),
 +                                                                   AsciiType.instance.decompose("jason"),
 +                                                                   System.currentTimeMillis()))));
 +
 +        Mutation rm2 = new Mutation(KS_NAME, decoratedKey(AsciiType.instance.decompose("key2")));
 +        rm2.add(PartitionUpdate.singleRowUpdate(store.metadata,
 +                                                rm2.key(),
 +                                                buildRow(buildCell(store.metadata,
 +                                                                   UTF8Type.instance.decompose("/data/output/id"),
 +                                                                   AsciiType.instance.decompose("pavel"),
 +                                                                   System.currentTimeMillis()))));
 +
 +        Mutation rm3 = new Mutation(KS_NAME, decoratedKey(AsciiType.instance.decompose("key3")));
 +        rm3.add(PartitionUpdate.singleRowUpdate(store.metadata,
 +                                                rm3.key(),
 +                                                buildRow(buildCell(store.metadata,
 +                                                                   UTF8Type.instance.decompose("/data/output/id"),
 +                                                                   AsciiType.instance.decompose("Aleksey"),
 +                                                                   System.currentTimeMillis()))));
 +
 +        rm1.apply();
 +        rm2.apply();
 +        rm3.apply();
 +
 +        if (forceFlush)
 +            store.forceBlockingFlush();
 +
 +        final ByteBuffer dataOutputId = UTF8Type.instance.decompose("/data/output/id");
 +
 +        Set<String> rows = getIndexed(store, 10, buildExpression(dataOutputId, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1", "key2" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(dataOutputId, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("A")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[]{ "key3" }, rows.toArray(new String[rows.size()])));
 +
 +        // doesn't really make sense to rebuild index for in-memory data
 +        if (!forceFlush)
 +            return;
 +
 +        store.indexManager.invalidateAllIndexesBlocking();
 +
 +        rows = getIndexed(store, 10, buildExpression(dataOutputId, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")));
 +        Assert.assertTrue(rows.toString(), rows.isEmpty());
 +
 +        rows = getIndexed(store, 10, buildExpression(dataOutputId, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("A")));
 +        Assert.assertTrue(rows.toString(), rows.isEmpty());
 +
 +        // now let's trigger index rebuild and check if we got the data back
 +        store.indexManager.buildIndexBlocking(store.indexManager.getIndexByName("data_output_id"));
 +
 +        rows = getIndexed(store, 10, buildExpression(dataOutputId, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1", "key2" }, rows.toArray(new String[rows.size()])));
 +
 +        // also let's try to build an index for column which has no data to make sure that doesn't fail
 +        store.indexManager.buildIndexBlocking(store.indexManager.getIndexByName("first_name"));
 +        store.indexManager.buildIndexBlocking(store.indexManager.getIndexByName("data_output_id"));
 +
 +        rows = getIndexed(store, 10, buildExpression(dataOutputId, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1", "key2" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(dataOutputId, Operator.LIKE_SUFFIX, UTF8Type.instance.decompose("el")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key2" }, rows.toArray(new String[rows.size()])));
 +    }
 +
 +    @Test
 +    public void testInvalidate() throws Exception
 +    {
 +        testInvalidate(false);
 +        cleanupData();
 +        testInvalidate(true);
 +    }
 +
 +    private void testInvalidate(boolean forceFlush) throws Exception
 +    {
 +        Map<String, Pair<String, Integer>> part1 = new HashMap<String, Pair<String, Integer>>()
 +        {{
 +                put("key0", Pair.create("Maxie", -1));
 +                put("key1", Pair.create("Chelsie", 33));
 +                put("key2", Pair.create((String) null, 43));
 +                put("key3", Pair.create("Shanna", 27));
 +                put("key4", Pair.create("Amiya", 36));
 +        }};
 +
 +        ColumnFamilyStore store = loadData(part1, forceFlush);
 +
 +        final ByteBuffer firstName = UTF8Type.instance.decompose("first_name");
 +        final ByteBuffer age = UTF8Type.instance.decompose("age");
 +
 +        Set<String> rows = getIndexed(store, 10, buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[]{ "key0", "key3", "key4" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(age, Operator.EQ, Int32Type.instance.decompose(33)));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[]{ "key1" }, rows.toArray(new String[rows.size()])));
 +
 +        store.indexManager.invalidateAllIndexesBlocking();
 +
 +        rows = getIndexed(store, 10, buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")));
 +        Assert.assertTrue(rows.toString(), rows.isEmpty());
 +
 +        rows = getIndexed(store, 10, buildExpression(age, Operator.EQ, Int32Type.instance.decompose(33)));
 +        Assert.assertTrue(rows.toString(), rows.isEmpty());
 +
 +
 +        Map<String, Pair<String, Integer>> part2 = new HashMap<String, Pair<String, Integer>>()
 +        {{
 +                put("key5", Pair.create("Americo", 20));
 +                put("key6", Pair.create("Fiona", 39));
 +                put("key7", Pair.create("Francis", 41));
 +                put("key8", Pair.create("Fred", 21));
 +                put("key9", Pair.create("Amely", 40));
 +                put("key14", Pair.create("Dino", 28));
 +        }};
 +
 +        loadData(part2, forceFlush);
 +
 +        rows = getIndexed(store, 10, buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[]{ "key6", "key7" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(age, Operator.EQ, Int32Type.instance.decompose(40)));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[]{ "key9" }, rows.toArray(new String[rows.size()])));
 +    }
 +
 +    @Test
 +    public void testTruncate()
 +    {
 +        Map<String, Pair<String, Integer>> part1 = new HashMap<String, Pair<String, Integer>>()
 +        {{
 +                put("key01", Pair.create("Ali", 33));
 +                put("key02", Pair.create("Jeremy", 41));
 +                put("key03", Pair.create("Elvera", 22));
 +                put("key04", Pair.create("Bailey", 45));
 +                put("key05", Pair.create("Emerson", 32));
 +                put("key06", Pair.create("Kadin", 38));
 +                put("key07", Pair.create("Maggie", 36));
 +                put("key08", Pair.create("Kailey", 36));
 +                put("key09", Pair.create("Armand", 21));
 +                put("key10", Pair.create("Arnold", 35));
 +        }};
 +
 +        Map<String, Pair<String, Integer>> part2 = new HashMap<String, Pair<String, Integer>>()
 +        {{
 +                put("key11", Pair.create("Ken", 38));
 +                put("key12", Pair.create("Penelope", 43));
 +                put("key13", Pair.create("Wyatt", 34));
 +                put("key14", Pair.create("Johnpaul", 34));
 +                put("key15", Pair.create("Trycia", 43));
 +                put("key16", Pair.create("Aida", 21));
 +                put("key17", Pair.create("Devon", 42));
 +        }};
 +
 +        Map<String, Pair<String, Integer>> part3 = new HashMap<String, Pair<String, Integer>>()
 +        {{
 +                put("key18", Pair.create("Christina", 20));
 +                put("key19", Pair.create("Rick", 19));
 +                put("key20", Pair.create("Fannie", 22));
 +                put("key21", Pair.create("Keegan", 29));
 +                put("key22", Pair.create("Ignatius", 36));
 +                put("key23", Pair.create("Ellis", 26));
 +                put("key24", Pair.create("Annamarie", 29));
 +                put("key25", Pair.create("Tianna", 31));
 +                put("key26", Pair.create("Dennis", 32));
 +        }};
 +
 +        ColumnFamilyStore store = loadData(part1, 1000, true);
 +
 +        loadData(part2, 2000, true);
 +        loadData(part3, 3000, true);
 +
 +        final ByteBuffer firstName = UTF8Type.instance.decompose("first_name");
 +
 +        Set<String> rows = getIndexed(store, 100, buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")));
 +        Assert.assertEquals(rows.toString(), 16, rows.size());
 +
 +        // make sure we don't prematurely delete anything
 +        store.indexManager.truncateAllIndexesBlocking(500);
 +
 +        rows = getIndexed(store, 100, buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")));
 +        Assert.assertEquals(rows.toString(), 16, rows.size());
 +
 +        store.indexManager.truncateAllIndexesBlocking(1500);
 +
 +        rows = getIndexed(store, 100, buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")));
 +        Assert.assertEquals(rows.toString(), 10, rows.size());
 +
 +        store.indexManager.truncateAllIndexesBlocking(2500);
 +
 +        rows = getIndexed(store, 100, buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")));
 +        Assert.assertEquals(rows.toString(), 6, rows.size());
 +
 +        store.indexManager.truncateAllIndexesBlocking(3500);
 +
 +        rows = getIndexed(store, 100, buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")));
 +        Assert.assertEquals(rows.toString(), 0, rows.size());
 +
 +        // add back in some data just to make sure it all still works
 +        Map<String, Pair<String, Integer>> part4 = new HashMap<String, Pair<String, Integer>>()
 +        {{
 +                put("key40", Pair.create("Tianna", 31));
 +                put("key41", Pair.create("Dennis", 32));
 +        }};
 +
 +        loadData(part4, 4000, true);
 +
 +        rows = getIndexed(store, 100, buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")));
 +        Assert.assertEquals(rows.toString(), 1, rows.size());
 +    }
 +
 +
 +    @Test
 +    public void testConcurrentMemtableReadsAndWrites() throws Exception
 +    {
 +        final ColumnFamilyStore store = Keyspace.open(KS_NAME).getColumnFamilyStore(CF_NAME);
 +
 +        ExecutorService scheduler = Executors.newFixedThreadPool(Runtime.getRuntime().availableProcessors());
 +
 +        final int writeCount = 10000;
 +        final AtomicInteger updates = new AtomicInteger(0);
 +
 +        for (int i = 0; i < writeCount; i++)
 +        {
 +            final String key = "key" + i;
 +            final String firstName = "first_name#" + i;
 +            final String lastName = "last_name#" + i;
 +
 +            scheduler.submit((Runnable) () -> {
 +                try
 +                {
 +                    newMutation(key, firstName, lastName, 26, System.currentTimeMillis()).apply();
 +                    Uninterruptibles.sleepUninterruptibly(5, TimeUnit.MILLISECONDS); // back up a bit to do more reads
 +                }
 +                finally
 +                {
 +                    updates.incrementAndGet();
 +                }
 +            });
 +        }
 +
 +        final ByteBuffer firstName = UTF8Type.instance.decompose("first_name");
 +        final ByteBuffer age = UTF8Type.instance.decompose("age");
 +
 +        int previousCount = 0;
 +
 +        do
 +        {
 +            // this loop figures out if number of search results monotonically increasing
 +            // to make sure that concurrent updates don't interfere with reads, uses first_name and age
 +            // indexes to test correctness of both Trie and SkipList ColumnIndex implementations.
 +
 +            Set<DecoratedKey> rows = getPaged(store, 100, buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                                                          buildExpression(age, Operator.EQ, Int32Type.instance.decompose(26)));
 +
 +            Assert.assertTrue(previousCount <= rows.size());
 +            previousCount = rows.size();
 +        }
 +        while (updates.get() < writeCount);
 +
 +        // to make sure that after all of the right are done we can read all "count" worth of rows
 +        Set<DecoratedKey> rows = getPaged(store, 100, buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                                                      buildExpression(age, Operator.EQ, Int32Type.instance.decompose(26)));
 +
 +        Assert.assertEquals(writeCount, rows.size());
 +    }
 +
 +    @Test
 +    public void testSameKeyInMemtableAndSSTables()
 +    {
 +        final ByteBuffer firstName = UTF8Type.instance.decompose("first_name");
 +        final ByteBuffer age = UTF8Type.instance.decompose("age");
 +
 +        Map<String, Pair<String, Integer>> data1 = new HashMap<String, Pair<String, Integer>>()
 +        {{
 +                put("key1", Pair.create("Pavel", 14));
 +                put("key2", Pair.create("Pavel", 26));
 +                put("key3", Pair.create("Pavel", 27));
 +                put("key4", Pair.create("Jason", 27));
 +        }};
 +
 +        ColumnFamilyStore store = loadData(data1, true);
 +
 +        Map<String, Pair<String, Integer>> data2 = new HashMap<String, Pair<String, Integer>>()
 +        {{
 +                put("key1", Pair.create("Pavel", 14));
 +                put("key2", Pair.create("Pavel", 27));
 +                put("key4", Pair.create("Jason", 28));
 +        }};
 +
 +        loadData(data2, true);
 +
 +        Map<String, Pair<String, Integer>> data3 = new HashMap<String, Pair<String, Integer>>()
 +        {{
 +                put("key1", Pair.create("Pavel", 15));
 +                put("key4", Pair.create("Jason", 29));
 +        }};
 +
 +        loadData(data3, false);
 +
 +        Set<String> rows = getIndexed(store, 100, buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1", "key2", "key3", "key4" }, rows.toArray(new String[rows.size()])));
 +
 +
 +        rows = getIndexed(store, 100, buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                                      buildExpression(age, Operator.EQ, Int32Type.instance.decompose(15)));
 +
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 100, buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                                      buildExpression(age, Operator.EQ, Int32Type.instance.decompose(29)));
 +
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key4" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 100, buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                                      buildExpression(age, Operator.EQ, Int32Type.instance.decompose(27)));
 +
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[]{"key2", "key3"}, rows.toArray(new String[rows.size()])));
 +    }
 +
 +    @Test
 +    public void testInsertingIncorrectValuesIntoAgeIndex()
 +    {
 +        ColumnFamilyStore store = Keyspace.open(KS_NAME).getColumnFamilyStore(CF_NAME);
 +
 +        final ByteBuffer firstName = UTF8Type.instance.decompose("first_name");
 +        final ByteBuffer age = UTF8Type.instance.decompose("age");
 +
 +        Mutation rm = new Mutation(KS_NAME, decoratedKey(AsciiType.instance.decompose("key1")));
 +        update(rm, new ArrayList<Cell>()
 +        {{
 +            add(buildCell(age, LongType.instance.decompose(26L), System.currentTimeMillis()));
 +            add(buildCell(firstName, AsciiType.instance.decompose("pavel"), System.currentTimeMillis()));
 +        }});
 +        rm.apply();
 +
 +        store.forceBlockingFlush();
 +
 +        Set<String> rows = getIndexed(store, 10, buildExpression(firstName, Operator.EQ, UTF8Type.instance.decompose("a")),
 +                                                 buildExpression(age, Operator.GTE, Int32Type.instance.decompose(26)));
 +
 +        // index is expected to have 0 results because age value was of wrong type
 +        Assert.assertEquals(0, rows.size());
 +    }
 +
 +
 +    @Test
 +    public void testUnicodeSupport()
 +    {
 +        testUnicodeSupport(false);
 +        cleanupData();
 +        testUnicodeSupport(true);
 +    }
 +
 +    private void testUnicodeSupport(boolean forceFlush)
 +    {
 +        ColumnFamilyStore store = Keyspace.open(KS_NAME).getColumnFamilyStore(CF_NAME);
 +
 +        final ByteBuffer comment = UTF8Type.instance.decompose("comment");
 +
 +        Mutation rm = new Mutation(KS_NAME, decoratedKey("key1"));
 +        update(rm, comment, UTF8Type.instance.decompose("ⓈⓅⒺⒸⒾⒶⓁ ⒞⒣⒜⒭⒮ and normal ones"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        rm = new Mutation(KS_NAME, decoratedKey("key2"));
 +        update(rm, comment, UTF8Type.instance.decompose("龍馭鬱"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        rm = new Mutation(KS_NAME, decoratedKey("key3"));
 +        update(rm, comment, UTF8Type.instance.decompose("インディアナ"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        rm = new Mutation(KS_NAME, decoratedKey("key4"));
 +        update(rm, comment, UTF8Type.instance.decompose("レストラン"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        rm = new Mutation(KS_NAME, decoratedKey("key5"));
 +        update(rm, comment, UTF8Type.instance.decompose("ベンジャミン ウエスト"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        if (forceFlush)
 +            store.forceBlockingFlush();
 +
 +        Set<String> rows;
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("ⓈⓅⒺⒸⒾ")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("normal")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("龍")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key2" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("鬱")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key2" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("馭鬱")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key2" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("龍馭鬱")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key2" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("ベンジャミン")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key5" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("レストラ")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key4" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("インディ")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key3" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("ベンジャミ")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key5" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_SUFFIX, UTF8Type.instance.decompose("ン")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key4", "key5" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_MATCHES, UTF8Type.instance.decompose("レストラン")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key4" }, rows.toArray(new String[rows.size()])));
 +    }
 +
 +    @Test
 +    public void testUnicodeSuffixModeNoSplits()
 +    {
 +        testUnicodeSuffixModeNoSplits(false);
 +        cleanupData();
 +        testUnicodeSuffixModeNoSplits(true);
 +    }
 +
 +    private void testUnicodeSuffixModeNoSplits(boolean forceFlush)
 +    {
 +        ColumnFamilyStore store = Keyspace.open(KS_NAME).getColumnFamilyStore(CF_NAME);
 +
 +        final ByteBuffer comment = UTF8Type.instance.decompose("comment_suffix_split");
 +
 +        Mutation rm = new Mutation(KS_NAME, decoratedKey("key1"));
 +        update(rm, comment, UTF8Type.instance.decompose("龍馭鬱"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        rm = new Mutation(KS_NAME, decoratedKey("key2"));
 +        update(rm, comment, UTF8Type.instance.decompose("インディアナ"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        rm = new Mutation(KS_NAME, decoratedKey("key3"));
 +        update(rm, comment, UTF8Type.instance.decompose("レストラン"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        rm = new Mutation(KS_NAME, decoratedKey("key4"));
 +        update(rm, comment, UTF8Type.instance.decompose("ベンジャミン ウエスト"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        if (forceFlush)
 +            store.forceBlockingFlush();
 +
 +        Set<String> rows;
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("龍")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("鬱")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("馭鬱")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("龍馭鬱")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("ベンジャミン")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key4" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("トラン")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key3" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("ディア")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key2" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("ジャミン")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key4" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("ン")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key2", "key3", "key4" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_SUFFIX, UTF8Type.instance.decompose("ン")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key3" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_MATCHES, UTF8Type.instance.decompose("ベンジャミン ウエスト")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key4" }, rows.toArray(new String[rows.size()])));
 +    }
 +
 +    @Test
 +    public void testThatTooBigValueIsRejected()
 +    {
 +        ColumnFamilyStore store = Keyspace.open(KS_NAME).getColumnFamilyStore(CF_NAME);
 +
 +        final ByteBuffer comment = UTF8Type.instance.decompose("comment_suffix_split");
 +
 +        for (int i = 0; i < 10; i++)
 +        {
 +            byte[] randomBytes = new byte[ThreadLocalRandom.current().nextInt(OnDiskIndexBuilder.MAX_TERM_SIZE, 5 * OnDiskIndexBuilder.MAX_TERM_SIZE)];
 +            ThreadLocalRandom.current().nextBytes(randomBytes);
 +
 +            final ByteBuffer bigValue = UTF8Type.instance.decompose(new String(randomBytes));
 +
 +            Mutation rm = new Mutation(KS_NAME, decoratedKey("key1"));
 +            update(rm, comment, bigValue, System.currentTimeMillis());
 +            rm.apply();
 +
 +            Set<String> rows;
 +
 +            rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_MATCHES, bigValue.duplicate()));
 +            Assert.assertEquals(0, rows.size());
 +
 +            store.forceBlockingFlush();
 +
 +            rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_MATCHES, bigValue.duplicate()));
 +            Assert.assertEquals(0, rows.size());
 +        }
 +    }
 +
 +    @Test
 +    public void testSearchTimeouts() throws Exception
 +    {
 +        final ByteBuffer firstName = UTF8Type.instance.decompose("first_name");
 +
 +        Map<String, Pair<String, Integer>> data1 = new HashMap<String, Pair<String, Integer>>()
 +        {{
 +                put("key1", Pair.create("Pavel", 14));
 +                put("key2", Pair.create("Pavel", 26));
 +                put("key3", Pair.create("Pavel", 27));
 +                put("key4", Pair.create("Jason", 27));
 +        }};
 +
 +        ColumnFamilyStore store = loadData(data1, true);
 +
 +        RowFilter filter = RowFilter.create();
 +        filter.add(store.metadata.getColumnDefinition(firstName), Operator.LIKE_CONTAINS, AsciiType.instance.fromString("a"));
 +
 +        ReadCommand command =
 +            PartitionRangeReadCommand.create(false,
 +                                             store.metadata,
 +                                             FBUtilities.nowInSeconds(),
 +                                             ColumnFilter.all(store.metadata),
 +                                             filter,
 +                                             DataLimits.NONE,
 +                                             DataRange.allData(store.metadata.partitioner));
 +        try
 +        {
 +            new QueryPlan(store, command, 0).execute(ReadExecutionController.empty());
 +            Assert.fail();
 +        }
 +        catch (TimeQuotaExceededException e)
 +        {
 +            // correct behavior
 +        }
 +        catch (Exception e)
 +        {
 +            Assert.fail();
 +            e.printStackTrace();
 +        }
 +
 +        // to make sure that query doesn't fail in normal conditions
 +
 +        try (ReadExecutionController controller = command.executionController())
 +        {
 +            Set<String> rows = getKeys(new QueryPlan(store, command, DatabaseDescriptor.getRangeRpcTimeout()).execute(controller));
 +            Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1", "key2", "key3", "key4" }, rows.toArray(new String[rows.size()])));
 +        }
 +    }
 +
 +    @Test
 +    public void testLowerCaseAnalyzer()
 +    {
 +        testLowerCaseAnalyzer(false);
 +        cleanupData();
 +        testLowerCaseAnalyzer(true);
 +    }
 +
 +    @Test
 +    public void testChinesePrefixSearch()
 +    {
 +        ColumnFamilyStore store = Keyspace.open(KS_NAME).getColumnFamilyStore(CF_NAME);
 +
 +        final ByteBuffer fullName = UTF8Type.instance.decompose("/output/full-name/");
 +
 +        Mutation rm = new Mutation(KS_NAME, decoratedKey("key1"));
 +        update(rm, fullName, UTF8Type.instance.decompose("美加 八田"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        rm = new Mutation(KS_NAME, decoratedKey("key2"));
 +        update(rm, fullName, UTF8Type.instance.decompose("仁美 瀧澤"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        rm = new Mutation(KS_NAME, decoratedKey("key3"));
 +        update(rm, fullName, UTF8Type.instance.decompose("晃宏 高須"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        rm = new Mutation(KS_NAME, decoratedKey("key4"));
 +        update(rm, fullName, UTF8Type.instance.decompose("弘孝 大竹"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        rm = new Mutation(KS_NAME, decoratedKey("key5"));
 +        update(rm, fullName, UTF8Type.instance.decompose("満枝 榎本"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        rm = new Mutation(KS_NAME, decoratedKey("key6"));
 +        update(rm, fullName, UTF8Type.instance.decompose("飛鳥 上原"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        rm = new Mutation(KS_NAME, decoratedKey("key7"));
 +        update(rm, fullName, UTF8Type.instance.decompose("大輝 鎌田"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        rm = new Mutation(KS_NAME, decoratedKey("key8"));
 +        update(rm, fullName, UTF8Type.instance.decompose("利久 寺地"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        store.forceBlockingFlush();
 +
 +
 +        Set<String> rows;
 +
 +        rows = getIndexed(store, 10, buildExpression(fullName, Operator.EQ, UTF8Type.instance.decompose("美加 八田")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(fullName, Operator.LIKE_PREFIX, UTF8Type.instance.decompose("美加")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(fullName, Operator.EQ, UTF8Type.instance.decompose("晃宏 高須")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key3" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(fullName, Operator.LIKE_PREFIX, UTF8Type.instance.decompose("大輝")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key7" }, rows.toArray(new String[rows.size()])));
 +    }
 +
 +    public void testLowerCaseAnalyzer(boolean forceFlush)
 +    {
 +        ColumnFamilyStore store = Keyspace.open(KS_NAME).getColumnFamilyStore(CF_NAME);
 +
 +        final ByteBuffer comment = UTF8Type.instance.decompose("address");
 +
 +        Mutation rm = new Mutation(KS_NAME, decoratedKey("key1"));
 +        update(rm, comment, UTF8Type.instance.decompose("577 Rogahn Valleys Apt. 178"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        rm = new Mutation(KS_NAME, decoratedKey("key2"));
 +        update(rm, comment, UTF8Type.instance.decompose("89809 Beverly Course Suite 089"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        rm = new Mutation(KS_NAME, decoratedKey("key3"));
 +        update(rm, comment, UTF8Type.instance.decompose("165 clydie oval apt. 399"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        if (forceFlush)
 +            store.forceBlockingFlush();
 +
 +        Set<String> rows;
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_PREFIX, UTF8Type.instance.decompose("577 Rogahn Valleys")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_PREFIX, UTF8Type.instance.decompose("577 ROgAhn VallEYs")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_PREFIX, UTF8Type.instance.decompose("577 rogahn valleys")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_PREFIX, UTF8Type.instance.decompose("577 rogahn")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_PREFIX, UTF8Type.instance.decompose("57")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_PREFIX, UTF8Type.instance.decompose("89809 Beverly Course")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key2" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_PREFIX, UTF8Type.instance.decompose("89809 BEVERly COURSE")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key2" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_PREFIX, UTF8Type.instance.decompose("89809 beverly course")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key2" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_PREFIX, UTF8Type.instance.decompose("89809 Beverly")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key2" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_PREFIX, UTF8Type.instance.decompose("8980")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key2" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_PREFIX, UTF8Type.instance.decompose("165 ClYdie OvAl APT. 399")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key3" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_PREFIX, UTF8Type.instance.decompose("165 Clydie Oval Apt. 399")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key3" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_PREFIX, UTF8Type.instance.decompose("165 clydie oval apt. 399")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key3" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_PREFIX, UTF8Type.instance.decompose("165 ClYdie OvA")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key3" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_PREFIX, UTF8Type.instance.decompose("165 ClYdi")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key3" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_PREFIX, UTF8Type.instance.decompose("165")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key3" }, rows.toArray(new String[rows.size()])));
 +    }
 +
 +    @Test
 +    public void testPrefixSSTableLookup()
 +    {
 +        // This test coverts particular case which interval lookup can return invalid results
 +        // when queried on the prefix e.g. "j".
 +        ColumnFamilyStore store = Keyspace.open(KS_NAME).getColumnFamilyStore(CF_NAME);
 +
 +        final ByteBuffer name = UTF8Type.instance.decompose("first_name_prefix");
 +
 +        Mutation rm;
 +
 +        rm = new Mutation(KS_NAME, decoratedKey("key1"));
 +        update(rm, name, UTF8Type.instance.decompose("Pavel"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        rm = new Mutation(KS_NAME, decoratedKey("key2"));
 +        update(rm, name, UTF8Type.instance.decompose("Jordan"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        rm = new Mutation(KS_NAME, decoratedKey("key3"));
 +        update(rm, name, UTF8Type.instance.decompose("Mikhail"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        rm = new Mutation(KS_NAME, decoratedKey("key4"));
 +        update(rm, name, UTF8Type.instance.decompose("Michael"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        rm = new Mutation(KS_NAME, decoratedKey("key5"));
 +        update(rm, name, UTF8Type.instance.decompose("Johnny"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        // first flush would make interval for name - 'johnny' -> 'pavel'
 +        store.forceBlockingFlush();
 +
 +        rm = new Mutation(KS_NAME, decoratedKey("key6"));
 +        update(rm, name, UTF8Type.instance.decompose("Jason"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        rm = new Mutation(KS_NAME, decoratedKey("key7"));
 +        update(rm, name, UTF8Type.instance.decompose("Vijay"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        rm = new Mutation(KS_NAME, decoratedKey("key8")); // this name is going to be tokenized
 +        update(rm, name, UTF8Type.instance.decompose("Jean-Claude"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        // this flush is going to produce range - 'jason' -> 'vijay'
 +        store.forceBlockingFlush();
 +
 +        // make sure that overlap of the prefixes is properly handled across sstables
 +        // since simple interval tree lookup is not going to cover it, prefix lookup actually required.
 +
 +        Set<String> rows;
 +
 +        rows = getIndexed(store, 10, buildExpression(name, Operator.LIKE_PREFIX, UTF8Type.instance.decompose("J")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key2", "key5", "key6", "key8"}, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(name, Operator.LIKE_PREFIX, UTF8Type.instance.decompose("j")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key2", "key5", "key6", "key8" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(name, Operator.LIKE_PREFIX, UTF8Type.instance.decompose("m")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key3", "key4" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(name, Operator.LIKE_PREFIX, UTF8Type.instance.decompose("v")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key7" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(name, Operator.LIKE_PREFIX, UTF8Type.instance.decompose("p")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(name, Operator.LIKE_PREFIX, UTF8Type.instance.decompose("j")),
 +                                     buildExpression(name, Operator.NEQ, UTF8Type.instance.decompose("joh")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key2", "key6", "key8" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(name, Operator.LIKE_MATCHES, UTF8Type.instance.decompose("pavel")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(name, Operator.EQ, UTF8Type.instance.decompose("Pave")));
 +        Assert.assertTrue(rows.isEmpty());
 +
 +        rows = getIndexed(store, 10, buildExpression(name, Operator.EQ, UTF8Type.instance.decompose("Pavel")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(name, Operator.LIKE_MATCHES, UTF8Type.instance.decompose("JeAn")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key8" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(name, Operator.LIKE_MATCHES, UTF8Type.instance.decompose("claUde")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key8" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(name, Operator.EQ, UTF8Type.instance.decompose("Jean")));
 +        Assert.assertTrue(rows.isEmpty());
 +
 +        rows = getIndexed(store, 10, buildExpression(name, Operator.EQ, UTF8Type.instance.decompose("Jean-Claude")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key8" }, rows.toArray(new String[rows.size()])));
 +    }
 +
 +    @Test
 +    public void testSettingIsLiteralOption()
 +    {
 +
 +        // special type which is UTF-8 but is only on the inside
 +        AbstractType<?> stringType = new AbstractType<String>(AbstractType.ComparisonType.CUSTOM)
 +        {
 +            public ByteBuffer fromString(String source) throws MarshalException
 +            {
 +                return UTF8Type.instance.fromString(source);
 +            }
 +
 +            public Term fromJSONObject(Object parsed) throws MarshalException
 +            {
 +                throw new UnsupportedOperationException();
 +            }
 +
 +            public TypeSerializer<String> getSerializer()
 +            {
 +                return UTF8Type.instance.getSerializer();
 +            }
 +
 +            public int compareCustom(ByteBuffer a, ByteBuffer b)
 +            {
 +                return UTF8Type.instance.compare(a, b);
 +            }
 +        };
 +
 +        // first let's check that we get 'false' for 'isLiteral' if we don't set the option with special comparator
 +        ColumnDefinition columnA = ColumnDefinition.regularDef(KS_NAME, CF_NAME, "special-A", stringType);
 +
 +        ColumnIndex indexA = new ColumnIndex(UTF8Type.instance, columnA, IndexMetadata.fromSchemaMetadata("special-index-A", IndexMetadata.Kind.CUSTOM, new HashMap<String, String>()
 +        {{
 +            put(IndexTarget.CUSTOM_INDEX_OPTION_NAME, SASIIndex.class.getName());
 +        }}));
 +
 +        Assert.assertEquals(true,  indexA.isIndexed());
 +        Assert.assertEquals(false, indexA.isLiteral());
 +
 +        // now let's double-check that we do get 'true' when we set it
 +        ColumnDefinition columnB = ColumnDefinition.regularDef(KS_NAME, CF_NAME, "special-B", stringType);
 +
 +        ColumnIndex indexB = new ColumnIndex(UTF8Type.instance, columnB, IndexMetadata.fromSchemaMetadata("special-index-B", IndexMetadata.Kind.CUSTOM, new HashMap<String, String>()
 +        {{
 +            put(IndexTarget.CUSTOM_INDEX_OPTION_NAME, SASIIndex.class.getName());
 +            put("is_literal", "true");
 +        }}));
 +
 +        Assert.assertEquals(true, indexB.isIndexed());
 +        Assert.assertEquals(true, indexB.isLiteral());
 +
 +        // and finally we should also get a 'true' if it's built-in UTF-8/ASCII comparator
 +        ColumnDefinition columnC = ColumnDefinition.regularDef(KS_NAME, CF_NAME, "special-C", UTF8Type.instance);
 +
 +        ColumnIndex indexC = new ColumnIndex(UTF8Type.instance, columnC, IndexMetadata.fromSchemaMetadata("special-index-C", IndexMetadata.Kind.CUSTOM, new HashMap<String, String>()
 +        {{
 +            put(IndexTarget.CUSTOM_INDEX_OPTION_NAME, SASIIndex.class.getName());
 +        }}));
 +
 +        Assert.assertEquals(true, indexC.isIndexed());
 +        Assert.assertEquals(true, indexC.isLiteral());
 +
 +        ColumnDefinition columnD = ColumnDefinition.regularDef(KS_NAME, CF_NAME, "special-D", AsciiType.instance);
 +
 +        ColumnIndex indexD = new ColumnIndex(UTF8Type.instance, columnD, IndexMetadata.fromSchemaMetadata("special-index-D", IndexMetadata.Kind.CUSTOM, new HashMap<String, String>()
 +        {{
 +            put(IndexTarget.CUSTOM_INDEX_OPTION_NAME, SASIIndex.class.getName());
 +        }}));
 +
 +        Assert.assertEquals(true, indexD.isIndexed());
 +        Assert.assertEquals(true, indexD.isLiteral());
 +
 +        // and option should supersedes the comparator type
 +        ColumnDefinition columnE = ColumnDefinition.regularDef(KS_NAME, CF_NAME, "special-E", UTF8Type.instance);
 +
 +        ColumnIndex indexE = new ColumnIndex(UTF8Type.instance, columnE, IndexMetadata.fromSchemaMetadata("special-index-E", IndexMetadata.Kind.CUSTOM, new HashMap<String, String>()
 +        {{
 +            put(IndexTarget.CUSTOM_INDEX_OPTION_NAME, SASIIndex.class.getName());
 +            put("is_literal", "false");
 +        }}));
 +
 +        Assert.assertEquals(true,  indexE.isIndexed());
 +        Assert.assertEquals(false, indexE.isLiteral());
 +
 +        // test frozen-collection
 +        ColumnDefinition columnF = ColumnDefinition.regularDef(KS_NAME,
 +                                                               CF_NAME,
 +                                                               "special-F",
 +                                                               ListType.getInstance(UTF8Type.instance, false));
 +
 +        ColumnIndex indexF = new ColumnIndex(UTF8Type.instance, columnF, IndexMetadata.fromSchemaMetadata("special-index-F", IndexMetadata.Kind.CUSTOM, new HashMap<String, String>()
 +        {{
 +            put(IndexTarget.CUSTOM_INDEX_OPTION_NAME, SASIIndex.class.getName());
 +        }}));
 +
 +        Assert.assertEquals(true,  indexF.isIndexed());
 +        Assert.assertEquals(false, indexF.isLiteral());
 +    }
 +
 +    @Test
 +    public void testClusteringIndexes() throws Exception
 +    {
 +        testClusteringIndexes(false);
 +        cleanupData();
 +        testClusteringIndexes(true);
 +    }
 +
 +    public void testClusteringIndexes(boolean forceFlush) throws Exception
 +    {
 +        ColumnFamilyStore store = Keyspace.open(KS_NAME).getColumnFamilyStore(CLUSTERING_CF_NAME_1);
 +
 +        executeCQL(CLUSTERING_CF_NAME_1, "INSERT INTO %s.%s (name, nick

<TRUNCATED>

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


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

Posted by if...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/07fbd8ee/test/unit/org/apache/cassandra/cql3/validation/operations/SelectTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/SelectTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/SelectTest.java
index e22519b..4f12b2b 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/SelectTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/SelectTest.java
@@ -60,7 +60,6 @@ public class SelectTest extends CQLTester
         );
 
         // Ascending order
-
         assertRows(execute("SELECT * FROM %s WHERE p=? ORDER BY c ASC", "p1"),
             row("p1", "k1", "sv1", "v1"),
             row("p1", "k2", "sv1", "v2")
@@ -313,65 +312,6 @@ public class SelectTest extends CQLTester
         assertRowCount(execute("SELECT firstname, lastname FROM %s WHERE userid IN (?, ?)", id1, id2), 2);
     }
 
-    /**
-     * Check query with KEY IN clause for wide row tables
-     * migrated from cql_tests.py:TestCQL.in_clause_wide_rows_test()
-     */
-    @Test
-    public void testSelectKeyInForWideRows() throws Throwable
-    {
-        createTable("CREATE TABLE %s (k int, c int, v int, PRIMARY KEY (k, c)) WITH COMPACT STORAGE");
-
-        for (int i = 0; i < 10; i++)
-            execute("INSERT INTO %s (k, c, v) VALUES (0, ?, ?)", i, i);
-
-        assertRows(execute("SELECT v FROM %s WHERE k = 0 AND c IN (5, 2, 8)"),
-                   row(2), row(5), row(8));
-
-        createTable("CREATE TABLE %s (k int, c1 int, c2 int, v int, PRIMARY KEY (k, c1, c2)) WITH COMPACT STORAGE");
-
-        for (int i = 0; i < 10; i++)
-            execute("INSERT INTO %s (k, c1, c2, v) VALUES (0, 0, ?, ?)", i, i);
-
-        assertEmpty(execute("SELECT v FROM %s WHERE k = 0 AND c1 IN (5, 2, 8) AND c2 = 3"));
-
-        assertRows(execute("SELECT v FROM %s WHERE k = 0 AND c1 = 0 AND c2 IN (5, 2, 8)"),
-                   row(2), row(5), row(8));
-    }
-
-    /**
-     * Check SELECT respects inclusive and exclusive bounds
-     * migrated from cql_tests.py:TestCQL.exclusive_slice_test()
-     */
-    @Test
-    public void testSelectBounds() throws Throwable
-    {
-        createTable("CREATE TABLE %s (k int, c int, v int, PRIMARY KEY (k, c)) WITH COMPACT STORAGE");
-
-        for (int i = 0; i < 10; i++)
-            execute("INSERT INTO %s (k, c, v) VALUES (0, ?, ?)", i, i);
-
-        assertRowCount(execute("SELECT v FROM %s WHERE k = 0"), 10);
-
-        assertRows(execute("SELECT v FROM %s WHERE k = 0 AND c >= 2 AND c <= 6"),
-                   row(2), row(3), row(4), row(5), row(6));
-
-        assertRows(execute("SELECT v FROM %s WHERE k = 0 AND c > 2 AND c <= 6"),
-                   row(3), row(4), row(5), row(6));
-
-        assertRows(execute("SELECT v FROM %s WHERE k = 0 AND c >= 2 AND c < 6"),
-                   row(2), row(3), row(4), row(5));
-
-        assertRows(execute("SELECT v FROM %s WHERE k = 0 AND c > 2 AND c < 6"),
-                   row(3), row(4), row(5));
-
-        assertRows(execute("SELECT v FROM %s WHERE k = 0 AND c > 2 AND c <= 6 LIMIT 2"),
-                   row(3), row(4));
-
-        assertRows(execute("SELECT v FROM %s WHERE k = 0 AND c >= 2 AND c < 6 ORDER BY c DESC LIMIT 2"),
-                   row(5), row(4));
-    }
-
     @Test
     public void testSetContainsWithIndex() throws Throwable
     {
@@ -800,128 +740,6 @@ public class SelectTest extends CQLTester
     }
 
     /**
-     * Test for #4716 bug and more generally for good behavior of ordering,
-     * migrated from cql_tests.py:TestCQL.reversed_compact_test()
-     */
-    @Test
-    public void testReverseCompact() throws Throwable
-    {
-        createTable("CREATE TABLE %s ( k text, c int, v int, PRIMARY KEY (k, c) ) WITH COMPACT STORAGE AND CLUSTERING ORDER BY (c DESC)");
-
-        for (int i = 0; i < 10; i++)
-            execute("INSERT INTO %s (k, c, v) VALUES ('foo', ?, ?)", i, i);
-
-        assertRows(execute("SELECT c FROM %s WHERE c > 2 AND c < 6 AND k = 'foo'"),
-                   row(5), row(4), row(3));
-
-        assertRows(execute("SELECT c FROM %s WHERE c >= 2 AND c <= 6 AND k = 'foo'"),
-                   row(6), row(5), row(4), row(3), row(2));
-
-        assertRows(execute("SELECT c FROM %s WHERE c > 2 AND c < 6 AND k = 'foo' ORDER BY c ASC"),
-                   row(3), row(4), row(5));
-
-        assertRows(execute("SELECT c FROM %s WHERE c >= 2 AND c <= 6 AND k = 'foo' ORDER BY c ASC"),
-                   row(2), row(3), row(4), row(5), row(6));
-
-        assertRows(execute("SELECT c FROM %s WHERE c > 2 AND c < 6 AND k = 'foo' ORDER BY c DESC"),
-                   row(5), row(4), row(3));
-
-        assertRows(execute("SELECT c FROM %s WHERE c >= 2 AND c <= 6 AND k = 'foo' ORDER BY c DESC"),
-                   row(6), row(5), row(4), row(3), row(2));
-
-        createTable("CREATE TABLE %s ( k text, c int, v int, PRIMARY KEY (k, c) ) WITH COMPACT STORAGE");
-
-        for (int i = 0; i < 10; i++)
-            execute("INSERT INTO %s(k, c, v) VALUES ('foo', ?, ?)", i, i);
-
-        assertRows(execute("SELECT c FROM %s WHERE c > 2 AND c < 6 AND k = 'foo'"),
-                   row(3), row(4), row(5));
-
-        assertRows(execute("SELECT c FROM %s WHERE c >= 2 AND c <= 6 AND k = 'foo'"),
-                   row(2), row(3), row(4), row(5), row(6));
-
-        assertRows(execute("SELECT c FROM %s WHERE c > 2 AND c < 6 AND k = 'foo' ORDER BY c ASC"),
-                   row(3), row(4), row(5));
-
-        assertRows(execute("SELECT c FROM %s WHERE c >= 2 AND c <= 6 AND k = 'foo' ORDER BY c ASC"),
-                   row(2), row(3), row(4), row(5), row(6));
-
-        assertRows(execute("SELECT c FROM %s WHERE c > 2 AND c < 6 AND k = 'foo' ORDER BY c DESC"),
-                   row(5), row(4), row(3));
-
-        assertRows(execute("SELECT c FROM %s WHERE c >= 2 AND c <= 6 AND k = 'foo' ORDER BY c DESC"),
-                   row(6), row(5), row(4), row(3), row(2));
-    }
-
-    /**
-     * Test for the bug from #4760 and #4759,
-     * migrated from cql_tests.py:TestCQL.reversed_compact_multikey_test()
-     */
-    @Test
-    public void testReversedCompactMultikey() throws Throwable
-    {
-        createTable("CREATE TABLE %s (key text, c1 int, c2 int, value text, PRIMARY KEY(key, c1, c2) ) WITH COMPACT STORAGE AND CLUSTERING ORDER BY(c1 DESC, c2 DESC)");
-
-        for (int i = 0; i < 3; i++)
-            for (int j = 0; j < 3; j++)
-                execute("INSERT INTO %s (key, c1, c2, value) VALUES ('foo', ?, ?, 'bar')", i, j);
-
-        // Equalities
-        assertRows(execute("SELECT c1, c2 FROM %s WHERE key='foo' AND c1 = 1"),
-                   row(1, 2), row(1, 1), row(1, 0));
-
-        assertRows(execute("SELECT c1, c2 FROM %s WHERE key='foo' AND c1 = 1 ORDER BY c1 ASC, c2 ASC"),
-                   row(1, 0), row(1, 1), row(1, 2));
-
-        assertRows(execute("SELECT c1, c2 FROM %s WHERE key='foo' AND c1 = 1 ORDER BY c1 DESC, c2 DESC"),
-                   row(1, 2), row(1, 1), row(1, 0));
-
-        // GT
-        assertRows(execute("SELECT c1, c2 FROM %s WHERE key='foo' AND c1 > 1"),
-                   row(2, 2), row(2, 1), row(2, 0));
-
-        assertRows(execute("SELECT c1, c2 FROM %s WHERE key='foo' AND c1 > 1 ORDER BY c1 ASC, c2 ASC"),
-                   row(2, 0), row(2, 1), row(2, 2));
-
-        assertRows(execute("SELECT c1, c2 FROM %s WHERE key='foo' AND c1 > 1 ORDER BY c1 DESC, c2 DESC"),
-                   row(2, 2), row(2, 1), row(2, 0));
-
-        assertRows(execute("SELECT c1, c2 FROM %s WHERE key='foo' AND c1 >= 1"),
-                   row(2, 2), row(2, 1), row(2, 0), row(1, 2), row(1, 1), row(1, 0));
-
-        assertRows(execute("SELECT c1, c2 FROM %s WHERE key='foo' AND c1 >= 1 ORDER BY c1 ASC, c2 ASC"),
-                   row(1, 0), row(1, 1), row(1, 2), row(2, 0), row(2, 1), row(2, 2));
-
-        assertRows(execute("SELECT c1, c2 FROM %s WHERE key='foo' AND c1 >= 1 ORDER BY c1 ASC"),
-                   row(1, 0), row(1, 1), row(1, 2), row(2, 0), row(2, 1), row(2, 2));
-
-        assertRows(execute("SELECT c1, c2 FROM %s WHERE key='foo' AND c1 >= 1 ORDER BY c1 DESC, c2 DESC"),
-                   row(2, 2), row(2, 1), row(2, 0), row(1, 2), row(1, 1), row(1, 0));
-
-        // LT
-        assertRows(execute("SELECT c1, c2 FROM %s WHERE key='foo' AND c1 < 1"),
-                   row(0, 2), row(0, 1), row(0, 0));
-
-        assertRows(execute("SELECT c1, c2 FROM %s WHERE key='foo' AND c1 < 1 ORDER BY c1 ASC, c2 ASC"),
-                   row(0, 0), row(0, 1), row(0, 2));
-
-        assertRows(execute("SELECT c1, c2 FROM %s WHERE key='foo' AND c1 < 1 ORDER BY c1 DESC, c2 DESC"),
-                   row(0, 2), row(0, 1), row(0, 0));
-
-        assertRows(execute("SELECT c1, c2 FROM %s WHERE key='foo' AND c1 <= 1"),
-                   row(1, 2), row(1, 1), row(1, 0), row(0, 2), row(0, 1), row(0, 0));
-
-        assertRows(execute("SELECT c1, c2 FROM %s WHERE key='foo' AND c1 <= 1 ORDER BY c1 ASC, c2 ASC"),
-                   row(0, 0), row(0, 1), row(0, 2), row(1, 0), row(1, 1), row(1, 2));
-
-        assertRows(execute("SELECT c1, c2 FROM %s WHERE key='foo' AND c1 <= 1 ORDER BY c1 ASC"),
-                   row(0, 0), row(0, 1), row(0, 2), row(1, 0), row(1, 1), row(1, 2));
-
-        assertRows(execute("SELECT c1, c2 FROM %s WHERE key='foo' AND c1 <= 1 ORDER BY c1 DESC, c2 DESC"),
-                   row(1, 2), row(1, 1), row(1, 0), row(0, 2), row(0, 1), row(0, 0));
-    }
-
-    /**
      * Migrated from cql_tests.py:TestCQL.bug_4882_test()
      */
     @Test
@@ -1032,26 +850,7 @@ public class SelectTest extends CQLTester
     @Test
     public void testMultiSelects() throws Throwable
     {
-        doTestVariousSelects(false);
-    }
-
-    /**
-     * Migrated from cql_tests.py:TestCQL.multi_in_compact_test()
-     */
-    @Test
-    public void testMultiSelectsCompactStorage() throws Throwable
-    {
-        doTestVariousSelects(true);
-    }
-
-
-    public void doTestVariousSelects(boolean compact) throws Throwable
-    {
-        createTable(
-                   "CREATE TABLE %s (group text, zipcode text, state text, fips_regions int, city text, PRIMARY KEY (group, zipcode, state, fips_regions))"
-                   + (compact
-                      ? " WITH COMPACT STORAGE"
-                      : ""));
+        createTable("CREATE TABLE %s (group text, zipcode text, state text, fips_regions int, city text, PRIMARY KEY (group, zipcode, state, fips_regions))");
 
         String str = "INSERT INTO %s (group, zipcode, state, fips_regions, city) VALUES (?, ?, ?, ?, ?)";
         execute(str, "test", "06029", "CT", 9, "Ellington");
@@ -1086,22 +885,6 @@ public class SelectTest extends CQLTester
     }
 
     /**
-     * Migrated from cql_tests.py:TestCQL.multi_in_compact_non_composite_test()
-     */
-    @Test
-    public void testMultiSelectsNonCompositeCompactStorage() throws Throwable
-    {
-        createTable("CREATE TABLE %s (key int, c int, v int, PRIMARY KEY (key, c)) WITH COMPACT STORAGE");
-
-        execute("INSERT INTO %s (key, c, v) VALUES (0, 0, 0)");
-        execute("INSERT INTO %s (key, c, v) VALUES (0, 1, 1)");
-        execute("INSERT INTO %s (key, c, v) VALUES (0, 2, 2)");
-
-        assertRows(execute("SELECT * FROM %s WHERE key=0 AND c IN (0, 2)"),
-                   row(0, 0, 0), row(0, 2, 2));
-    }
-
-    /**
      * Migrated from cql_tests.py:TestCQL.ticket_5230_test()
      */
     @Test
@@ -1215,37 +998,6 @@ public class SelectTest extends CQLTester
         // Test selection validation.
         assertInvalidMessage("queries must request all the partition key columns", "SELECT DISTINCT pk0 FROM %s");
         assertInvalidMessage("queries must only request partition key columns", "SELECT DISTINCT pk0, pk1, ck0 FROM %s");
-
-        //Test a 'compact storage' table.
-        createTable("CREATE TABLE %s (pk0 int, pk1 int, val int, PRIMARY KEY((pk0, pk1))) WITH COMPACT STORAGE");
-
-        for (int i = 0; i < 3; i++)
-            execute("INSERT INTO %s (pk0, pk1, val) VALUES (?, ?, ?)", i, i, i);
-
-        assertRows(execute("SELECT DISTINCT pk0, pk1 FROM %s LIMIT 1"),
-                   row(0, 0));
-
-        assertRows(execute("SELECT DISTINCT pk0, pk1 FROM %s LIMIT 3"),
-                   row(0, 0),
-                   row(2, 2),
-                   row(1, 1));
-
-        // Test a 'wide row' thrift table.
-        createTable("CREATE TABLE %s (pk int, name text, val int, PRIMARY KEY(pk, name)) WITH COMPACT STORAGE");
-
-        for (int i = 0; i < 3; i++)
-        {
-            execute("INSERT INTO %s (pk, name, val) VALUES (?, 'name0', 0)", i);
-            execute("INSERT INTO %s (pk, name, val) VALUES (?, 'name1', 1)", i);
-        }
-
-        assertRows(execute("SELECT DISTINCT pk FROM %s LIMIT 1"),
-                   row(1));
-
-        assertRows(execute("SELECT DISTINCT pk FROM %s LIMIT 3"),
-                   row(1),
-                   row(0),
-                   row(2));
     }
 
     /**
@@ -1358,7 +1110,7 @@ public class SelectTest extends CQLTester
         createTable("CREATE TABLE %s ( k int, v int, PRIMARY KEY (k, v))");
 
         execute("INSERT INTO %s (k, v) VALUES (0, 0)");
-
+        
         flush();
 
         assertRows(execute("SELECT v FROM %s WHERE k=0 AND v IN (1, 0)"),
@@ -1590,154 +1342,6 @@ public class SelectTest extends CQLTester
     }
 
     @Test
-    public void testFilteringOnCompactTablesWithoutIndices() throws Throwable
-    {
-        //----------------------------------------------
-        // Test COMPACT table with clustering columns
-        //----------------------------------------------
-        createTable("CREATE TABLE %s (a int, b int, c int, PRIMARY KEY (a, b)) WITH COMPACT STORAGE");
-
-        execute("INSERT INTO %s (a, b, c) VALUES (1, 2, 4)");
-        execute("INSERT INTO %s (a, b, c) VALUES (1, 3, 6)");
-        execute("INSERT INTO %s (a, b, c) VALUES (1, 4, 4)");
-        execute("INSERT INTO %s (a, b, c) VALUES (2, 3, 7)");
-
-        // Adds tomstones
-        execute("INSERT INTO %s (a, b, c) VALUES (1, 1, 4)");
-        execute("INSERT INTO %s (a, b, c) VALUES (2, 2, 7)");
-        execute("DELETE FROM %s WHERE a = 1 AND b = 1");
-        execute("DELETE FROM %s WHERE a = 2 AND b = 2");
-
-        beforeAndAfterFlush(() -> {
-
-            // Checks filtering
-            assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                                 "SELECT * FROM %s WHERE a = 1 AND b = 4 AND c = 4");
-
-            assertRows(execute("SELECT * FROM %s WHERE a = 1 AND b = 4 AND c = 4 ALLOW FILTERING"),
-                       row(1, 4, 4));
-
-            assertInvalidMessage("IN predicates on non-primary-key columns (c) is not yet supported",
-                                 "SELECT * FROM %s WHERE a IN (1, 2) AND c IN (6, 7)");
-
-            assertInvalidMessage("IN predicates on non-primary-key columns (c) is not yet supported",
-                                 "SELECT * FROM %s WHERE a IN (1, 2) AND c IN (6, 7) ALLOW FILTERING");
-
-            assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                                 "SELECT * FROM %s WHERE c > 4");
-
-            assertRows(execute("SELECT * FROM %s WHERE c > 4 ALLOW FILTERING"),
-                       row(1, 3, 6),
-                       row(2, 3, 7));
-
-            assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                                 "SELECT * FROM %s WHERE b < 3 AND c <= 4");
-
-            assertRows(execute("SELECT * FROM %s WHERE b < 3 AND c <= 4 ALLOW FILTERING"),
-                       row(1, 2, 4));
-
-            assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                                 "SELECT * FROM %s WHERE c >= 3 AND c <= 6");
-
-            assertRows(execute("SELECT * FROM %s WHERE c >= 3 AND c <= 6 ALLOW FILTERING"),
-                       row(1, 2, 4),
-                       row(1, 3, 6),
-                       row(1, 4, 4));
-
-        });
-
-        // Checks filtering with null
-        assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                             "SELECT * FROM %s WHERE c = null");
-        assertInvalidMessage("Unsupported null value for column c",
-                             "SELECT * FROM %s WHERE c = null ALLOW FILTERING");
-        assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                             "SELECT * FROM %s WHERE c > null");
-        assertInvalidMessage("Unsupported null value for column c",
-                             "SELECT * FROM %s WHERE c > null ALLOW FILTERING");
-
-        // Checks filtering with unset
-        assertInvalidMessage("Unsupported unset value for column c",
-                             "SELECT * FROM %s WHERE c = ? ALLOW FILTERING",
-                             unset());
-        assertInvalidMessage("Unsupported unset value for column c",
-                             "SELECT * FROM %s WHERE c > ? ALLOW FILTERING",
-                             unset());
-
-        //----------------------------------------------
-        // Test COMPACT table without clustering columns
-        //----------------------------------------------
-        createTable("CREATE TABLE %s (a int PRIMARY KEY, b int, c int) WITH COMPACT STORAGE");
-
-        execute("INSERT INTO %s (a, b, c) VALUES (1, 2, 4)");
-        execute("INSERT INTO %s (a, b, c) VALUES (2, 1, 6)");
-        execute("INSERT INTO %s (a, b, c) VALUES (3, 2, 4)");
-        execute("INSERT INTO %s (a, b, c) VALUES (4, 1, 7)");
-
-        // Adds tomstones
-        execute("INSERT INTO %s (a, b, c) VALUES (0, 1, 4)");
-        execute("INSERT INTO %s (a, b, c) VALUES (5, 2, 7)");
-        execute("DELETE FROM %s WHERE a = 0");
-        execute("DELETE FROM %s WHERE a = 5");
-
-        beforeAndAfterFlush(() -> {
-
-            // Checks filtering
-            assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                                 "SELECT * FROM %s WHERE a = 1 AND b = 4 AND c = 4");
-
-            assertRows(execute("SELECT * FROM %s WHERE a = 1 AND b = 2 AND c = 4 ALLOW FILTERING"),
-                       row(1, 2, 4));
-
-            assertInvalidMessage("IN predicates on non-primary-key columns (c) is not yet supported",
-                                 "SELECT * FROM %s WHERE a IN (1, 2) AND c IN (6, 7)");
-
-            assertInvalidMessage("IN predicates on non-primary-key columns (c) is not yet supported",
-                                 "SELECT * FROM %s WHERE a IN (1, 2) AND c IN (6, 7) ALLOW FILTERING");
-
-            assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                                 "SELECT * FROM %s WHERE c > 4");
-
-            assertRows(execute("SELECT * FROM %s WHERE c > 4 ALLOW FILTERING"),
-                       row(2, 1, 6),
-                       row(4, 1, 7));
-
-            assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                                 "SELECT * FROM %s WHERE b < 3 AND c <= 4");
-
-            assertRows(execute("SELECT * FROM %s WHERE b < 3 AND c <= 4 ALLOW FILTERING"),
-                       row(1, 2, 4),
-                       row(3, 2, 4));
-
-            assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                                 "SELECT * FROM %s WHERE c >= 3 AND c <= 6");
-
-            assertRows(execute("SELECT * FROM %s WHERE c >= 3 AND c <= 6 ALLOW FILTERING"),
-                       row(1, 2, 4),
-                       row(2, 1, 6),
-                       row(3, 2, 4));
-        });
-
-        // Checks filtering with null
-        assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                             "SELECT * FROM %s WHERE c = null");
-        assertInvalidMessage("Unsupported null value for column c",
-                             "SELECT * FROM %s WHERE c = null ALLOW FILTERING");
-        assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                             "SELECT * FROM %s WHERE c > null");
-        assertInvalidMessage("Unsupported null value for column c",
-                             "SELECT * FROM %s WHERE c > null ALLOW FILTERING");
-
-        // // Checks filtering with unset
-        assertInvalidMessage("Unsupported unset value for column c",
-                             "SELECT * FROM %s WHERE c = ? ALLOW FILTERING",
-                             unset());
-        assertInvalidMessage("Unsupported unset value for column c",
-                             "SELECT * FROM %s WHERE c > ? ALLOW FILTERING",
-                             unset());
-    }
-
-    @Test
     public void testFilteringWithoutIndicesWithCollections() throws Throwable
     {
         createTable("CREATE TABLE %s (a int, b int, c list<int>, d set<int>, e map<int, int>, PRIMARY KEY (a, b))");
@@ -1984,349 +1588,31 @@ public class SelectTest extends CQLTester
                              unset());
     }
 
+
     @Test
-    public void testFilteringOnCompactTablesWithoutIndicesAndWithLists() throws Throwable
+    public void testIndexQueryWithValueOver64K() throws Throwable
     {
-        //----------------------------------------------
-        // Test COMPACT table with clustering columns
-        //----------------------------------------------
-        createTable("CREATE TABLE %s (a int, b int, c frozen<list<int>>, PRIMARY KEY (a, b)) WITH COMPACT STORAGE");
-
-        execute("INSERT INTO %s (a, b, c) VALUES (1, 2, [4, 2])");
-        execute("INSERT INTO %s (a, b, c) VALUES (1, 3, [6, 2])");
-        execute("INSERT INTO %s (a, b, c) VALUES (1, 4, [4, 1])");
-        execute("INSERT INTO %s (a, b, c) VALUES (2, 3, [7, 1])");
-
-        beforeAndAfterFlush(() -> {
+        createTable("CREATE TABLE %s (a int, b int, c blob, PRIMARY KEY (a, b))");
+        String idx = createIndex("CREATE INDEX ON %s (c)");
 
-            // Checks filtering
-            assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                                 "SELECT * FROM %s WHERE a = 1 AND b = 4 AND c = [4, 1]");
+        execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, 0, bytes(1));
+        execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, 1, bytes(2));
 
-            assertRows(execute("SELECT * FROM %s WHERE a = 1 AND b = 4 AND c = [4, 1] ALLOW FILTERING"),
-                       row(1, 4, list(4, 1)));
+        assertInvalidMessage("Index expression values may not be larger than 64K",
+                             "SELECT * FROM %s WHERE c = ?  ALLOW FILTERING", TOO_BIG);
 
-            assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                                 "SELECT * FROM %s WHERE c > [4, 2]");
+        dropIndex("DROP INDEX %s." + idx);
+        assertEmpty(execute("SELECT * FROM %s WHERE c = ?  ALLOW FILTERING", TOO_BIG));
+    }
 
-            assertRows(execute("SELECT * FROM %s WHERE c > [4, 2] ALLOW FILTERING"),
-                       row(1, 3, list(6, 2)),
-                       row(2, 3, list(7, 1)));
+    @Test
+    public void testPKQueryWithValueOver64K() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a text, b text, PRIMARY KEY (a, b))");
 
-            assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                                 "SELECT * FROM %s WHERE b <= 3 AND c < [6, 2]");
-
-            assertRows(execute("SELECT * FROM %s WHERE b <= 3 AND c < [6, 2] ALLOW FILTERING"),
-                       row(1, 2, list(4, 2)));
-
-            assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                                 "SELECT * FROM %s WHERE c >= [4, 2] AND c <= [6, 4]");
-
-            assertRows(execute("SELECT * FROM %s WHERE c >= [4, 2] AND c <= [6, 4] ALLOW FILTERING"),
-                       row(1, 2, list(4, 2)),
-                       row(1, 3, list(6, 2)));
-
-            assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                                 "SELECT * FROM %s WHERE c CONTAINS 2");
-
-            assertRows(execute("SELECT * FROM %s WHERE c CONTAINS 2 ALLOW FILTERING"),
-                       row(1, 2, list(4, 2)),
-                       row(1, 3, list(6, 2)));
-
-            assertInvalidMessage("Cannot use CONTAINS KEY on non-map column c",
-                                 "SELECT * FROM %s WHERE c CONTAINS KEY 2 ALLOW FILTERING");
-
-            assertRows(execute("SELECT * FROM %s WHERE c CONTAINS 2 AND c CONTAINS 6 ALLOW FILTERING"),
-                       row(1, 3, list(6, 2)));
-        });
-
-        // Checks filtering with null
-        assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                             "SELECT * FROM %s WHERE c = null");
-        assertInvalidMessage("Unsupported null value for column c",
-                             "SELECT * FROM %s WHERE c = null ALLOW FILTERING");
-        assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                             "SELECT * FROM %s WHERE c > null");
-        assertInvalidMessage("Unsupported null value for column c",
-                             "SELECT * FROM %s WHERE c > null ALLOW FILTERING");
-        assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                             "SELECT * FROM %s WHERE c CONTAINS null");
-        assertInvalidMessage("Unsupported null value for column c",
-                             "SELECT * FROM %s WHERE c CONTAINS null ALLOW FILTERING");
-
-        // Checks filtering with unset
-        assertInvalidMessage("Unsupported unset value for column c",
-                             "SELECT * FROM %s WHERE c = ? ALLOW FILTERING",
-                             unset());
-        assertInvalidMessage("Unsupported unset value for column c",
-                             "SELECT * FROM %s WHERE c > ? ALLOW FILTERING",
-                             unset());
-        assertInvalidMessage("Unsupported unset value for column c",
-                             "SELECT * FROM %s WHERE c CONTAINS ? ALLOW FILTERING",
-                             unset());
-
-        //----------------------------------------------
-        // Test COMPACT table without clustering columns
-        //----------------------------------------------
-        createTable("CREATE TABLE %s (a int PRIMARY KEY, b int, c frozen<list<int>>) WITH COMPACT STORAGE");
-
-        execute("INSERT INTO %s (a, b, c) VALUES (1, 2, [4, 2])");
-        execute("INSERT INTO %s (a, b, c) VALUES (2, 1, [6, 2])");
-        execute("INSERT INTO %s (a, b, c) VALUES (3, 2, [4, 1])");
-        execute("INSERT INTO %s (a, b, c) VALUES (4, 1, [7, 1])");
-
-        beforeAndAfterFlush(() -> {
-
-            // Checks filtering
-            assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                                 "SELECT * FROM %s WHERE a = 1 AND b = 2 AND c = [4, 2]");
-
-            assertRows(execute("SELECT * FROM %s WHERE a = 1 AND b = 2 AND c = [4, 2] ALLOW FILTERING"),
-                       row(1, 2, list(4, 2)));
-
-            assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                                 "SELECT * FROM %s WHERE c > [4, 2]");
-
-            assertRows(execute("SELECT * FROM %s WHERE c > [4, 2] ALLOW FILTERING"),
-                       row(2, 1, list(6, 2)),
-                       row(4, 1, list(7, 1)));
-
-            assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                                 "SELECT * FROM %s WHERE b < 3 AND c <= [4, 2]");
-
-            assertRows(execute("SELECT * FROM %s WHERE b < 3 AND c <= [4, 2] ALLOW FILTERING"),
-                       row(1, 2, list(4, 2)),
-                       row(3, 2, list(4, 1)));
-
-            assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                                 "SELECT * FROM %s WHERE c >= [4, 3] AND c <= [7]");
-
-            assertRows(execute("SELECT * FROM %s WHERE c >= [4, 3] AND c <= [7] ALLOW FILTERING"),
-                       row(2, 1, list(6, 2)));
-
-            assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                                 "SELECT * FROM %s WHERE c CONTAINS 2");
-
-            assertRows(execute("SELECT * FROM %s WHERE c CONTAINS 2 ALLOW FILTERING"),
-                       row(1, 2, list(4, 2)),
-                       row(2, 1, list(6, 2)));
-
-            assertInvalidMessage("Cannot use CONTAINS KEY on non-map column c",
-                                 "SELECT * FROM %s WHERE c CONTAINS KEY 2 ALLOW FILTERING");
-
-            assertRows(execute("SELECT * FROM %s WHERE c CONTAINS 2 AND c CONTAINS 6 ALLOW FILTERING"),
-                       row(2, 1, list(6, 2)));
-        });
-
-        // Checks filtering with null
-        assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                             "SELECT * FROM %s WHERE c = null");
-        assertInvalidMessage("Unsupported null value for column c",
-                             "SELECT * FROM %s WHERE c = null ALLOW FILTERING");
-        assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                             "SELECT * FROM %s WHERE c > null");
-        assertInvalidMessage("Unsupported null value for column c",
-                             "SELECT * FROM %s WHERE c > null ALLOW FILTERING");
-        assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                             "SELECT * FROM %s WHERE c CONTAINS null");
-        assertInvalidMessage("Unsupported null value for column c",
-                             "SELECT * FROM %s WHERE c CONTAINS null ALLOW FILTERING");
-
-        // Checks filtering with unset
-        assertInvalidMessage("Unsupported unset value for column c",
-                             "SELECT * FROM %s WHERE c = ? ALLOW FILTERING",
-                             unset());
-        assertInvalidMessage("Unsupported unset value for column c",
-                             "SELECT * FROM %s WHERE c > ? ALLOW FILTERING",
-                             unset());
-        assertInvalidMessage("Unsupported unset value for column c",
-                             "SELECT * FROM %s WHERE c CONTAINS ? ALLOW FILTERING",
-                             unset());
-    }
-
-    @Test
-    public void testFilteringOnCompactTablesWithoutIndicesAndWithSets() throws Throwable
-    {
-        //----------------------------------------------
-        // Test COMPACT table with clustering columns
-        //----------------------------------------------
-        createTable("CREATE TABLE %s (a int, b int, c frozen<set<int>>, PRIMARY KEY (a, b)) WITH COMPACT STORAGE");
-
-        execute("INSERT INTO %s (a, b, c) VALUES (1, 2, {4, 2})");
-        execute("INSERT INTO %s (a, b, c) VALUES (1, 3, {6, 2})");
-        execute("INSERT INTO %s (a, b, c) VALUES (1, 4, {4, 1})");
-        execute("INSERT INTO %s (a, b, c) VALUES (2, 3, {7, 1})");
-
-        beforeAndAfterFlush(() -> {
-
-            // Checks filtering
-            assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                                 "SELECT * FROM %s WHERE a = 1 AND b = 4 AND c = {4, 1}");
-
-            assertRows(execute("SELECT * FROM %s WHERE a = 1 AND b = 4 AND c = {4, 1} ALLOW FILTERING"),
-                       row(1, 4, set(4, 1)));
-
-            assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                                 "SELECT * FROM %s WHERE c > {4, 2}");
-
-            assertRows(execute("SELECT * FROM %s WHERE c > {4, 2} ALLOW FILTERING"),
-                       row(1, 3, set(6, 2)));
-
-            assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                                 "SELECT * FROM %s WHERE b <= 3 AND c < {6, 2}");
-
-            assertRows(execute("SELECT * FROM %s WHERE b <= 3 AND c < {6, 2} ALLOW FILTERING"),
-                       row(1, 2, set(2, 4)),
-                       row(2, 3, set(1, 7)));
-
-            assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                                 "SELECT * FROM %s WHERE c >= {4, 2} AND c <= {6, 4}");
-
-            assertRows(execute("SELECT * FROM %s WHERE c >= {4, 2} AND c <= {6, 4} ALLOW FILTERING"),
-                       row(1, 2, set(4, 2)),
-                       row(1, 3, set(6, 2)));
-
-            assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                                 "SELECT * FROM %s WHERE c CONTAINS 2");
-
-            assertRows(execute("SELECT * FROM %s WHERE c CONTAINS 2 ALLOW FILTERING"),
-                       row(1, 2, set(4, 2)),
-                       row(1, 3, set(6, 2)));
-
-            assertInvalidMessage("Cannot use CONTAINS KEY on non-map column c",
-                                 "SELECT * FROM %s WHERE c CONTAINS KEY 2 ALLOW FILTERING");
-
-            assertRows(execute("SELECT * FROM %s WHERE c CONTAINS 2 AND c CONTAINS 6 ALLOW FILTERING"),
-                       row(1, 3, set(6, 2)));
-        });
-        // Checks filtering with null
-        assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                             "SELECT * FROM %s WHERE c = null");
-        assertInvalidMessage("Unsupported null value for column c",
-                             "SELECT * FROM %s WHERE c = null ALLOW FILTERING");
-        assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                             "SELECT * FROM %s WHERE c > null");
-        assertInvalidMessage("Unsupported null value for column c",
-                             "SELECT * FROM %s WHERE c > null ALLOW FILTERING");
-        assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                             "SELECT * FROM %s WHERE c CONTAINS null");
-        assertInvalidMessage("Unsupported null value for column c",
-                             "SELECT * FROM %s WHERE c CONTAINS null ALLOW FILTERING");
-
-        // Checks filtering with unset
-        assertInvalidMessage("Unsupported unset value for column c",
-                             "SELECT * FROM %s WHERE c = ? ALLOW FILTERING",
-                             unset());
-        assertInvalidMessage("Unsupported unset value for column c",
-                             "SELECT * FROM %s WHERE c > ? ALLOW FILTERING",
-                             unset());
-        assertInvalidMessage("Unsupported unset value for column c",
-                             "SELECT * FROM %s WHERE c CONTAINS ? ALLOW FILTERING",
-                             unset());
-
-        //----------------------------------------------
-        // Test COMPACT table without clustering columns
-        //----------------------------------------------
-        createTable("CREATE TABLE %s (a int PRIMARY KEY, b int, c frozen<set<int>>) WITH COMPACT STORAGE");
-
-        execute("INSERT INTO %s (a, b, c) VALUES (1, 2, {4, 2})");
-        execute("INSERT INTO %s (a, b, c) VALUES (2, 1, {6, 2})");
-        execute("INSERT INTO %s (a, b, c) VALUES (3, 2, {4, 1})");
-        execute("INSERT INTO %s (a, b, c) VALUES (4, 1, {7, 1})");
-
-        beforeAndAfterFlush(() -> {
-
-            // Checks filtering
-            assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                                 "SELECT * FROM %s WHERE a = 1 AND b = 2 AND c = {4, 2}");
-
-            assertRows(execute("SELECT * FROM %s WHERE a = 1 AND b = 2 AND c = {4, 2} ALLOW FILTERING"),
-                       row(1, 2, set(4, 2)));
-
-            assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                                 "SELECT * FROM %s WHERE c > {4, 2}");
-
-            assertRows(execute("SELECT * FROM %s WHERE c > {4, 2} ALLOW FILTERING"),
-                       row(2, 1, set(6, 2)));
-
-            assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                                 "SELECT * FROM %s WHERE b < 3 AND c <= {4, 2}");
-
-            assertRows(execute("SELECT * FROM %s WHERE b < 3 AND c <= {4, 2} ALLOW FILTERING"),
-                       row(1, 2, set(4, 2)),
-                       row(4, 1, set(1, 7)),
-                       row(3, 2, set(4, 1)));
-
-            assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                                 "SELECT * FROM %s WHERE c >= {4, 3} AND c <= {7}");
-
-            assertRows(execute("SELECT * FROM %s WHERE c >= {5, 2} AND c <= {7} ALLOW FILTERING"),
-                       row(2, 1, set(6, 2)));
-
-            assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                                 "SELECT * FROM %s WHERE c CONTAINS 2");
-
-            assertRows(execute("SELECT * FROM %s WHERE c CONTAINS 2 ALLOW FILTERING"),
-                       row(1, 2, set(4, 2)),
-                       row(2, 1, set(6, 2)));
-
-            assertInvalidMessage("Cannot use CONTAINS KEY on non-map column c",
-                                 "SELECT * FROM %s WHERE c CONTAINS KEY 2 ALLOW FILTERING");
-
-            assertRows(execute("SELECT * FROM %s WHERE c CONTAINS 2 AND c CONTAINS 6 ALLOW FILTERING"),
-                       row(2, 1, set(6, 2)));
-        });
-
-        // Checks filtering with null
-        assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                             "SELECT * FROM %s WHERE c = null");
-        assertInvalidMessage("Unsupported null value for column c",
-                             "SELECT * FROM %s WHERE c = null ALLOW FILTERING");
-        assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                             "SELECT * FROM %s WHERE c > null");
-        assertInvalidMessage("Unsupported null value for column c",
-                             "SELECT * FROM %s WHERE c > null ALLOW FILTERING");
-        assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                             "SELECT * FROM %s WHERE c CONTAINS null");
-        assertInvalidMessage("Unsupported null value for column c",
-                             "SELECT * FROM %s WHERE c CONTAINS null ALLOW FILTERING");
-
-        // Checks filtering with unset
-        assertInvalidMessage("Unsupported unset value for column c",
-                             "SELECT * FROM %s WHERE c = ? ALLOW FILTERING",
-                             unset());
-        assertInvalidMessage("Unsupported unset value for column c",
-                             "SELECT * FROM %s WHERE c > ? ALLOW FILTERING",
-                             unset());
-        assertInvalidMessage("Unsupported unset value for column c",
-                             "SELECT * FROM %s WHERE c CONTAINS ? ALLOW FILTERING",
-                             unset());
-    }
-
-    @Test
-    public void testIndexQueryWithValueOver64K() throws Throwable
-    {
-        String tableName = createTable("CREATE TABLE %s (a int, b int, c blob, PRIMARY KEY (a, b))");
-        String idx = createIndex("CREATE INDEX ON %s (c)");
-
-        execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, 0, bytes(1));
-        execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, 1, bytes(2));
-
-        assertInvalidMessage("Index expression values may not be larger than 64K",
-                             "SELECT * FROM %s WHERE c = ?  ALLOW FILTERING", TOO_BIG);
-
-        dropIndex("DROP INDEX %s." + idx);
-        assertEmpty(execute("SELECT * FROM %s WHERE c = ?  ALLOW FILTERING", TOO_BIG));
-    }
-
-    @Test
-    public void testPKQueryWithValueOver64K() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a text, b text, PRIMARY KEY (a, b))");
-
-        assertInvalidThrow(InvalidRequestException.class,
-                           "SELECT * FROM %s WHERE a = ?", new String(TOO_BIG.array()));
-    }
+        assertInvalidThrow(InvalidRequestException.class,
+                           "SELECT * FROM %s WHERE a = ?", new String(TOO_BIG.array()));
+    }
 
     @Test
     public void testCKQueryWithValueOver64K() throws Throwable
@@ -2369,41 +1655,6 @@ public class SelectTest extends CQLTester
             assertInvalidMessage("queries must only request partition key columns",
                     "SELECT DISTINCT pk0, pk1, ck0 FROM %s ALLOW FILTERING");
         });
-
-        // Test a 'compact storage' table.
-        createTable("CREATE TABLE %s (pk0 int, pk1 int, val int, PRIMARY KEY((pk0, pk1))) WITH COMPACT STORAGE");
-
-        for (int i = 0; i < 3; i++)
-            execute("INSERT INTO %s (pk0, pk1, val) VALUES (?, ?, ?)", i, i, i);
-
-        beforeAndAfterFlush(() -> {
-            assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                                 "SELECT DISTINCT pk0, pk1 FROM %s WHERE pk1 = 1 LIMIT 3");
-
-            assertRows(execute("SELECT DISTINCT pk0, pk1 FROM %s WHERE pk0 < 2 AND pk1 = 1 LIMIT 1 ALLOW FILTERING"),
-                    row(1, 1));
-
-            assertRows(execute("SELECT DISTINCT pk0, pk1 FROM %s WHERE pk1 > 1 LIMIT 3 ALLOW FILTERING"),
-                    row(2, 2));
-        });
-
-        // Test a 'wide row' thrift table.
-        createTable("CREATE TABLE %s (pk int, name text, val int, PRIMARY KEY(pk, name)) WITH COMPACT STORAGE");
-
-        for (int i = 0; i < 3; i++)
-        {
-            execute("INSERT INTO %s (pk, name, val) VALUES (?, 'name0', 0)", i);
-            execute("INSERT INTO %s (pk, name, val) VALUES (?, 'name1', 1)", i);
-        }
-
-        beforeAndAfterFlush(() -> {
-            assertRows(execute("SELECT DISTINCT pk FROM %s WHERE pk > 1 LIMIT 1 ALLOW FILTERING"),
-                    row(2));
-
-            assertRows(execute("SELECT DISTINCT pk FROM %s WHERE pk > 0 LIMIT 3 ALLOW FILTERING"),
-                    row(1),
-                    row(2));
-        });
     }
 
     @Test
@@ -2651,940 +1902,111 @@ public class SelectTest extends CQLTester
                     execute("SELECT * FROM %s WHERE a > 0 AND b <= 3 AND c CONTAINS 2 AND c CONTAINS 3 ALLOW FILTERING"),
                        row(1, 3, list(3, 2), set(6, 4), map(3, 2)));
 
-            // Checks filtering for sets
-            assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                    "SELECT * FROM %s WHERE a = 1 AND d CONTAINS 4");
-
-            assertRows(execute("SELECT * FROM %s WHERE d CONTAINS 4 ALLOW FILTERING"),
-                       row(1, 3, list(3, 2), set(6, 4), map(3, 2)),
-                       row(1, 4, list(1, 2), set(2, 4), map(1, 2)));
-
-            assertRows(execute("SELECT * FROM %s WHERE d CONTAINS 4 AND d CONTAINS 6 ALLOW FILTERING"),
-                       row(1, 3, list(3, 2), set(6, 4), map(3, 2)));
-
-            // Checks filtering for maps
-            assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                                 "SELECT * FROM %s WHERE e CONTAINS 2");
-
-            assertRows(execute("SELECT * FROM %s WHERE a < 2 AND b >= 3 AND e CONTAINS 2 ALLOW FILTERING"),
-                       row(1, 3, list(3, 2), set(6, 4), map(3, 2)),
-                       row(1, 4, list(1, 2), set(2, 4), map(1, 2)));
-
-            assertRows(execute("SELECT * FROM %s WHERE a = 1 AND e CONTAINS KEY 1 ALLOW FILTERING"),
-                       row(1, 4, list(1, 2), set(2, 4), map(1, 2)),
-                       row(1, 2, list(1, 6), set(2, 12), map(1, 6)));
-
-            assertRows(execute("SELECT * FROM %s WHERE a in (1) AND b in (2) AND e[1] = 6 ALLOW FILTERING"),
-                       row(1, 2, list(1, 6), set(2, 12), map(1, 6)));
-
-            assertRows(execute("SELECT * FROM %s WHERE a = 1 AND e CONTAINS KEY 1 AND e CONTAINS 2 ALLOW FILTERING"),
-                       row(1, 4, list(1, 2), set(2, 4), map(1, 2)));
-
-            assertRows(
-                    execute("SELECT * FROM %s WHERE a >= 1 AND b in (3) AND c CONTAINS 2 AND d CONTAINS 4 AND e CONTAINS KEY 3 ALLOW FILTERING"),
-                       row(1, 3, list(3, 2), set(6, 4), map(3, 2)));
-        });
-
-        // Checks filtering with null
-        assertInvalidMessage("Unsupported null value for column c",
-                             "SELECT * FROM %s WHERE a > 1 AND c CONTAINS null ALLOW FILTERING");
-        assertInvalidMessage("Unsupported null value for column d",
-                             "SELECT * FROM %s WHERE b < 1 AND d CONTAINS null ALLOW FILTERING");
-        assertInvalidMessage("Unsupported null value for column e",
-                             "SELECT * FROM %s WHERE a >= 1 AND b < 1 AND e CONTAINS null ALLOW FILTERING");
-        assertInvalidMessage("Unsupported null value for column e",
-                             "SELECT * FROM %s WHERE a >= 1 AND b < 1 AND e CONTAINS KEY null ALLOW FILTERING");
-        assertInvalidMessage("Unsupported null map key for column e",
-                             "SELECT * FROM %s WHERE a >= 1 AND b < 1 AND e[null] = 2 ALLOW FILTERING");
-        assertInvalidMessage("Unsupported null map value for column e",
-                             "SELECT * FROM %s WHERE a >= 1 AND b < 1 AND e[1] = null ALLOW FILTERING");
-
-        // Checks filtering with unset
-        assertInvalidMessage("Unsupported unset value for column c",
-                             "SELECT * FROM %s WHERE a >= 1 AND b < 1 AND c CONTAINS ? ALLOW FILTERING",
-                             unset());
-        assertInvalidMessage("Unsupported unset value for column d",
-                             "SELECT * FROM %s WHERE a >= 1 AND b < 1 AND d CONTAINS ? ALLOW FILTERING",
-                             unset());
-        assertInvalidMessage("Unsupported unset value for column e",
-                             "SELECT * FROM %s WHERE a >= 1 AND b < 1 AND e CONTAINS ? ALLOW FILTERING",
-                             unset());
-        assertInvalidMessage("Unsupported unset value for column e",
-                             "SELECT * FROM %s WHERE a >= 1 AND b < 1 AND e CONTAINS KEY ? ALLOW FILTERING",
-                             unset());
-        assertInvalidMessage("Unsupported unset map key for column e",
-                             "SELECT * FROM %s WHERE a >= 1 AND b < 1 AND e[?] = 2 ALLOW FILTERING",
-                             unset());
-        assertInvalidMessage("Unsupported unset map value for column e",
-                             "SELECT * FROM %s WHERE a >= 1 AND b < 1 AND e[1] = ? ALLOW FILTERING",
-                             unset());
-    }
-
-    @Test
-    public void testAllowFilteringOnPartitionKeyWithCounters() throws Throwable
-    {
-        for (String compactStorageClause : new String[] { "", " WITH COMPACT STORAGE" })
-        {
-            createTable("CREATE TABLE %s (a int, b int, c int, cnt counter, PRIMARY KEY ((a, b), c))"
-                    + compactStorageClause);
-
-            execute("UPDATE %s SET cnt = cnt + ? WHERE a = ? AND b = ? AND c = ?", 14L, 11, 12, 13);
-            execute("UPDATE %s SET cnt = cnt + ? WHERE a = ? AND b = ? AND c = ?", 24L, 21, 22, 23);
-            execute("UPDATE %s SET cnt = cnt + ? WHERE a = ? AND b = ? AND c = ?", 27L, 21, 22, 26);
-            execute("UPDATE %s SET cnt = cnt + ? WHERE a = ? AND b = ? AND c = ?", 34L, 31, 32, 33);
-            execute("UPDATE %s SET cnt = cnt + ? WHERE a = ? AND b = ? AND c = ?", 24L, 41, 42, 43);
-
-            beforeAndAfterFlush(() -> {
-
-                assertRows(executeFilteringOnly("SELECT * FROM %s WHERE cnt = 24"),
-                        row(41, 42, 43, 24L),
-                        row(21, 22, 23, 24L));
-                assertRows(executeFilteringOnly("SELECT * FROM %s WHERE b > 22 AND cnt = 24"),
-                        row(41, 42, 43, 24L));
-                assertRows(executeFilteringOnly("SELECT * FROM %s WHERE b > 10 AND b < 25 AND cnt = 24"),
-                        row(21, 22, 23, 24L));
-                assertRows(executeFilteringOnly("SELECT * FROM %s WHERE b > 10 AND c < 25 AND cnt = 24"),
-                        row(21, 22, 23, 24L));
-
-                assertInvalidMessage(
-                        "ORDER BY is only supported when the partition key is restricted by an EQ or an IN.",
-                        "SELECT * FROM %s WHERE a = 21 AND b > 10 AND cnt > 23 ORDER BY c DESC ALLOW FILTERING");
-
-                assertRows(executeFilteringOnly("SELECT * FROM %s WHERE a = 21 AND b = 22 AND cnt > 23 ORDER BY c DESC"),
-                        row(21, 22, 26, 27L),
-                        row(21, 22, 23, 24L));
-
-                assertRows(executeFilteringOnly("SELECT * FROM %s WHERE cnt > 20 AND cnt < 30"),
-                        row(41, 42, 43, 24L),
-                        row(21, 22, 23, 24L),
-                        row(21, 22, 26, 27L));
-            });
-        }
-    }
-
-    @Test
-    public void testAllowFilteringOnPartitionKeyOnCompactTablesWithoutIndicesAndWithLists() throws Throwable
-    {
-        // ----------------------------------------------
-        // Test COMPACT table with clustering columns
-        // ----------------------------------------------
-        createTable("CREATE TABLE %s (a int, b int, c frozen<list<int>>, PRIMARY KEY (a, b)) WITH COMPACT STORAGE");
-
-        execute("INSERT INTO %s (a, b, c) VALUES (1, 2, [4, 2])");
-        execute("INSERT INTO %s (a, b, c) VALUES (1, 3, [6, 2])");
-        execute("INSERT INTO %s (a, b, c) VALUES (1, 4, [4, 1])");
-        execute("INSERT INTO %s (a, b, c) VALUES (2, 3, [7, 1])");
-
-        beforeAndAfterFlush(() -> {
-
-            // Checks filtering
-            assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                    "SELECT * FROM %s WHERE a >= 1 AND b = 4 AND c = [4, 1]");
-
-            assertRows(execute("SELECT * FROM %s WHERE a >= 1 AND b >= 4 AND c = [4, 1] ALLOW FILTERING"),
-                    row(1, 4, list(4, 1)));
-
-            assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                    "SELECT * FROM %s WHERE a > 0 AND c > [4, 2]");
-
-            assertRows(execute("SELECT * FROM %s WHERE a >= 1 AND c > [4, 2] ALLOW FILTERING"),
-                    row(1, 3, list(6, 2)),
-                    row(2, 3, list(7, 1)));
-
-            assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                    "SELECT * FROM %s WHERE a > 1 AND b <= 3 AND c < [6, 2]");
-
-            assertRows(execute("SELECT * FROM %s WHERE a <= 1 AND b <= 3 AND c < [6, 2] ALLOW FILTERING"),
-                    row(1, 2, list(4, 2)));
-
-            assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                    "SELECT * FROM %s WHERE a <= 1 AND c >= [4, 2] AND c <= [6, 4]");
-
-            assertRows(execute("SELECT * FROM %s WHERE a > 0 AND b <= 3 AND c >= [4, 2] AND c <= [6, 4] ALLOW FILTERING"),
-                    row(1, 2, list(4, 2)),
-                    row(1, 3, list(6, 2)));
-
-            assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                    "SELECT * FROM %s WHERE a > 1 AND c CONTAINS 2");
-
-            assertRows(execute("SELECT * FROM %s WHERE a > 0 AND c CONTAINS 2 ALLOW FILTERING"),
-                    row(1, 2, list(4, 2)),
-                    row(1, 3, list(6, 2)));
-
-            assertInvalidMessage("Cannot use CONTAINS KEY on non-map column c",
-                    "SELECT * FROM %s WHERE a > 1 AND c CONTAINS KEY 2 ALLOW FILTERING");
-
-            assertRows(execute("SELECT * FROM %s WHERE a < 2 AND c CONTAINS 2 AND c CONTAINS 6 ALLOW FILTERING"),
-                    row(1, 3, list(6, 2)));
-        });
-        // Checks filtering with null
-        assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                "SELECT * FROM %s WHERE a > 1 AND c = null");
-        assertInvalidMessage("Unsupported null value for column c",
-                "SELECT * FROM %s WHERE a > 1 AND c = null ALLOW FILTERING");
-        assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                "SELECT * FROM %s WHERE a > 1 AND c > null");
-        assertInvalidMessage("Unsupported null value for column c",
-                "SELECT * FROM %s WHERE a > 1 AND c > null ALLOW FILTERING");
-        assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                "SELECT * FROM %s WHERE a > 1 AND c CONTAINS null");
-        assertInvalidMessage("Unsupported null value for column c",
-                "SELECT * FROM %s WHERE a > 1 AND c CONTAINS null ALLOW FILTERING");
-
-        // Checks filtering with unset
-        assertInvalidMessage("Unsupported unset value for column c",
-                "SELECT * FROM %s WHERE a > 1 AND c = ? ALLOW FILTERING",
-                unset());
-        assertInvalidMessage("Unsupported unset value for column c",
-                "SELECT * FROM %s WHERE a > 1 AND c > ? ALLOW FILTERING",
-                unset());
-        assertInvalidMessage("Unsupported unset value for column c",
-                "SELECT * FROM %s WHERE a > 1 AND c CONTAINS ? ALLOW FILTERING",
-                unset());
-
-        // ----------------------------------------------
-        // Test COMPACT table without clustering columns
-        // ----------------------------------------------
-        createTable("CREATE TABLE %s (a int PRIMARY KEY, b int, c frozen<list<int>>) WITH COMPACT STORAGE");
-
-        execute("INSERT INTO %s (a, b, c) VALUES (1, 2, [4, 2])");
-        execute("INSERT INTO %s (a, b, c) VALUES (2, 1, [6, 2])");
-        execute("INSERT INTO %s (a, b, c) VALUES (3, 2, [4, 1])");
-        execute("INSERT INTO %s (a, b, c) VALUES (4, 1, [7, 1])");
-
-        beforeAndAfterFlush(() -> {
-
-            // Checks filtering
-            assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                    "SELECT * FROM %s WHERE a >= 1 AND b = 2 AND c = [4, 2]");
-
-            assertRows(execute("SELECT * FROM %s WHERE a >= 1 AND b = 2 AND c = [4, 2] ALLOW FILTERING"),
-                    row(1, 2, list(4, 2)));
-
-            assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                    "SELECT * FROM %s WHERE a > 1 AND c > [4, 2]");
-
-            assertRows(execute("SELECT * FROM %s WHERE a > 3 AND c > [4, 2] ALLOW FILTERING"),
-                    row(4, 1, list(7, 1)));
-
-            assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                    "SELECT * FROM %s WHERE a < 1 AND b < 3 AND c <= [4, 2]");
-
-            assertRows(execute("SELECT * FROM %s WHERE a < 3 AND b < 3 AND c <= [4, 2] ALLOW FILTERING"),
-                    row(1, 2, list(4, 2)));
-
-            assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                    "SELECT * FROM %s WHERE a > 1 AND c >= [4, 3] AND c <= [7]");
-
-            assertRows(execute("SELECT * FROM %s WHERE a >= 2 AND c >= [4, 3] AND c <= [7] ALLOW FILTERING"),
-                    row(2, 1, list(6, 2)));
-
-            assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                    "SELECT * FROM %s WHERE a > 3 AND c CONTAINS 2");
-
-            assertRows(execute("SELECT * FROM %s WHERE a >= 1 AND c CONTAINS 2 ALLOW FILTERING"),
-                    row(1, 2, list(4, 2)),
-                    row(2, 1, list(6, 2)));
-
-            assertInvalidMessage("Cannot use CONTAINS KEY on non-map column c",
-                    "SELECT * FROM %s WHERE a >=1 AND c CONTAINS KEY 2 ALLOW FILTERING");
-
-            assertRows(execute("SELECT * FROM %s WHERE a < 3 AND c CONTAINS 2 AND c CONTAINS 6 ALLOW FILTERING"),
-                    row(2, 1, list(6, 2)));
-        });
-
-        // Checks filtering with null
-        assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                "SELECT * FROM %s WHERE a > 1 AND c = null");
-        assertInvalidMessage("Unsupported null value for column c",
-                "SELECT * FROM %s WHERE a > 1 AND c = null ALLOW FILTERING");
-        assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                "SELECT * FROM %s WHERE a > 1 AND c > null");
-        assertInvalidMessage("Unsupported null value for column c",
-                "SELECT * FROM %s WHERE a > 1 AND c > null ALLOW FILTERING");
-        assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                "SELECT * FROM %s WHERE a > 1 AND c CONTAINS null");
-        assertInvalidMessage("Unsupported null value for column c",
-                "SELECT * FROM %s WHERE a > 1 AND c CONTAINS null ALLOW FILTERING");
-
-        // Checks filtering with unset
-        assertInvalidMessage("Unsupported unset value for column c",
-                "SELECT * FROM %s WHERE a > 1 AND c = ? ALLOW FILTERING",
-                unset());
-        assertInvalidMessage("Unsupported unset value for column c",
-                "SELECT * FROM %s WHERE a > 1 AND c > ? ALLOW FILTERING",
-                unset());
-        assertInvalidMessage("Unsupported unset value for column c",
-                "SELECT * FROM %s WHERE a > 1 AND c CONTAINS ? ALLOW FILTERING",
-                unset());
-    }
-
-
-    @Test
-    public void testAllowFilteringOnPartitionKeyOnCompactTablesWithoutIndicesAndWithMaps() throws Throwable
-    {
-        //----------------------------------------------
-        // Test COMPACT table with clustering columns
-        //----------------------------------------------
-        createTable("CREATE TABLE %s (a int, b int, c frozen<map<int, int>>, PRIMARY KEY (a, b)) WITH COMPACT STORAGE");
-
-        execute("INSERT INTO %s (a, b, c) VALUES (1, 2, {4 : 2})");
-        execute("INSERT INTO %s (a, b, c) VALUES (1, 3, {6 : 2})");
-        execute("INSERT INTO %s (a, b, c) VALUES (1, 4, {4 : 1})");
-        execute("INSERT INTO %s (a, b, c) VALUES (2, 3, {7 : 1})");
-
-        beforeAndAfterFlush(() -> {
-
-            // Checks filtering
-            assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                                 "SELECT * FROM %s WHERE a >= 1 AND b = 4 AND c = {4 : 1}");
-
-            assertRows(execute("SELECT * FROM %s WHERE a <= 1 AND b = 4 AND c = {4 : 1} ALLOW FILTERING"),
-                       row(1, 4, map(4, 1)));
-
-            assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                                 "SELECT * FROM %s WHERE a > 1 AND c > {4 : 2}");
-
-            assertRows(execute("SELECT * FROM %s WHERE a > 1 AND c > {4 : 2} ALLOW FILTERING"),
-                       row(2, 3, map(7, 1)));
-
-            assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                                 "SELECT * FROM %s WHERE a > 1 AND b <= 3 AND c < {6 : 2}");
-
-            assertRows(execute("SELECT * FROM %s WHERE a >= 1 AND b <= 3 AND c < {6 : 2} ALLOW FILTERING"),
-                       row(1, 2, map(4, 2)));
-
-            assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                                 "SELECT * FROM %s WHERE a > 1 AND c >= {4 : 2} AND c <= {6 : 4}");
-
-            assertRows(execute("SELECT * FROM %s WHERE a > 0 AND c >= {4 : 2} AND c <= {6 : 4} ALLOW FILTERING"),
-                       row(1, 2, map(4, 2)),
-                       row(1, 3, map(6, 2)));
-
-            assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                                 "SELECT * FROM %s WHERE a > 10 AND c CONTAINS 2");
-
-            assertRows(execute("SELECT * FROM %s WHERE a > 0 AND c CONTAINS 2 ALLOW FILTERING"),
-                       row(1, 2, map(4, 2)),
-                       row(1, 3, map(6, 2)));
-
-            assertRows(execute("SELECT * FROM %s WHERE a < 2 AND c CONTAINS KEY 6 ALLOW FILTERING"),
-                       row(1, 3, map(6, 2)));
-
-            assertRows(execute("SELECT * FROM %s WHERE a >= 1 AND c CONTAINS 2 AND c CONTAINS KEY 6 ALLOW FILTERING"),
-                       row(1, 3, map(6, 2)));
-        });
-
-        // Checks filtering with null
-        assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                             "SELECT * FROM %s WHERE a >= 1 AND c = null");
-        assertInvalidMessage("Unsupported null value for column c",
-                             "SELECT * FROM %s WHERE a >= 1 AND c = null ALLOW FILTERING");
-        assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                             "SELECT * FROM %s WHERE a >= 1 AND c > null");
-        assertInvalidMessage("Unsupported null value for column c",
-                             "SELECT * FROM %s WHERE a >= 1 AND c > null ALLOW FILTERING");
-        assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                             "SELECT * FROM %s WHERE a >= 1 AND c CONTAINS null");
-        assertInvalidMessage("Unsupported null value for column c",
-                             "SELECT * FROM %s WHERE a >= 1 AND c CONTAINS null ALLOW FILTERING");
-        assertInvalidMessage("Unsupported null value for column c",
-                             "SELECT * FROM %s WHERE a >= 1 AND c CONTAINS KEY null ALLOW FILTERING");
-
-        // Checks filtering with unset
-        assertInvalidMessage("Unsupported unset value for column c",
-                             "SELECT * FROM %s WHERE a >= 1 AND c = ? ALLOW FILTERING",
-                             unset());
-        assertInvalidMessage("Unsupported unset value for column c",
-                             "SELECT * FROM %s WHERE a >= 1 AND c > ? ALLOW FILTERING",
-                             unset());
-        assertInvalidMessage("Unsupported unset value for column c",
-                             "SELECT * FROM %s WHERE a >= 1 AND c CONTAINS ? ALLOW FILTERING",
-                             unset());
-        assertInvalidMessage("Unsupported unset value for column c",
-                             "SELECT * FROM %s WHERE a >= 1 AND c CONTAINS KEY ? ALLOW FILTERING",
-                             unset());
-
-        //----------------------------------------------
-        // Test COMPACT table without clustering columns
-        //----------------------------------------------
-        createTable("CREATE TABLE %s (a int PRIMARY KEY, b int, c frozen<map<int, int>>) WITH COMPACT STORAGE");
-
-        execute("INSERT INTO %s (a, b, c) VALUES (1, 2, {4 : 2})");
-        execute("INSERT INTO %s (a, b, c) VALUES (2, 1, {6 : 2})");
-        execute("INSERT INTO %s (a, b, c) VALUES (3, 2, {4 : 1})");
-        execute("INSERT INTO %s (a, b, c) VALUES (4, 1, {7 : 1})");
-
-        beforeAndAfterFlush(() -> {
-
-            // Checks filtering
-            assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                                 "SELECT * FROM %s WHERE a >= 1 AND b = 2 AND c = {4 : 2}");
-
-            assertRows(execute("SELECT * FROM %s WHERE a >= 1 AND b = 2 AND c = {4 : 2} ALLOW FILTERING"),
-                       row(1, 2, map(4, 2)));
-
-            assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                                 "SELECT * FROM %s WHERE a >= 1 AND c > {4 : 2}");
-
-            assertRows(execute("SELECT * FROM %s WHERE a >= 1 AND c > {4 : 2} ALLOW FILTERING"),
-                       row(2, 1, map(6, 2)),
-                       row(4, 1, map(7, 1)));
-
-            assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                                 "SELECT * FROM %s WHERE a >= 1 AND b < 3 AND c <= {4 : 2}");
-
-            assertRows(execute("SELECT * FROM %s WHERE a >= 1 AND b < 3 AND c <= {4 : 2} ALLOW FILTERING"),
-                       row(1, 2, map(4, 2)),
-                       row(3, 2, map(4, 1)));
-
-            assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                                 "SELECT * FROM %s WHERE a >= 1 AND c >= {4 : 3} AND c <= {7 : 1}");
-
-            assertRows(execute("SELECT * FROM %s WHERE a >= 2 AND c >= {5 : 2} AND c <= {7 : 0} ALLOW FILTERING"),
-                       row(2, 1, map(6, 2)));
-
-            assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                                 "SELECT * FROM %s WHERE a >= 1 AND c CONTAINS 2");
-
-            assertRows(execute("SELECT * FROM %s WHERE a >= 1 AND c CONTAINS 2 ALLOW FILTERING"),
-                       row(1, 2, map(4, 2)),
-                       row(2, 1, map(6, 2)));
-
-            assertRows(execute("SELECT * FROM %s WHERE a > 0 AND c CONTAINS KEY 4 ALLOW FILTERING"),
-                       row(1, 2, map(4, 2)),
-                       row(3, 2, map(4, 1)));
-
-            assertRows(execute("SELECT * FROM %s WHERE a >= 2 AND c CONTAINS 2 AND c CONTAINS KEY 6 ALLOW FILTERING"),
-                       row(2, 1, map(6, 2)));
-        });
-
-        // Checks filtering with null
-        assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                             "SELECT * FROM %s WHERE a >= 1 AND c = null");
-        assertInvalidMessage("Unsupported null value for column c",
-                             "SELECT * FROM %s WHERE a >= 1 AND c = null ALLOW FILTERING");
-        assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                             "SELECT * FROM %s WHERE a >= 1 AND c > null");
-        assertInvalidMessage("Unsupported null value for column c",
-                             "SELECT * FROM %s WHERE a >= 1 AND c > null ALLOW FILTERING");
-        assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                             "SELECT * FROM %s WHERE a >= 1 AND c CONTAINS null");
-        assertInvalidMessage("Unsupported null value for column c",
-                             "SELECT * FROM %s WHERE a >= 1 AND c CONTAINS null ALLOW FILTERING");
-        assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                             "SELECT * FROM %s WHERE a >= 1 AND c CONTAINS KEY null");
-        assertInvalidMessage("Unsupported null value for column c",
-                             "SELECT * FROM %s WHERE a >= 1 AND c CONTAINS KEY null ALLOW FILTERING");
-
-        // Checks filtering with unset
-        assertInvalidMessage("Unsupported unset value for column c",
-                             "SELECT * FROM %s WHERE a >= 1 AND c = ? ALLOW FILTERING",
-                             unset());
-        assertInvalidMessage("Unsupported unset value for column c",
-                             "SELECT * FROM %s WHERE a >= 1 AND c > ? ALLOW FILTERING",
-                             unset());
-        assertInvalidMessage("Unsupported unset value for column c",
-                             "SELECT * FROM %s WHERE a >= 1 AND c CONTAINS ? ALLOW FILTERING",
-                             unset());
-        assertInvalidMessage("Unsupported unset value for column c",
-                             "SELECT * FROM %s WHERE a >= 1 AND c CONTAINS KEY ? ALLOW FILTERING",
-                             unset());
-    }
-
-    @Test
-    public void testAllowFilteringOnPartitionKeyOnCompactTablesWithoutIndicesAndWithSets() throws Throwable
-    {
-        //----------------------------------------------
-        // Test COMPACT table with clustering columns
-        //----------------------------------------------
-        createTable("CREATE TABLE %s (a int, b int, c frozen<set<int>>, PRIMARY KEY (a, b)) WITH COMPACT STORAGE");
-
-        execute("INSERT INTO %s (a, b, c) VALUES (1, 2, {4, 2})");
-        execute("INSERT INTO %s (a, b, c) VALUES (1, 3, {6, 2})");
-        execute("INSERT INTO %s (a, b, c) VALUES (1, 4, {4, 1})");
-        execute("INSERT INTO %s (a, b, c) VALUES (2, 3, {7, 1})");
-
-        beforeAndAfterFlush(() -> {
-
-            // Checks filtering
-            assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                                 "SELECT * FROM %s WHERE a >= 1 AND b = 4 AND c = {4, 1}");
-
-            assertRows(execute("SELECT * FROM %s WHERE a >= 1 AND b = 4 AND c = {4, 1} ALLOW FILTERING"),
-                       row(1, 4, set(4, 1)));
-
-            assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                                 "SELECT * FROM %s WHERE a >= 1 AND c > {4, 2}");
-
-            assertRows(execute("SELECT * FROM %s WHERE a >= 1 AND c > {4, 2} ALLOW FILTERING"),
-                       row(1, 3, set(6, 2)));
-
-            assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                                 "SELECT * FROM %s WHERE a >= 1 AND b <= 3 AND c < {6, 2}");
-
-            assertRows(execute("SELECT * FROM %s WHERE a > 0 AND b <= 3 AND c < {6, 2} ALLOW FILTERING"),
-                       row(1, 2, set(2, 4)),
-                       row(2, 3, set(1, 7)));
-
-            assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                                 "SELECT * FROM %s WHERE a >= 1 AND c >= {4, 2} AND c <= {6, 4}");
-
-            assertRows(execute("SELECT * FROM %s WHERE a >= 0 AND c >= {4, 2} AND c <= {6, 4} ALLOW FILTERING"),
-                       row(1, 2, set(4, 2)),
-                       row(1, 3, set(6, 2)));
-
-            assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                                 "SELECT * FROM %s WHERE a >= 1 AND c CONTAINS 2");
-
-            assertRows(execute("SELECT * FROM %s WHERE a < 2 AND c CONTAINS 2 ALLOW FILTERING"),
-                       row(1, 2, set(4, 2)),
-                       row(1, 3, set(6, 2)));
-
-            assertInvalidMessage("Cannot use CONTAINS KEY on non-map column c",
-                                 "SELECT * FROM %s WHERE a >= 1 AND c CONTAINS KEY 2 ALLOW FILTERING");
-
-            assertRows(execute("SELECT * FROM %s WHERE a >= 1 AND c CONTAINS 2 AND c CONTAINS 6 ALLOW FILTERING"),
-                       row(1, 3, set(6, 2)));
-        });
-        // Checks filtering with null
-        assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                             "SELECT * FROM %s WHERE a >= 1 AND c = null");
-        assertInvalidMessage("Unsupported null value for column c",
-                             "SELECT * FROM %s WHERE a >= 1 AND c = null ALLOW FILTERING");
-        assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                             "SELECT * FROM %s WHERE a >= 1 AND c > null");
-        assertInvalidMessage("Unsupported null value for column c",
-                             "SELECT * FROM %s WHERE a >= 1 AND c > null ALLOW FILTERING");
-        assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                             "SELECT * FROM %s WHERE a >= 1 AND c CONTAINS null");
-        assertInvalidMessage("Unsupported null value for column c",
-                             "SELECT * FROM %s WHERE a >= 1 AND c CONTAINS null ALLOW FILTERING");
-
-        // Checks filtering with unset
-        assertInvalidMessage("Unsupported unset value for column c",
-                             "SELECT * FROM %s WHERE a >= 1 AND c = ? ALLOW FILTERING",
-                             unset());
-        assertInvalidMessage("Unsupported unset value for column c",
-                             "SELECT * FROM %s WHERE a >= 1 AND c > ? ALLOW FILTERING",
-                             unset());
-        assertInvalidMessage("Unsupported unset value for column c",
-                             "SELECT * FROM %s WHERE a >= 1 AND c CONTAINS ? ALLOW FILTERING",
-                             unset());
-
-        //----------------------------------------------
-        // Test COMPACT table without clustering columns
-        //----------------------------------------------
-        createTable("CREATE TABLE %s (a int PRIMARY KEY, b int, c frozen<set<int>>) WITH COMPACT STORAGE");
-
-        execute("INSERT INTO %s (a, b, c) VALUES (1, 2, {4, 2})");
-        execute("INSERT INTO %s (a, b, c) VALUES (2, 1, {6, 2})");
-        execute("INSERT INTO %s (a, b, c) VALUES (3, 2, {4, 1})");
-        execute("INSERT INTO %s (a, b, c) VALUES (4, 1, {7, 1})");
-
-        beforeAndAfterFlush(() -> {
-
-            // Checks filtering
-            assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                                 "SELECT * FROM %s WHERE a >= 1 AND b = 2 AND c = {4, 2}");
-
-            assertRows(execute("SELECT * FROM %s WHERE a >= 1 AND b = 2 AND c = {4, 2} ALLOW FILTERING"),
-                       row(1, 2, set(4, 2)));
-
-            assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                                 "SELECT * FROM %s WHERE a >= 1 AND c > {4, 2}");
-
-            assertRows(execute("SELECT * FROM %s WHERE a >= 1 AND c > {4, 2} ALLOW FILTERING"),
-                       row(2, 1, set(6, 2)));
-
-            assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                                 "SELECT * FROM %s WHERE a >= 1 AND b < 3 AND c <= {4, 2}");
-
-            assertRows(execute("SELECT * FROM %s WHERE a <= 4 AND b < 3 AND c <= {4, 2} ALLOW FILTERING"),
-                       row(1, 2, set(4, 2)),
-                       row(4, 1, set(1, 7)),
-                       row(3, 2, set(4, 1)));
-
-            assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                                 "SELECT * FROM %s WHERE a >= 1 AND c >= {4, 3} AND c <= {7}");
-
-            assertRows(execute("SELECT * FROM %s WHERE a < 3 AND c >= {5, 2} AND c <= {7} ALLOW FILTERING"),
-                       row(2, 1, set(6, 2)));
-
-            assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                                 "SELECT * FROM %s WHERE a >= 1 AND c CONTAINS 2");
-
-            assertRows(execute("SELECT * FROM %s WHERE a >= 0 AND c CONTAINS 2 ALLOW FILTERING"),
-                       row(1, 2, set(4, 2)),
-                       row(2, 1, set(6, 2)));
-
-            assertInvalidMessage("Cannot use CONTAINS KEY on non-map column c",
-                                 "SELECT * FROM %s WHERE a >= 1 AND c CONTAINS KEY 2 ALLOW FILTERING");
-
-            assertRows(execute("SELECT * FROM %s WHERE a >= 2 AND c CONTAINS 2 AND c CONTAINS 6 ALLOW FILTERING"),
-                       row(2, 1, set(6, 2)));
-        });
-
-        // Checks filtering with null
-        assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                             "SELECT * FROM %s WHERE a >= 1 AND c = null");
-        assertInvalidMessage("Unsupported null value for column c",
-                             "SELECT * FROM %s WHERE a >= 1 AND c = null ALLOW FILTERING");
-        assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                             "SELECT * FROM %s WHERE a >= 1 AND c > null");
-        assertInvalidMessage("Unsupported null value for column c",
-                             "SELECT * FROM %s WHERE a >= 1 AND c > null ALLOW FILTERING");
-        assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                             "SELECT * FROM %s WHERE a >= 1 AND c CONTAINS null");
-        assertInvalidMessage("Unsupported null value for column c",
-                             "SELECT * FROM %s WHERE a >= 1 AND c CONTAINS null ALLOW FILTERING");
-
-        // Checks filtering with unset
-        assertInvalidMessage("Unsupported unset value for column c",
-                             "SELECT * FROM %s WHERE a >= 1 AND c = ? ALLOW FILTERING",
-                             unset());
-        assertInvalidMessage("Unsupported unset value for column c",
-                             "SELECT * FROM %s WHERE a >= 1 AND c > ? ALLOW FILTERING",
-                             unset());
-        assertInvalidMessage("Unsupported unset value for column c",
-                             "SELECT * FROM %s WHERE a >= 1 AND c CONTAINS ? ALLOW FILTERING",
-                             unset());
-    }
-
-    @Test
-    public void testAllowFilteringOnPartitionKeyOnCompactTablesWithoutIndices() throws Throwable
-    {
-        // ----------------------------------------------
-        // Test COMPACT table with clustering columns
-        // ----------------------------------------------
-        createTable("CREATE TABLE %s (a int, b int, c int, d int, PRIMARY KEY ((a, b), c)) WITH COMPACT STORAGE");
-
-        execute("INSERT INTO %s (a, b, c, d) VALUES (1, 2, 4, 5)");
-        execute("INSERT INTO %s (a, b, c, d) VALUES (1, 3, 6, 7)");
-        execute("INSERT INTO %s (a, b, c, d) VALUES (1, 4, 4, 5)");
-        execute("INSERT INTO %s (a, b, c, d) VALUES (2, 3, 7, 8)");
-
-        // Adds tomstones
-        execute("INSERT INTO %s (a, b, c, d) VALUES (1, 1, 4, 5)");
-        execute("INSERT INTO %s (a, b, c, d) VALUES (2, 2, 7, 8)");
-        execute("DELETE FROM %s WHERE a = 1 AND b = 1 AND c = 4");
-        execute("DELETE FROM %s WHERE a = 2 AND b = 2 AND c = 7");
-
-        beforeAndAfterFlush(() -> {
-            assertRows(execute("SELECT * FROM %s WHERE a = 1 AND b = 4 AND c = 4"),
-                    row(1, 4, 4, 5));
-
-            // Checks filtering
-            assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                    "SELECT * FROM %s WHERE a = 1 AND b = 4 AND c = 4 AND d = 5");
-
-            assertRows(execute("SELECT * FROM %s WHERE a = 1 AND b = 4 AND c = 4 ALLOW FILTERING"),
-                    row(1, 4, 4, 5));
-
-            assertInvalidMessage("IN predicates on non-primary-key columns (d) is not yet supported",
-                    "SELECT * FROM %s WHERE a IN (1, 2) AND b = 3 AND d IN (6, 7)");
-
-            assertInvalidMessage("IN predicates on non-primary-key columns (d) is not yet supported",
-                    "SELECT * FROM %s WHERE a IN (1, 2) AND b = 3 AND d IN (6, 7) ALLOW FILTERING");
-
-            assertRows(execute("SELECT * FROM %s WHERE a < 2 AND c > 4 AND c <= 6 ALLOW FILTERING"),
-                    row(1, 3, 6, 7));
-
-            assertRows(execute("SELECT * FROM %s WHERE a <= 1 AND b >= 2 AND c >= 4 AND d <= 8 ALLOW FILTERING"),
-                    row(1, 3, 6, 7),
-                    row(1, 4, 4, 5),
-                    row(1, 2, 4, 5));
-
-            assertRows(execute("SELECT * FROM %s WHERE a = 1 AND c >= 4 AND d <= 8 ALLOW FILTERING"),
-                    row(1, 3, 6, 7),
-                    row(1, 4, 4, 5),
-                    row(1, 2, 4, 5));
-
-            assertRows(execute("SELECT * FROM %s WHERE a >= 2 AND c >= 4 AND d <= 8 ALLOW FILTERING"),
-                    row(2, 3, 7, 8));
-        });
-
-        // Checks filtering with null
-        assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                             "SELECT * FROM %s WHERE d = null");
-        assertInvalidMessage("Unsupported null value for column a",
-                             "SELECT * FROM %s WHERE a = null ALLOW FILTERING");
-        assertInvalidMessage("Unsupported null value for column a",
-                             "SELECT * FROM %s WHERE a > null ALLOW FILTERING");
-
-        // Checks filtering with unset
-        assertInvalidMessage("Unsupported unset value for column a",
-                             "SELECT * FROM %s WHERE a = ? ALLOW FILTERING",
-                             unset());
-        assertInvalidMessage("Unsupported unset value for column a",
-                             "SELECT * FROM %s WHERE a > ? ALLOW FILTERING",
-                             unset());
-
-        //----------------------------------------------
-        // Test COMPACT table without clustering columns
-        //----------------------------------------------
-        createTable("CREATE TABLE %s (a int primary key, b int, c int) WITH COMPACT STORAGE");
-
-        execute("INSERT INTO %s (a, b, c) VALUES (1, 2, 4)");
-        execute("INSERT INTO %s (a, b, c) VALUES (2, 1, 6)");
-        execute("INSERT INTO %s (a, b, c) VALUES (3, 2, 4)");
-        execute("INSERT INTO %s (a, b, c) VALUES (4, 1, 7)");
-
-        // Adds tomstones
-        execute("INSERT INTO %s (a, b, c) VALUES (0, 1, 4)");
-        execute("INSERT INTO %s (a, b, c) VALUES (5, 2, 7)");
-        execute("DELETE FROM %s WHERE a = 0");
-        execute("DELETE FROM %s WHERE a = 5");
-
-        beforeAndAfterFlush(() -> {
-
-            // Checks filtering
-            assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                    "SELECT * FROM %s WHERE a = 1 AND b = 4 AND c = 4");
-
-            assertRows(execute("SELECT * FROM %s WHERE a = 1 AND b = 2 AND c = 4 ALLOW FILTERING"),
-                    row(1, 2, 4));
-
-            assertRows(execute("SELECT * FROM %s WHERE a = 1 AND b = 2 ALLOW FILTERING"),
-                    row(1, 2, 4));
-
-            assertRows(execute("SELECT * FROM %s WHERE b >= 2 AND c <= 4 ALLOW FILTERING"),
-                    row(1, 2, 4),
-                    row(3, 2, 4));
-
-            assertRows(execute("SELECT * FROM %s WHERE a = 1 ALLOW FILTERING"),
-                    row(1, 2, 4));
-
-            assertRows(execute("SELECT * FROM %s WHERE b >= 2 ALLOW FILTERING"),
-                    row(1, 2, 4),
-                    row(3, 2, 4));
-
-            assertRows(execute("SELECT * FROM %s WHERE a >= 2 AND b <=1 ALLOW FILTERING"),
-                    row(2, 1, 6),
-                    row(4, 1, 7));
-
-            assertRows(execute("SELECT * FROM %s WHERE a = 1 AND c >= 4 ALLOW FILTERING"),
-                    row(1, 2, 4));
-
-            assertInvalidMessage("IN predicates on non-primary-key columns (b) is not yet supported",
-                                 "SELECT * FROM %s WHERE a = 1 AND b IN (1, 2) AND c IN (6, 7)");
-
-            assertInvalidMessage("IN predicates on non-primary-key columns (c) is not yet supported",
-                                 "SELECT * FROM %s WHERE a IN (1, 2) AND c IN (6, 7) ALLOW FILTERING");
-
-            assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                                 "SELECT * FROM %s WHERE c > 4");
-
-            assertRows(execute("SELECT * FROM %s WHERE c > 4 ALLOW FILTERING"),
-                    row(2, 1, 6),
-                    row(4, 1, 7));
-
-            assertRows(execute("SELECT * FROM %s WHERE a >= 1 AND b >= 2 AND c <= 4 ALLOW FILTERING"),
-                    row(1, 2, 4),
-                    row(3, 2, 4));
-
-            assertRows(execute("SELECT * FROM %s WHERE a < 3 AND c <= 4 ALLOW FILTERING"),
-                    row(1, 2, 4));
-
-            assertRows(execute("SELECT * FROM %s WHERE a < 3 AND b >= 2 AND c <= 4 ALLOW FILTERING"),
-                    row(1, 2, 4));
-
-            assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                                 "SELECT * FROM %s WHERE c >= 3 AND c <= 6");
-
-            assertRows(execute("SELECT * FROM %s WHERE c <=6 ALLOW FILTERING"),
-                    row(1, 2, 4),
-                    row(2, 1, 6),
-                    row(3, 2, 4));
-
-            assertRows(execute("SELECT * FROM %s WHERE token(a) >= token(2)"),
-                    row(2, 1, 6),
-                    row(4, 1, 7),
-                    row(3, 2, 4));
-
-            assertRows(execute("SELECT * FROM %s WHERE token(a) >= token(2) ALLOW FILTERING"),
-                    row(2, 1, 6),
-                    row(4, 1, 7),
-                    row(3, 2, 4));
-
-            assertRows(execute("SELECT * FROM %s WHERE token(a) >= token(2) AND b = 1 ALLOW FILTERING"),
-                       row(2, 1, 6),
-                       row(4, 1, 7));
-
-        });
-    }
-
-    @Test
-    public void testFilteringOnCompactTablesWithoutIndicesAndWithMaps() throws Throwable
-    {
-        //----------------------------------------------
-        // Test COMPACT table with clustering columns
-        //----------------------------------------------
-        createTable("CREATE TABLE %s (a int, b int, c frozen<map<int, int>>, PRIMARY KEY (a, b)) WITH COMPACT STORAGE");
-
-        execute("INSERT INTO %s (a, b, c) VALUES (1, 2, {4 : 2})");
-        execute("INSERT INTO %s (a, b, c) VALUES (1, 3, {6 : 2})");
-        execute("INSERT INTO %s (a, b, c) VALUES (1, 4, {4 : 1})");
-        execute("INSERT INTO %s (a, b, c) VALUES (2, 3, {7 : 1})");
-
-        beforeAndAfterFlush(() -> {
-
-            // Checks filtering
-            assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                                 "SELECT * FROM %s WHERE a = 1 AND b = 4 AND c = {4 : 1}");
-
-            assertRows(execute("SELECT * FROM %s WHERE a = 1 AND b = 4 AND c = {4 : 1} ALLOW FILTERING"),
-                       row(1, 4, map(4, 1)));
-
-            assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                                 "SELECT * FROM %s WHERE c > {4 : 2}");
-
-            assertRows(execute("SELECT * FROM %s WHERE c > {4 : 2} ALLOW FILTERING"),
-                       row(1, 3, map(6, 2)),
-                       row(2, 3, map(7, 1)));
-
+            // Checks filtering for sets
             assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                                 "SELECT * FROM %s WHERE b <= 3 AND c < {6 : 2}");
+                    "SELECT * FROM %s WHERE a = 1 AND d CONTAINS 4");
+
+            assertRows(execute("SELECT * FROM %s WHERE d CONTAINS 4 ALLOW FILTERING"),
+                       row(1, 3, list(3, 2), set(6, 4), map(3, 2)),
+                       row(1, 4, list(1, 2), set(2, 4), map(1, 2)));
 
-            assertRows(execute("SELECT * FROM %s WHERE b <= 3 AND c < {6 : 2} ALLOW FILTERING"),
-                       row(1, 2, map(4, 2)));
+            assertRows(execute("SELECT * FROM %s WHERE d CONTAINS 4 AND d CONTAINS 6 ALLOW FILTERING"),
+                       row(1, 3, list(3, 2), set(6, 4), map(3, 2)));
 
+            // Checks filtering for maps
             assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                                 "SELECT * FROM %s WHERE c >= {4 : 2} AND c <= {6 : 4}");
+                                 "SELECT * FROM %s WHERE e CONTAINS 2");
 
-            assertRows(execute("SELECT * FROM %s WHERE c >= {4 : 2} AND c <= {6 : 4} ALLOW FILTERING"),
-                       row(1, 2, map(4, 2)),
-                       row(1, 3, map(6, 2)));
+            assertRows(execute("SELECT * FROM %s WHERE a < 2 AND b >= 3 AND e CONTAINS 2 ALLOW FILTERING"),
+                       row(1, 3, list(3, 2), set(6, 4), map(3, 2)),
+                       row(1, 4, list(1, 2), set(2, 4), map(1, 2)));
 
-            assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
-                                 "SELECT * FROM %s WHERE c CONTAINS 2");
+            assertRows(execute("SELECT * FROM %s WHERE a = 1 AND e CONTAINS KEY 1 ALLOW FILTERING"),
+                       row(1, 4, list(1, 2), set(2, 4), map(1, 2)),
+                       row(1, 2, list(1, 6), set(2, 12), map(1, 6)));
 
-            assertRows(execute("SELECT * FROM %s WHERE c CONTAINS 2 ALLOW FILTERING"),
-                       row(1, 2, map(4, 2)),
-                       row(1, 3, map(6, 2)));
+            assertRows(execute("SELECT * FROM %s WHERE a in (1) AND b in (2) AND e[1] = 6 ALLOW FILTERING"),
+                       row(1, 2, list(1, 6), set(2, 12), map(1, 6)));
 
-            assertRows(execute("SELECT * FROM %s WHERE c CONTAINS KEY 6 ALLOW FILTERING"),
-                       row(1, 3, map(6, 2)));
+            assertRows(execute("SELECT * FROM %s WHERE a = 1 AND e CONTAINS KEY 1 AND e CONTAINS 2 ALLOW FILTERING"),
+                       row(1, 4, list(1, 2), set(2, 4), map(1, 2)));
 
-            assertRows(execute("SELECT * FROM %s WHERE c CONTAINS 2 AND c CONTAINS KEY 6 ALLOW FILTERING"),
-                       row(1, 3, map(6, 2)));
+            assertRows(
+                    execute("SELECT * FROM %s WHERE a >= 1 AND b in (3) AND c CONTAINS 2 AND d CONTAINS 4 AND e CONTAINS KEY 3 ALLOW FILTERING"),
+             

<TRUNCATED>

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


[11/25] cassandra git commit: Merge branch 'cassandra-3.0' into cassandra-3.11

Posted by if...@apache.org.
Merge branch 'cassandra-3.0' into cassandra-3.11


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

Branch: refs/heads/trunk
Commit: 6d429cd0315d3509c904d0e83f91f7d12ba12085
Parents: c8a3b58 6c29ee8
Author: Alex Petrov <ol...@gmail.com>
Authored: Mon Nov 6 15:52:31 2017 +0100
Committer: Alex Petrov <ol...@gmail.com>
Committed: Mon Nov 6 15:52:31 2017 +0100

----------------------------------------------------------------------
 NEWS.txt                                        |  20 +
 bin/cqlsh.py                                    |   7 +-
 doc/native_protocol_v4.spec                     |   4 +
 doc/source/cql/appendices.rst                   |  22 +
 ...dra-driver-internal-only-3.11.0-bb96859b.zip | Bin 0 -> 266661 bytes
 src/antlr/Lexer.g                               |   4 +
 src/antlr/Parser.g                              |  41 +-
 .../cassandra/auth/CassandraRoleManager.java    |   3 +-
 .../org/apache/cassandra/config/CFMetaData.java |  59 ++-
 .../apache/cassandra/cql3/QueryProcessor.java   |   2 +-
 .../cql3/statements/AlterTableStatement.java    |  26 +-
 .../statements/AuthenticationStatement.java     |   2 +-
 .../cql3/statements/AuthorizationStatement.java |   2 +-
 .../cql3/statements/BatchStatement.java         |   4 +-
 .../statements/CreateAggregateStatement.java    |   4 +-
 .../statements/CreateFunctionStatement.java     |   4 +-
 .../cql3/statements/CreateIndexStatement.java   |   9 +-
 .../cql3/statements/CreateTableStatement.java   |   2 +-
 .../cql3/statements/CreateViewStatement.java    |   2 +-
 .../cql3/statements/DropFunctionStatement.java  |   4 +-
 .../cql3/statements/ModificationStatement.java  |  12 +-
 .../cql3/statements/ParsedStatement.java        |   3 +-
 .../statements/SchemaAlteringStatement.java     |  32 +-
 .../cql3/statements/SelectStatement.java        |   8 +-
 .../cql3/statements/TruncateStatement.java      |   2 +-
 .../cassandra/cql3/statements/UseStatement.java |   2 +-
 src/java/org/apache/cassandra/db/view/View.java |  31 +-
 .../index/internal/keys/KeysSearcher.java       |   8 +-
 .../cassandra/io/sstable/CQLSSTableWriter.java  |   2 +-
 .../apache/cassandra/repair/RepairRunnable.java |   3 +-
 .../apache/cassandra/service/ClientState.java   |  16 +
 .../cassandra/thrift/ThriftValidation.java      |  10 +-
 .../transport/messages/StartupMessage.java      |   4 +
 .../org/apache/cassandra/cql3/ViewTest.java     |  16 +-
 .../cql3/validation/entities/UFTest.java        |  25 +
 .../cql3/validation/entities/UserTypesTest.java |  20 +
 .../cql3/validation/operations/AlterTest.java   |   8 +
 .../DropCompactStorageThriftTest.java           | 525 +++++++++++++++++++
 .../cassandra/index/sasi/SASIIndexTest.java     |   3 +-
 .../io/sstable/StressCQLSSTableWriter.java      |   2 +-
 40 files changed, 876 insertions(+), 77 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d429cd0/NEWS.txt
----------------------------------------------------------------------
diff --cc NEWS.txt
index f8dd93b,621866b..9812b22
--- a/NEWS.txt
+++ b/NEWS.txt
@@@ -33,16 -34,28 +33,36 @@@ Materialized View
  
  Upgrading
  ---------
 -   - Nothing specific to this release, but please see previous upgrading sections,
 -     especially if you are upgrading from 2.2.
 +    - Creating Materialized View with filtering on non-primary-key base column
 +      (added in CASSANDRA-10368) is disabled, because the liveness of view row
 +      is depending on multiple filtered base non-key columns and base non-key
 +      column used in view primary-key. This semantic cannot be supported without
 +      storage format change, see CASSANDRA-13826. For append-only use case, you
 +      may still use this feature with a startup flag: "-Dcassandra.mv.allow_filtering_nonkey_columns_unsafe=true"
  
 -Compact Storage
++Compact Storage (only when upgrading from 3.X or any version lower than 3.0.15)
+ ---------------
+     - Starting version 4.0, Thrift and COMPACT STORAGE is no longer supported.
+       'ALTER ... DROP COMPACT STORAGE' statement makes Compact Tables CQL-compatible,
+       exposing internal structure of Thrift/Compact Tables. You can find more details
+       on exposed internal structure under: 
+       http://cassandra.apache.org/doc/latest/cql/appendices.html#appendix-c-dropping-compact-storage
+ 
+       For uninterrupted cluster upgrades, drivers now support 'NO_COMPACT' startup option.
+       Supplying this flag will have same effect as 'DROP COMPACT STORAGE', but only for the
+       current connection.
+ 
+       In order to upgrade, clients supporting a non-compact schema view can be rolled out
+       gradually. When all the clients are updated 'ALTER ... DROP COMPACT STORAGE' can be
+       executed. After dropping compact storage, ’NO_COMPACT' option will have no effect
+       after that.
+ 
++
+ Materialized Views
+ -------------------
 +
 +Materialized Views (only when upgrading from 3.X or any version lower than 3.0.15)
 +---------------------------------------------------------------------------------------
      - Cassandra will no longer allow dropping columns on tables with Materialized Views.
      - A change was made in the way the Materialized View timestamp is computed, which
        may cause an old deletion to a base column which is view primary key (PK) column

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d429cd0/bin/cqlsh.py
----------------------------------------------------------------------
diff --cc bin/cqlsh.py
index 85a0489,8d05d9d..ca76e34
--- a/bin/cqlsh.py
+++ b/bin/cqlsh.py
@@@ -439,8 -701,9 +440,9 @@@ class Shell(cmd.Cmd)
      def __init__(self, hostname, port, color=False,
                   username=None, password=None, encoding=None, stdin=None, tty=True,
                   completekey=DEFAULT_COMPLETEKEY, browser=None, use_conn=None,
 -                 cqlver=DEFAULT_CQLVER, keyspace=None,
 +                 cqlver=None, keyspace=None,
                   tracing_enabled=False, expand_enabled=False,
+                  no_compact=False,
                   display_nanotime_format=DEFAULT_NANOTIME_FORMAT,
                   display_timestamp_format=DEFAULT_TIMESTAMP_FORMAT,
                   display_date_format=DEFAULT_DATE_FORMAT,
@@@ -470,11 -732,9 +472,12 @@@
          if use_conn:
              self.conn = use_conn
          else:
 +            kwargs = {}
 +            if protocol_version is not None:
 +                kwargs['protocol_version'] = protocol_version
              self.conn = Cluster(contact_points=(self.hostname,), port=self.port, cql_version=cqlver,
 -                                protocol_version=protocol_version,
 -                                auth_provider=self.auth_provider, no_compact=no_compact,
 +                                auth_provider=self.auth_provider,
++                                no_compact=no_compact,
                                  ssl_options=sslhandling.ssl_settings(hostname, CONFIG_FILE) if ssl else None,
                                  load_balancing_policy=WhiteListRoundRobinPolicy([self.hostname]),
                                  control_connection_timeout=connect_timeout,
@@@ -2396,9 -2644,9 +2400,10 @@@ def main(options, hostname, port)
                        tty=options.tty,
                        completekey=options.completekey,
                        browser=options.browser,
 +                      protocol_version=options.protocol_version,
                        cqlver=options.cqlversion,
                        keyspace=options.keyspace,
+                       no_compact=options.no_compact,
                        display_timestamp_format=options.time_format,
                        display_nanotime_format=options.nanotime_format,
                        display_date_format=options.date_format,

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d429cd0/doc/source/cql/appendices.rst
----------------------------------------------------------------------
diff --cc doc/source/cql/appendices.rst
index 8c63a32,0000000..480b78e
mode 100644,000000..100644
--- a/doc/source/cql/appendices.rst
+++ b/doc/source/cql/appendices.rst
@@@ -1,308 -1,0 +1,330 @@@
 +.. 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.
 +
 +.. highlight:: cql
 +
 +Appendices
 +----------
 +
 +.. _appendix-A:
 +
 +Appendix A: CQL Keywords
 +~~~~~~~~~~~~~~~~~~~~~~~~
 +
 +CQL distinguishes between *reserved* and *non-reserved* keywords.
 +Reserved keywords cannot be used as identifier, they are truly reserved
 +for the language (but one can enclose a reserved keyword by
 +double-quotes to use it as an identifier). Non-reserved keywords however
 +only have a specific meaning in certain context but can used as
 +identifier otherwise. The only *raison d’être* of these non-reserved
 +keywords is convenience: some keyword are non-reserved when it was
 +always easy for the parser to decide whether they were used as keywords
 +or not.
 +
 ++--------------------+-------------+
 +| Keyword            | Reserved?   |
 ++====================+=============+
 +| ``ADD``            | yes         |
 ++--------------------+-------------+
 +| ``AGGREGATE``      | no          |
 ++--------------------+-------------+
 +| ``ALL``            | no          |
 ++--------------------+-------------+
 +| ``ALLOW``          | yes         |
 ++--------------------+-------------+
 +| ``ALTER``          | yes         |
 ++--------------------+-------------+
 +| ``AND``            | yes         |
 ++--------------------+-------------+
 +| ``APPLY``          | yes         |
 ++--------------------+-------------+
 +| ``AS``             | no          |
 ++--------------------+-------------+
 +| ``ASC``            | yes         |
 ++--------------------+-------------+
 +| ``ASCII``          | no          |
 ++--------------------+-------------+
 +| ``AUTHORIZE``      | yes         |
 ++--------------------+-------------+
 +| ``BATCH``          | yes         |
 ++--------------------+-------------+
 +| ``BEGIN``          | yes         |
 ++--------------------+-------------+
 +| ``BIGINT``         | no          |
 ++--------------------+-------------+
 +| ``BLOB``           | no          |
 ++--------------------+-------------+
 +| ``BOOLEAN``        | no          |
 ++--------------------+-------------+
 +| ``BY``             | yes         |
 ++--------------------+-------------+
 +| ``CALLED``         | no          |
 ++--------------------+-------------+
 +| ``CLUSTERING``     | no          |
 ++--------------------+-------------+
 +| ``COLUMNFAMILY``   | yes         |
 ++--------------------+-------------+
 +| ``COMPACT``        | no          |
 ++--------------------+-------------+
 +| ``CONTAINS``       | no          |
 ++--------------------+-------------+
 +| ``COUNT``          | no          |
 ++--------------------+-------------+
 +| ``COUNTER``        | no          |
 ++--------------------+-------------+
 +| ``CREATE``         | yes         |
 ++--------------------+-------------+
 +| ``CUSTOM``         | no          |
 ++--------------------+-------------+
 +| ``DATE``           | no          |
 ++--------------------+-------------+
 +| ``DECIMAL``        | no          |
 ++--------------------+-------------+
 +| ``DELETE``         | yes         |
 ++--------------------+-------------+
 +| ``DESC``           | yes         |
 ++--------------------+-------------+
 +| ``DESCRIBE``       | yes         |
 ++--------------------+-------------+
 +| ``DISTINCT``       | no          |
 ++--------------------+-------------+
 +| ``DOUBLE``         | no          |
 ++--------------------+-------------+
 +| ``DROP``           | yes         |
 ++--------------------+-------------+
 +| ``ENTRIES``        | yes         |
 ++--------------------+-------------+
 +| ``EXECUTE``        | yes         |
 ++--------------------+-------------+
 +| ``EXISTS``         | no          |
 ++--------------------+-------------+
 +| ``FILTERING``      | no          |
 ++--------------------+-------------+
 +| ``FINALFUNC``      | no          |
 ++--------------------+-------------+
 +| ``FLOAT``          | no          |
 ++--------------------+-------------+
 +| ``FROM``           | yes         |
 ++--------------------+-------------+
 +| ``FROZEN``         | no          |
 ++--------------------+-------------+
 +| ``FULL``           | yes         |
 ++--------------------+-------------+
 +| ``FUNCTION``       | no          |
 ++--------------------+-------------+
 +| ``FUNCTIONS``      | no          |
 ++--------------------+-------------+
 +| ``GRANT``          | yes         |
 ++--------------------+-------------+
 +| ``IF``             | yes         |
 ++--------------------+-------------+
 +| ``IN``             | yes         |
 ++--------------------+-------------+
 +| ``INDEX``          | yes         |
 ++--------------------+-------------+
 +| ``INET``           | no          |
 ++--------------------+-------------+
 +| ``INFINITY``       | yes         |
 ++--------------------+-------------+
 +| ``INITCOND``       | no          |
 ++--------------------+-------------+
 +| ``INPUT``          | no          |
 ++--------------------+-------------+
 +| ``INSERT``         | yes         |
 ++--------------------+-------------+
 +| ``INT``            | no          |
 ++--------------------+-------------+
 +| ``INTO``           | yes         |
 ++--------------------+-------------+
 +| ``JSON``           | no          |
 ++--------------------+-------------+
 +| ``KEY``            | no          |
 ++--------------------+-------------+
 +| ``KEYS``           | no          |
 ++--------------------+-------------+
 +| ``KEYSPACE``       | yes         |
 ++--------------------+-------------+
 +| ``KEYSPACES``      | no          |
 ++--------------------+-------------+
 +| ``LANGUAGE``       | no          |
 ++--------------------+-------------+
 +| ``LIMIT``          | yes         |
 ++--------------------+-------------+
 +| ``LIST``           | no          |
 ++--------------------+-------------+
 +| ``LOGIN``          | no          |
 ++--------------------+-------------+
 +| ``MAP``            | no          |
 ++--------------------+-------------+
 +| ``MODIFY``         | yes         |
 ++--------------------+-------------+
 +| ``NAN``            | yes         |
 ++--------------------+-------------+
 +| ``NOLOGIN``        | no          |
 ++--------------------+-------------+
 +| ``NORECURSIVE``    | yes         |
 ++--------------------+-------------+
 +| ``NOSUPERUSER``    | no          |
 ++--------------------+-------------+
 +| ``NOT``            | yes         |
 ++--------------------+-------------+
 +| ``NULL``           | yes         |
 ++--------------------+-------------+
 +| ``OF``             | yes         |
 ++--------------------+-------------+
 +| ``ON``             | yes         |
 ++--------------------+-------------+
 +| ``OPTIONS``        | no          |
 ++--------------------+-------------+
 +| ``OR``             | yes         |
 ++--------------------+-------------+
 +| ``ORDER``          | yes         |
 ++--------------------+-------------+
 +| ``PASSWORD``       | no          |
 ++--------------------+-------------+
 +| ``PERMISSION``     | no          |
 ++--------------------+-------------+
 +| ``PERMISSIONS``    | no          |
 ++--------------------+-------------+
 +| ``PRIMARY``        | yes         |
 ++--------------------+-------------+
 +| ``RENAME``         | yes         |
 ++--------------------+-------------+
 +| ``REPLACE``        | yes         |
 ++--------------------+-------------+
 +| ``RETURNS``        | no          |
 ++--------------------+-------------+
 +| ``REVOKE``         | yes         |
 ++--------------------+-------------+
 +| ``ROLE``           | no          |
 ++--------------------+-------------+
 +| ``ROLES``          | no          |
 ++--------------------+-------------+
 +| ``SCHEMA``         | yes         |
 ++--------------------+-------------+
 +| ``SELECT``         | yes         |
 ++--------------------+-------------+
 +| ``SET``            | yes         |
 ++--------------------+-------------+
 +| ``SFUNC``          | no          |
 ++--------------------+-------------+
 +| ``SMALLINT``       | no          |
 ++--------------------+-------------+
 +| ``STATIC``         | no          |
 ++--------------------+-------------+
 +| ``STORAGE``        | no          |
 ++--------------------+-------------+
 +| ``STYPE``          | no          |
 ++--------------------+-------------+
 +| ``SUPERUSER``      | no          |
 ++--------------------+-------------+
 +| ``TABLE``          | yes         |
 ++--------------------+-------------+
 +| ``TEXT``           | no          |
 ++--------------------+-------------+
 +| ``TIME``           | no          |
 ++--------------------+-------------+
 +| ``TIMESTAMP``      | no          |
 ++--------------------+-------------+
 +| ``TIMEUUID``       | no          |
 ++--------------------+-------------+
 +| ``TINYINT``        | no          |
 ++--------------------+-------------+
 +| ``TO``             | yes         |
 ++--------------------+-------------+
 +| ``TOKEN``          | yes         |
 ++--------------------+-------------+
 +| ``TRIGGER``        | no          |
 ++--------------------+-------------+
 +| ``TRUNCATE``       | yes         |
 ++--------------------+-------------+
 +| ``TTL``            | no          |
 ++--------------------+-------------+
 +| ``TUPLE``          | no          |
 ++--------------------+-------------+
 +| ``TYPE``           | no          |
 ++--------------------+-------------+
 +| ``UNLOGGED``       | yes         |
 ++--------------------+-------------+
 +| ``UPDATE``         | yes         |
 ++--------------------+-------------+
 +| ``USE``            | yes         |
 ++--------------------+-------------+
 +| ``USER``           | no          |
 ++--------------------+-------------+
 +| ``USERS``          | no          |
 ++--------------------+-------------+
 +| ``USING``          | yes         |
 ++--------------------+-------------+
 +| ``UUID``           | no          |
 ++--------------------+-------------+
 +| ``VALUES``         | no          |
 ++--------------------+-------------+
 +| ``VARCHAR``        | no          |
 ++--------------------+-------------+
 +| ``VARINT``         | no          |
 ++--------------------+-------------+
 +| ``WHERE``          | yes         |
 ++--------------------+-------------+
 +| ``WITH``           | yes         |
 ++--------------------+-------------+
 +| ``WRITETIME``      | no          |
 ++--------------------+-------------+
 +
 +Appendix B: CQL Reserved Types
 +~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
 +
 +The following type names are not currently used by CQL, but are reserved
 +for potential future use. User-defined types may not use reserved type
 +names as their name.
 +
 ++-----------------+
 +| type            |
 ++=================+
 +| ``bitstring``   |
 ++-----------------+
 +| ``byte``        |
 ++-----------------+
 +| ``complex``     |
 ++-----------------+
 +| ``enum``        |
 ++-----------------+
 +| ``interval``    |
 ++-----------------+
 +| ``macaddr``     |
 ++-----------------+
++
++
++Appendix C: Dropping Compact Storage
++~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
++
++Starting version 4.0, Thrift and COMPACT STORAGE is no longer supported.
++
++'ALTER ... DROP COMPACT STORAGE' statement makes Compact Tables CQL-compatible,
++exposing internal structure of Thrift/Compact Tables:
++
++- CQL-created Compact Tables that have no clustering columns, will expose an
++  additional clustering column ``column1`` with ``UTF8Type``.
++- CQL-created Compact Tables that had no regular columns, will expose a
++  regular column ``value`` with ``BytesType``.
++- For CQL-Created Compact Tables, all columns originally defined as
++  ``regular`` will be come ``static``
++- CQL-created Compact Tables that have clustering but have no regular
++  columns will have an empty value column (of ``EmptyType``)
++- SuperColumn Tables (can only be created through Thrift) will expose
++  a compact value map with an empty name.
++- Thrift-created Compact Tables will have types corresponding to their
++  Thrift definition.

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d429cd0/src/antlr/Lexer.g
----------------------------------------------------------------------
diff --cc src/antlr/Lexer.g
index 23cbed6,0000000..1c52d4c
mode 100644,000000..100644
--- a/src/antlr/Lexer.g
+++ b/src/antlr/Lexer.g
@@@ -1,350 -1,0 +1,354 @@@
 +/*
 + * 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.
 + */
 +
 +lexer grammar Lexer;
 +
 +@lexer::members {
 +    List<Token> tokens = new ArrayList<Token>();
 +
 +    public void emit(Token token)
 +    {
 +        state.token = token;
 +        tokens.add(token);
 +    }
 +
 +    public Token nextToken()
 +    {
 +        super.nextToken();
 +        if (tokens.size() == 0)
 +            return new CommonToken(Token.EOF);
 +        return tokens.remove(0);
 +    }
 +
 +    private final List<ErrorListener> listeners = new ArrayList<ErrorListener>();
 +
 +    public void addErrorListener(ErrorListener listener)
 +    {
 +        this.listeners.add(listener);
 +    }
 +
 +    public void removeErrorListener(ErrorListener listener)
 +    {
 +        this.listeners.remove(listener);
 +    }
 +
 +    public void displayRecognitionError(String[] tokenNames, RecognitionException e)
 +    {
 +        for (int i = 0, m = listeners.size(); i < m; i++)
 +            listeners.get(i).syntaxError(this, tokenNames, e);
 +    }
 +}
 +
 +// Case-insensitive keywords
 +// When adding a new reserved keyword, add entry to o.a.c.cql3.ReservedKeywords as well
 +// When adding a new unreserved keyword, add entry to unreserved keywords in Parser.g
 +K_SELECT:      S E L E C T;
 +K_FROM:        F R O M;
 +K_AS:          A S;
 +K_WHERE:       W H E R E;
 +K_AND:         A N D;
 +K_KEY:         K E Y;
 +K_KEYS:        K E Y S;
 +K_ENTRIES:     E N T R I E S;
 +K_FULL:        F U L L;
 +K_INSERT:      I N S E R T;
 +K_UPDATE:      U P D A T E;
 +K_WITH:        W I T H;
 +K_LIMIT:       L I M I T;
 +K_PER:         P E R;
 +K_PARTITION:   P A R T I T I O N;
 +K_USING:       U S I N G;
 +K_USE:         U S E;
 +K_DISTINCT:    D I S T I N C T;
 +K_COUNT:       C O U N T;
 +K_SET:         S E T;
 +K_BEGIN:       B E G I N;
 +K_UNLOGGED:    U N L O G G E D;
 +K_BATCH:       B A T C H;
 +K_APPLY:       A P P L Y;
 +K_TRUNCATE:    T R U N C A T E;
 +K_DELETE:      D E L E T E;
 +K_IN:          I N;
 +K_CREATE:      C R E A T E;
 +K_KEYSPACE:    ( K E Y S P A C E
 +                 | S C H E M A );
 +K_KEYSPACES:   K E Y S P A C E S;
 +K_COLUMNFAMILY:( C O L U M N F A M I L Y
 +                 | T A B L E );
 +K_MATERIALIZED:M A T E R I A L I Z E D;
 +K_VIEW:        V I E W;
 +K_INDEX:       I N D E X;
 +K_CUSTOM:      C U S T O M;
 +K_ON:          O N;
 +K_TO:          T O;
 +K_DROP:        D R O P;
 +K_PRIMARY:     P R I M A R Y;
 +K_INTO:        I N T O;
 +K_VALUES:      V A L U E S;
 +K_TIMESTAMP:   T I M E S T A M P;
 +K_TTL:         T T L;
 +K_CAST:        C A S T;
 +K_ALTER:       A L T E R;
 +K_RENAME:      R E N A M E;
 +K_ADD:         A D D;
 +K_TYPE:        T Y P E;
 +K_COMPACT:     C O M P A C T;
 +K_STORAGE:     S T O R A G E;
 +K_ORDER:       O R D E R;
 +K_BY:          B Y;
 +K_ASC:         A S C;
 +K_DESC:        D E S C;
 +K_ALLOW:       A L L O W;
 +K_FILTERING:   F I L T E R I N G;
 +K_IF:          I F;
 +K_IS:          I S;
 +K_CONTAINS:    C O N T A I N S;
 +K_GROUP:       G R O U P;
 +
 +K_GRANT:       G R A N T;
 +K_ALL:         A L L;
 +K_PERMISSION:  P E R M I S S I O N;
 +K_PERMISSIONS: P E R M I S S I O N S;
 +K_OF:          O F;
 +K_REVOKE:      R E V O K E;
 +K_MODIFY:      M O D I F Y;
 +K_AUTHORIZE:   A U T H O R I Z E;
 +K_DESCRIBE:    D E S C R I B E;
 +K_EXECUTE:     E X E C U T E;
 +K_NORECURSIVE: N O R E C U R S I V E;
 +K_MBEAN:       M B E A N;
 +K_MBEANS:      M B E A N S;
 +
 +K_USER:        U S E R;
 +K_USERS:       U S E R S;
 +K_ROLE:        R O L E;
 +K_ROLES:       R O L E S;
 +K_SUPERUSER:   S U P E R U S E R;
 +K_NOSUPERUSER: N O S U P E R U S E R;
 +K_PASSWORD:    P A S S W O R D;
 +K_LOGIN:       L O G I N;
 +K_NOLOGIN:     N O L O G I N;
 +K_OPTIONS:     O P T I O N S;
 +
 +K_CLUSTERING:  C L U S T E R I N G;
 +K_ASCII:       A S C I I;
 +K_BIGINT:      B I G I N T;
 +K_BLOB:        B L O B;
 +K_BOOLEAN:     B O O L E A N;
 +K_COUNTER:     C O U N T E R;
 +K_DECIMAL:     D E C I M A L;
 +K_DOUBLE:      D O U B L E;
 +K_DURATION:    D U R A T I O N;
 +K_FLOAT:       F L O A T;
 +K_INET:        I N E T;
 +K_INT:         I N T;
 +K_SMALLINT:    S M A L L I N T;
 +K_TINYINT:     T I N Y I N T;
 +K_TEXT:        T E X T;
 +K_UUID:        U U I D;
 +K_VARCHAR:     V A R C H A R;
 +K_VARINT:      V A R I N T;
 +K_TIMEUUID:    T I M E U U I D;
 +K_TOKEN:       T O K E N;
 +K_WRITETIME:   W R I T E T I M E;
 +K_DATE:        D A T E;
 +K_TIME:        T I M E;
 +
 +K_NULL:        N U L L;
 +K_NOT:         N O T;
 +K_EXISTS:      E X I S T S;
 +
 +K_MAP:         M A P;
 +K_LIST:        L I S T;
 +K_NAN:         N A N;
 +K_INFINITY:    I N F I N I T Y;
 +K_TUPLE:       T U P L E;
 +
 +K_TRIGGER:     T R I G G E R;
 +K_STATIC:      S T A T I C;
 +K_FROZEN:      F R O Z E N;
 +
 +K_FUNCTION:    F U N C T I O N;
 +K_FUNCTIONS:   F U N C T I O N S;
 +K_AGGREGATE:   A G G R E G A T E;
 +K_SFUNC:       S F U N C;
 +K_STYPE:       S T Y P E;
 +K_FINALFUNC:   F I N A L F U N C;
 +K_INITCOND:    I N I T C O N D;
 +K_RETURNS:     R E T U R N S;
 +K_CALLED:      C A L L E D;
 +K_INPUT:       I N P U T;
 +K_LANGUAGE:    L A N G U A G E;
 +K_OR:          O R;
 +K_REPLACE:     R E P L A C E;
 +
 +K_JSON:        J S O N;
 +K_DEFAULT:     D E F A U L T;
 +K_UNSET:       U N S E T;
 +K_LIKE:        L I K E;
 +
 +// Case-insensitive alpha characters
 +fragment A: ('a'|'A');
 +fragment B: ('b'|'B');
 +fragment C: ('c'|'C');
 +fragment D: ('d'|'D');
 +fragment E: ('e'|'E');
 +fragment F: ('f'|'F');
 +fragment G: ('g'|'G');
 +fragment H: ('h'|'H');
 +fragment I: ('i'|'I');
 +fragment J: ('j'|'J');
 +fragment K: ('k'|'K');
 +fragment L: ('l'|'L');
 +fragment M: ('m'|'M');
 +fragment N: ('n'|'N');
 +fragment O: ('o'|'O');
 +fragment P: ('p'|'P');
 +fragment Q: ('q'|'Q');
 +fragment R: ('r'|'R');
 +fragment S: ('s'|'S');
 +fragment T: ('t'|'T');
 +fragment U: ('u'|'U');
 +fragment V: ('v'|'V');
 +fragment W: ('w'|'W');
 +fragment X: ('x'|'X');
 +fragment Y: ('y'|'Y');
 +fragment Z: ('z'|'Z');
 +
 +STRING_LITERAL
 +    @init{
 +        StringBuilder txt = new StringBuilder(); // temporary to build pg-style-string
 +    }
 +    @after{ setText(txt.toString()); }
 +    :
 +      /* pg-style string literal */
 +      (
 +        '\$' '\$'
 +        ( /* collect all input until '$$' is reached again */
 +          {  (input.size() - input.index() > 1)
 +               && !"$$".equals(input.substring(input.index(), input.index() + 1)) }?
 +             => c=. { txt.appendCodePoint(c); }
 +        )*
 +        '\$' '\$'
 +      )
 +      |
 +      /* conventional quoted string literal */
 +      (
 +        '\'' (c=~('\'') { txt.appendCodePoint(c);} | '\'' '\'' { txt.appendCodePoint('\''); })* '\''
 +      )
 +    ;
 +
 +QUOTED_NAME
 +    @init{ StringBuilder b = new StringBuilder(); }
 +    @after{ setText(b.toString()); }
 +    : '\"' (c=~('\"') { b.appendCodePoint(c); } | '\"' '\"' { b.appendCodePoint('\"'); })+ '\"'
 +    ;
 +
++EMPTY_QUOTED_NAME
++    : '\"' '\"'
++    ;
++
 +fragment DIGIT
 +    : '0'..'9'
 +    ;
 +
 +fragment LETTER
 +    : ('A'..'Z' | 'a'..'z')
 +    ;
 +
 +fragment HEX
 +    : ('A'..'F' | 'a'..'f' | '0'..'9')
 +    ;
 +
 +fragment EXPONENT
 +    : E ('+' | '-')? DIGIT+
 +    ;
 +
 +fragment DURATION_UNIT
 +    : Y
 +    | M O
 +    | W
 +    | D
 +    | H
 +    | M
 +    | S
 +    | M S
 +    | U S
 +    | '\u00B5' S
 +    | N S
 +    ;
 +
 +INTEGER
 +    : '-'? DIGIT+
 +    ;
 +
 +QMARK
 +    : '?'
 +    ;
 +
 +/*
 + * Normally a lexer only emits one token at a time, but ours is tricked out
 + * to support multiple (see @lexer::members near the top of the grammar).
 + */
 +FLOAT
 +    : INTEGER EXPONENT
 +    | INTEGER '.' DIGIT* EXPONENT?
 +    ;
 +
 +/*
 + * This has to be before IDENT so it takes precendence over it.
 + */
 +BOOLEAN
 +    : T R U E | F A L S E
 +    ;
 +
 +DURATION
 +    : '-'? DIGIT+ DURATION_UNIT (DIGIT+ DURATION_UNIT)*
 +    | '-'? 'P' (DIGIT+ 'Y')? (DIGIT+ 'M')? (DIGIT+ 'D')? ('T' (DIGIT+ 'H')? (DIGIT+ 'M')? (DIGIT+ 'S')?)? // ISO 8601 "format with designators"
 +    | '-'? 'P' DIGIT+ 'W'
 +    | '-'? 'P' DIGIT DIGIT DIGIT DIGIT '-' DIGIT DIGIT '-' DIGIT DIGIT 'T' DIGIT DIGIT ':' DIGIT DIGIT ':' DIGIT DIGIT // ISO 8601 "alternative format"
 +    ;
 +
 +IDENT
 +    : LETTER (LETTER | DIGIT | '_')*
 +    ;
 +
 +HEXNUMBER
 +    : '0' X HEX*
 +    ;
 +
 +UUID
 +    : HEX HEX HEX HEX HEX HEX HEX HEX '-'
 +      HEX HEX HEX HEX '-'
 +      HEX HEX HEX HEX '-'
 +      HEX HEX HEX HEX '-'
 +      HEX HEX HEX HEX HEX HEX HEX HEX HEX HEX HEX HEX
 +    ;
 +
 +WS
 +    : (' ' | '\t' | '\n' | '\r')+ { $channel = HIDDEN; }
 +    ;
 +
 +COMMENT
 +    : ('--' | '//') .* ('\n'|'\r') { $channel = HIDDEN; }
 +    ;
 +
 +MULTILINE_COMMENT
 +    : '/*' .* '*/' { $channel = HIDDEN; }
 +    ;


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


[15/25] cassandra git commit: Merge branch 'cassandra-3.0' into cassandra-3.11

Posted by if...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d429cd0/src/antlr/Parser.g
----------------------------------------------------------------------
diff --cc src/antlr/Parser.g
index e5b7584,0000000..26074b8
mode 100644,000000..100644
--- a/src/antlr/Parser.g
+++ b/src/antlr/Parser.g
@@@ -1,1675 -1,0 +1,1688 @@@
 +/*
 + * 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.
 + */
 +
 +parser grammar Parser;
 +
 +options {
 +    language = Java;
 +}
 +
 +@members {
 +    private final List<ErrorListener> listeners = new ArrayList<ErrorListener>();
 +    protected final List<ColumnIdentifier> bindVariables = new ArrayList<ColumnIdentifier>();
 +
 +    public static final Set<String> reservedTypeNames = new HashSet<String>()
 +    {{
 +        add("byte");
 +        add("complex");
 +        add("enum");
 +        add("date");
 +        add("interval");
 +        add("macaddr");
 +        add("bitstring");
 +    }};
 +
 +    public AbstractMarker.Raw newBindVariables(ColumnIdentifier name)
 +    {
 +        AbstractMarker.Raw marker = new AbstractMarker.Raw(bindVariables.size());
 +        bindVariables.add(name);
 +        return marker;
 +    }
 +
 +    public AbstractMarker.INRaw newINBindVariables(ColumnIdentifier name)
 +    {
 +        AbstractMarker.INRaw marker = new AbstractMarker.INRaw(bindVariables.size());
 +        bindVariables.add(name);
 +        return marker;
 +    }
 +
 +    public Tuples.Raw newTupleBindVariables(ColumnIdentifier name)
 +    {
 +        Tuples.Raw marker = new Tuples.Raw(bindVariables.size());
 +        bindVariables.add(name);
 +        return marker;
 +    }
 +
 +    public Tuples.INRaw newTupleINBindVariables(ColumnIdentifier name)
 +    {
 +        Tuples.INRaw marker = new Tuples.INRaw(bindVariables.size());
 +        bindVariables.add(name);
 +        return marker;
 +    }
 +
 +    public Json.Marker newJsonBindVariables(ColumnIdentifier name)
 +    {
 +        Json.Marker marker = new Json.Marker(bindVariables.size());
 +        bindVariables.add(name);
 +        return marker;
 +    }
 +
 +    public void addErrorListener(ErrorListener listener)
 +    {
 +        this.listeners.add(listener);
 +    }
 +
 +    public void removeErrorListener(ErrorListener listener)
 +    {
 +        this.listeners.remove(listener);
 +    }
 +
 +    public void displayRecognitionError(String[] tokenNames, RecognitionException e)
 +    {
 +        for (int i = 0, m = listeners.size(); i < m; i++)
 +            listeners.get(i).syntaxError(this, tokenNames, e);
 +    }
 +
 +    protected void addRecognitionError(String msg)
 +    {
 +        for (int i = 0, m = listeners.size(); i < m; i++)
 +            listeners.get(i).syntaxError(this, msg);
 +    }
 +
 +    public Map<String, String> convertPropertyMap(Maps.Literal map)
 +    {
 +        if (map == null || map.entries == null || map.entries.isEmpty())
 +            return Collections.<String, String>emptyMap();
 +
 +        Map<String, String> res = new HashMap<>(map.entries.size());
 +
 +        for (Pair<Term.Raw, Term.Raw> entry : map.entries)
 +        {
 +            // Because the parser tries to be smart and recover on error (to
 +            // allow displaying more than one error I suppose), we have null
 +            // entries in there. Just skip those, a proper error will be thrown in the end.
 +            if (entry.left == null || entry.right == null)
 +                break;
 +
 +            if (!(entry.left instanceof Constants.Literal))
 +            {
 +                String msg = "Invalid property name: " + entry.left;
 +                if (entry.left instanceof AbstractMarker.Raw)
 +                    msg += " (bind variables are not supported in DDL queries)";
 +                addRecognitionError(msg);
 +                break;
 +            }
 +            if (!(entry.right instanceof Constants.Literal))
 +            {
 +                String msg = "Invalid property value: " + entry.right + " for property: " + entry.left;
 +                if (entry.right instanceof AbstractMarker.Raw)
 +                    msg += " (bind variables are not supported in DDL queries)";
 +                addRecognitionError(msg);
 +                break;
 +            }
 +
 +            if (res.put(((Constants.Literal)entry.left).getRawText(), ((Constants.Literal)entry.right).getRawText()) != null)
 +            {
 +                addRecognitionError(String.format("Multiple definition for property " + ((Constants.Literal)entry.left).getRawText()));
 +            }
 +        }
 +
 +        return res;
 +    }
 +
 +    public void addRawUpdate(List<Pair<ColumnDefinition.Raw, Operation.RawUpdate>> operations, ColumnDefinition.Raw key, Operation.RawUpdate update)
 +    {
 +        for (Pair<ColumnDefinition.Raw, Operation.RawUpdate> p : operations)
 +        {
 +            if (p.left.equals(key) && !p.right.isCompatibleWith(update))
 +                addRecognitionError("Multiple incompatible setting of column " + key);
 +        }
 +        operations.add(Pair.create(key, update));
 +    }
 +
 +    public Set<Permission> filterPermissions(Set<Permission> permissions, IResource resource)
 +    {
 +        if (resource == null)
 +            return Collections.emptySet();
 +        Set<Permission> filtered = new HashSet<>(permissions);
 +        filtered.retainAll(resource.applicablePermissions());
 +        if (filtered.isEmpty())
 +            addRecognitionError("Resource type " + resource.getClass().getSimpleName() +
 +                                    " does not support any of the requested permissions");
 +
 +        return filtered;
 +    }
 +
 +    public String canonicalizeObjectName(String s, boolean enforcePattern)
 +    {
 +        // these two conditions are here because technically they are valid
 +        // ObjectNames, but we want to restrict their use without adding unnecessary
 +        // work to JMXResource construction as that also happens on hotter code paths
 +        if ("".equals(s))
 +            addRecognitionError("Empty JMX object name supplied");
 +
 +        if ("*:*".equals(s))
 +            addRecognitionError("Please use ALL MBEANS instead of wildcard pattern");
 +
 +        try
 +        {
 +            javax.management.ObjectName objectName = javax.management.ObjectName.getInstance(s);
 +            if (enforcePattern && !objectName.isPattern())
 +                addRecognitionError("Plural form used, but non-pattern JMX object name specified (" + s + ")");
 +            return objectName.getCanonicalName();
 +        }
 +        catch (javax.management.MalformedObjectNameException e)
 +        {
 +          addRecognitionError(s + " is not a valid JMX object name");
 +          return s;
 +        }
 +    }
 +
 +    ////////////////////////////////////////////////////////////////////////////////////////////////////////////////
 +    // Recovery methods are overridden to avoid wasting work on recovering from errors when the result will be
 +    // ignored anyway.
 +    ////////////////////////////////////////////////////////////////////////////////////////////////////////////////
 +
 +    @Override
 +    protected Object recoverFromMismatchedToken(IntStream input, int ttype, BitSet follow) throws RecognitionException
 +    {
 +        throw new MismatchedTokenException(ttype, input);
 +    }
 +
 +    @Override
 +    public void recover(IntStream input, RecognitionException re)
 +    {
 +        // Do nothing.
 +    }
 +}
 +
 +/** STATEMENTS **/
 +
 +cqlStatement returns [ParsedStatement stmt]
 +    @after{ if (stmt != null) stmt.setBoundVariables(bindVariables); }
 +    : st1= selectStatement                 { $stmt = st1; }
 +    | st2= insertStatement                 { $stmt = st2; }
 +    | st3= updateStatement                 { $stmt = st3; }
 +    | st4= batchStatement                  { $stmt = st4; }
 +    | st5= deleteStatement                 { $stmt = st5; }
 +    | st6= useStatement                    { $stmt = st6; }
 +    | st7= truncateStatement               { $stmt = st7; }
 +    | st8= createKeyspaceStatement         { $stmt = st8; }
 +    | st9= createTableStatement            { $stmt = st9; }
 +    | st10=createIndexStatement            { $stmt = st10; }
 +    | st11=dropKeyspaceStatement           { $stmt = st11; }
 +    | st12=dropTableStatement              { $stmt = st12; }
 +    | st13=dropIndexStatement              { $stmt = st13; }
 +    | st14=alterTableStatement             { $stmt = st14; }
 +    | st15=alterKeyspaceStatement          { $stmt = st15; }
 +    | st16=grantPermissionsStatement       { $stmt = st16; }
 +    | st17=revokePermissionsStatement      { $stmt = st17; }
 +    | st18=listPermissionsStatement        { $stmt = st18; }
 +    | st19=createUserStatement             { $stmt = st19; }
 +    | st20=alterUserStatement              { $stmt = st20; }
 +    | st21=dropUserStatement               { $stmt = st21; }
 +    | st22=listUsersStatement              { $stmt = st22; }
 +    | st23=createTriggerStatement          { $stmt = st23; }
 +    | st24=dropTriggerStatement            { $stmt = st24; }
 +    | st25=createTypeStatement             { $stmt = st25; }
 +    | st26=alterTypeStatement              { $stmt = st26; }
 +    | st27=dropTypeStatement               { $stmt = st27; }
 +    | st28=createFunctionStatement         { $stmt = st28; }
 +    | st29=dropFunctionStatement           { $stmt = st29; }
 +    | st30=createAggregateStatement        { $stmt = st30; }
 +    | st31=dropAggregateStatement          { $stmt = st31; }
 +    | st32=createRoleStatement             { $stmt = st32; }
 +    | st33=alterRoleStatement              { $stmt = st33; }
 +    | st34=dropRoleStatement               { $stmt = st34; }
 +    | st35=listRolesStatement              { $stmt = st35; }
 +    | st36=grantRoleStatement              { $stmt = st36; }
 +    | st37=revokeRoleStatement             { $stmt = st37; }
 +    | st38=createMaterializedViewStatement { $stmt = st38; }
 +    | st39=dropMaterializedViewStatement   { $stmt = st39; }
 +    | st40=alterMaterializedViewStatement  { $stmt = st40; }
 +    ;
 +
 +/*
 + * USE <KEYSPACE>;
 + */
 +useStatement returns [UseStatement stmt]
 +    : K_USE ks=keyspaceName { $stmt = new UseStatement(ks); }
 +    ;
 +
 +/**
 + * SELECT <expression>
 + * FROM <CF>
 + * WHERE KEY = "key1" AND COL > 1 AND COL < 100
 + * LIMIT <NUMBER>;
 + */
 +selectStatement returns [SelectStatement.RawStatement expr]
 +    @init {
 +        boolean isDistinct = false;
 +        Term.Raw limit = null;
 +        Term.Raw perPartitionLimit = null;
 +        Map<ColumnDefinition.Raw, Boolean> orderings = new LinkedHashMap<>();
 +        List<ColumnDefinition.Raw> groups = new ArrayList<>();
 +        boolean allowFiltering = false;
 +        boolean isJson = false;
 +    }
 +    : K_SELECT
 +      ( K_JSON { isJson = true; } )?
 +      ( ( K_DISTINCT { isDistinct = true; } )? sclause=selectClause )
 +      K_FROM cf=columnFamilyName
 +      ( K_WHERE wclause=whereClause )?
 +      ( K_GROUP K_BY groupByClause[groups] ( ',' groupByClause[groups] )* )?
 +      ( K_ORDER K_BY orderByClause[orderings] ( ',' orderByClause[orderings] )* )?
 +      ( K_PER K_PARTITION K_LIMIT rows=intValue { perPartitionLimit = rows; } )?
 +      ( K_LIMIT rows=intValue { limit = rows; } )?
 +      ( K_ALLOW K_FILTERING  { allowFiltering = true; } )?
 +      {
 +          SelectStatement.Parameters params = new SelectStatement.Parameters(orderings,
 +                                                                             groups,
 +                                                                             isDistinct,
 +                                                                             allowFiltering,
 +                                                                             isJson);
 +          WhereClause where = wclause == null ? WhereClause.empty() : wclause.build();
 +          $expr = new SelectStatement.RawStatement(cf, params, sclause, where, limit, perPartitionLimit);
 +      }
 +    ;
 +
 +selectClause returns [List<RawSelector> expr]
 +    : t1=selector { $expr = new ArrayList<RawSelector>(); $expr.add(t1); } (',' tN=selector { $expr.add(tN); })*
 +    | '\*' { $expr = Collections.<RawSelector>emptyList();}
 +    ;
 +
 +selector returns [RawSelector s]
 +    @init{ ColumnIdentifier alias = null; }
 +    : us=unaliasedSelector (K_AS c=noncol_ident { alias = c; })? { $s = new RawSelector(us, alias); }
 +    ;
 +
 +/*
 + * A single selection. The core of it is selecting a column, but we also allow any term and function, as well as
 + * sub-element selection for UDT.
 + */
 +unaliasedSelector returns [Selectable.Raw s]
 +    @init { Selectable.Raw tmp = null; }
 +    :  ( c=cident                                  { tmp = c; }
 +       | v=value                                   { tmp = new Selectable.WithTerm.Raw(v); }
 +       | '(' ct=comparatorType ')' v=value         { tmp = new Selectable.WithTerm.Raw(new TypeCast(ct, v)); }
 +       | K_COUNT '(' '\*' ')'                      { tmp = Selectable.WithFunction.Raw.newCountRowsFunction(); }
 +       | K_WRITETIME '(' c=cident ')'              { tmp = new Selectable.WritetimeOrTTL.Raw(c, true); }
 +       | K_TTL       '(' c=cident ')'              { tmp = new Selectable.WritetimeOrTTL.Raw(c, false); }
 +       | K_CAST      '(' sn=unaliasedSelector K_AS t=native_type ')' {tmp = new Selectable.WithCast.Raw(sn, t);}
 +       | f=functionName args=selectionFunctionArgs { tmp = new Selectable.WithFunction.Raw(f, args); }
 +       ) ( '.' fi=fident { tmp = new Selectable.WithFieldSelection.Raw(tmp, fi); } )* { $s = tmp; }
 +    ;
 +
 +selectionFunctionArgs returns [List<Selectable.Raw> a]
 +    : '(' ')' { $a = Collections.emptyList(); }
 +    | '(' s1=unaliasedSelector { List<Selectable.Raw> args = new ArrayList<Selectable.Raw>(); args.add(s1); }
 +          ( ',' sn=unaliasedSelector { args.add(sn); } )*
 +      ')' { $a = args; }
 +    ;
 +
 +whereClause returns [WhereClause.Builder clause]
 +    @init{ $clause = new WhereClause.Builder(); }
 +    : relationOrExpression[$clause] (K_AND relationOrExpression[$clause])*
 +    ;
 +
 +relationOrExpression [WhereClause.Builder clause]
 +    : relation[$clause]
 +    | customIndexExpression[$clause]
 +    ;
 +
 +customIndexExpression [WhereClause.Builder clause]
 +    @init{IndexName name = new IndexName();}
 +    : 'expr(' idxName[name] ',' t=term ')' { clause.add(new CustomIndexExpression(name, t));}
 +    ;
 +
 +orderByClause[Map<ColumnDefinition.Raw, Boolean> orderings]
 +    @init{
 +        boolean reversed = false;
 +    }
 +    : c=cident (K_ASC | K_DESC { reversed = true; })? { orderings.put(c, reversed); }
 +    ;
 +
 +groupByClause[List<ColumnDefinition.Raw> groups]
 +    : c=cident { groups.add(c); }
 +    ;
 +
 +/**
 + * INSERT INTO <CF> (<column>, <column>, <column>, ...)
 + * VALUES (<value>, <value>, <value>, ...)
 + * USING TIMESTAMP <long>;
 + *
 + */
 +insertStatement returns [ModificationStatement.Parsed expr]
 +    : K_INSERT K_INTO cf=columnFamilyName
 +        ( st1=normalInsertStatement[cf] { $expr = st1; }
 +        | K_JSON st2=jsonInsertStatement[cf] { $expr = st2; })
 +    ;
 +
 +normalInsertStatement [CFName cf] returns [UpdateStatement.ParsedInsert expr]
 +    @init {
 +        Attributes.Raw attrs = new Attributes.Raw();
 +        List<ColumnDefinition.Raw> columnNames  = new ArrayList<>();
 +        List<Term.Raw> values = new ArrayList<>();
 +        boolean ifNotExists = false;
 +    }
 +    : '(' c1=cident { columnNames.add(c1); }  ( ',' cn=cident { columnNames.add(cn); } )* ')'
 +      K_VALUES
 +      '(' v1=term { values.add(v1); } ( ',' vn=term { values.add(vn); } )* ')'
 +      ( K_IF K_NOT K_EXISTS { ifNotExists = true; } )?
 +      ( usingClause[attrs] )?
 +      {
 +          $expr = new UpdateStatement.ParsedInsert(cf, attrs, columnNames, values, ifNotExists);
 +      }
 +    ;
 +
 +jsonInsertStatement [CFName cf] returns [UpdateStatement.ParsedInsertJson expr]
 +    @init {
 +        Attributes.Raw attrs = new Attributes.Raw();
 +        boolean ifNotExists = false;
 +        boolean defaultUnset = false;
 +    }
 +    : val=jsonValue
 +      ( K_DEFAULT ( K_NULL | ( { defaultUnset = true; } K_UNSET) ) )?
 +      ( K_IF K_NOT K_EXISTS { ifNotExists = true; } )?
 +      ( usingClause[attrs] )?
 +      {
 +          $expr = new UpdateStatement.ParsedInsertJson(cf, attrs, val, defaultUnset, ifNotExists);
 +      }
 +    ;
 +
 +jsonValue returns [Json.Raw value]
 +    : s=STRING_LITERAL { $value = new Json.Literal($s.text); }
 +    | ':' id=noncol_ident     { $value = newJsonBindVariables(id); }
 +    | QMARK            { $value = newJsonBindVariables(null); }
 +    ;
 +
 +usingClause[Attributes.Raw attrs]
 +    : K_USING usingClauseObjective[attrs] ( K_AND usingClauseObjective[attrs] )*
 +    ;
 +
 +usingClauseObjective[Attributes.Raw attrs]
 +    : K_TIMESTAMP ts=intValue { attrs.timestamp = ts; }
 +    | K_TTL t=intValue { attrs.timeToLive = t; }
 +    ;
 +
 +/**
 + * UPDATE <CF>
 + * USING TIMESTAMP <long>
 + * SET name1 = value1, name2 = value2
 + * WHERE key = value;
 + * [IF (EXISTS | name = value, ...)];
 + */
 +updateStatement returns [UpdateStatement.ParsedUpdate expr]
 +    @init {
 +        Attributes.Raw attrs = new Attributes.Raw();
 +        List<Pair<ColumnDefinition.Raw, Operation.RawUpdate>> operations = new ArrayList<>();
 +        boolean ifExists = false;
 +    }
 +    : K_UPDATE cf=columnFamilyName
 +      ( usingClause[attrs] )?
 +      K_SET columnOperation[operations] (',' columnOperation[operations])*
 +      K_WHERE wclause=whereClause
 +      ( K_IF ( K_EXISTS { ifExists = true; } | conditions=updateConditions ))?
 +      {
 +          $expr = new UpdateStatement.ParsedUpdate(cf,
 +                                                   attrs,
 +                                                   operations,
 +                                                   wclause.build(),
 +                                                   conditions == null ? Collections.<Pair<ColumnDefinition.Raw, ColumnCondition.Raw>>emptyList() : conditions,
 +                                                   ifExists);
 +     }
 +    ;
 +
 +updateConditions returns [List<Pair<ColumnDefinition.Raw, ColumnCondition.Raw>> conditions]
 +    @init { conditions = new ArrayList<Pair<ColumnDefinition.Raw, ColumnCondition.Raw>>(); }
 +    : columnCondition[conditions] ( K_AND columnCondition[conditions] )*
 +    ;
 +
 +
 +/**
 + * DELETE name1, name2
 + * FROM <CF>
 + * USING TIMESTAMP <long>
 + * WHERE KEY = keyname
 +   [IF (EXISTS | name = value, ...)];
 + */
 +deleteStatement returns [DeleteStatement.Parsed expr]
 +    @init {
 +        Attributes.Raw attrs = new Attributes.Raw();
 +        List<Operation.RawDeletion> columnDeletions = Collections.emptyList();
 +        boolean ifExists = false;
 +    }
 +    : K_DELETE ( dels=deleteSelection { columnDeletions = dels; } )?
 +      K_FROM cf=columnFamilyName
 +      ( usingClauseDelete[attrs] )?
 +      K_WHERE wclause=whereClause
 +      ( K_IF ( K_EXISTS { ifExists = true; } | conditions=updateConditions ))?
 +      {
 +          $expr = new DeleteStatement.Parsed(cf,
 +                                             attrs,
 +                                             columnDeletions,
 +                                             wclause.build(),
 +                                             conditions == null ? Collections.<Pair<ColumnDefinition.Raw, ColumnCondition.Raw>>emptyList() : conditions,
 +                                             ifExists);
 +      }
 +    ;
 +
 +deleteSelection returns [List<Operation.RawDeletion> operations]
 +    : { $operations = new ArrayList<Operation.RawDeletion>(); }
 +          t1=deleteOp { $operations.add(t1); }
 +          (',' tN=deleteOp { $operations.add(tN); })*
 +    ;
 +
 +deleteOp returns [Operation.RawDeletion op]
 +    : c=cident                { $op = new Operation.ColumnDeletion(c); }
 +    | c=cident '[' t=term ']' { $op = new Operation.ElementDeletion(c, t); }
 +    | c=cident '.' field=fident { $op = new Operation.FieldDeletion(c, field); }
 +    ;
 +
 +usingClauseDelete[Attributes.Raw attrs]
 +    : K_USING K_TIMESTAMP ts=intValue { attrs.timestamp = ts; }
 +    ;
 +
 +/**
 + * BEGIN BATCH
 + *   UPDATE <CF> SET name1 = value1 WHERE KEY = keyname1;
 + *   UPDATE <CF> SET name2 = value2 WHERE KEY = keyname2;
 + *   UPDATE <CF> SET name3 = value3 WHERE KEY = keyname3;
 + *   ...
 + * APPLY BATCH
 + *
 + * OR
 + *
 + * BEGIN BATCH
 + *   INSERT INTO <CF> (KEY, <name>) VALUES ('<key>', '<value>');
 + *   INSERT INTO <CF> (KEY, <name>) VALUES ('<key>', '<value>');
 + *   ...
 + * APPLY BATCH
 + *
 + * OR
 + *
 + * BEGIN BATCH
 + *   DELETE name1, name2 FROM <CF> WHERE key = <key>
 + *   DELETE name3, name4 FROM <CF> WHERE key = <key>
 + *   ...
 + * APPLY BATCH
 + */
 +batchStatement returns [BatchStatement.Parsed expr]
 +    @init {
 +        BatchStatement.Type type = BatchStatement.Type.LOGGED;
 +        List<ModificationStatement.Parsed> statements = new ArrayList<ModificationStatement.Parsed>();
 +        Attributes.Raw attrs = new Attributes.Raw();
 +    }
 +    : K_BEGIN
 +      ( K_UNLOGGED { type = BatchStatement.Type.UNLOGGED; } | K_COUNTER { type = BatchStatement.Type.COUNTER; } )?
 +      K_BATCH ( usingClause[attrs] )?
 +          ( s=batchStatementObjective ';'? { statements.add(s); } )*
 +      K_APPLY K_BATCH
 +      {
 +          $expr = new BatchStatement.Parsed(type, attrs, statements);
 +      }
 +    ;
 +
 +batchStatementObjective returns [ModificationStatement.Parsed statement]
 +    : i=insertStatement  { $statement = i; }
 +    | u=updateStatement  { $statement = u; }
 +    | d=deleteStatement  { $statement = d; }
 +    ;
 +
 +createAggregateStatement returns [CreateAggregateStatement expr]
 +    @init {
 +        boolean orReplace = false;
 +        boolean ifNotExists = false;
 +
 +        List<CQL3Type.Raw> argsTypes = new ArrayList<>();
 +    }
 +    : K_CREATE (K_OR K_REPLACE { orReplace = true; })?
 +      K_AGGREGATE
 +      (K_IF K_NOT K_EXISTS { ifNotExists = true; })?
 +      fn=functionName
 +      '('
 +        (
 +          v=comparatorType { argsTypes.add(v); }
 +          ( ',' v=comparatorType { argsTypes.add(v); } )*
 +        )?
 +      ')'
 +      K_SFUNC sfunc = allowedFunctionName
 +      K_STYPE stype = comparatorType
 +      (
 +        K_FINALFUNC ffunc = allowedFunctionName
 +      )?
 +      (
 +        K_INITCOND ival = term
 +      )?
 +      { $expr = new CreateAggregateStatement(fn, argsTypes, sfunc, stype, ffunc, ival, orReplace, ifNotExists); }
 +    ;
 +
 +dropAggregateStatement returns [DropAggregateStatement expr]
 +    @init {
 +        boolean ifExists = false;
 +        List<CQL3Type.Raw> argsTypes = new ArrayList<>();
 +        boolean argsPresent = false;
 +    }
 +    : K_DROP K_AGGREGATE
 +      (K_IF K_EXISTS { ifExists = true; } )?
 +      fn=functionName
 +      (
 +        '('
 +          (
 +            v=comparatorType { argsTypes.add(v); }
 +            ( ',' v=comparatorType { argsTypes.add(v); } )*
 +          )?
 +        ')'
 +        { argsPresent = true; }
 +      )?
 +      { $expr = new DropAggregateStatement(fn, argsTypes, argsPresent, ifExists); }
 +    ;
 +
 +createFunctionStatement returns [CreateFunctionStatement expr]
 +    @init {
 +        boolean orReplace = false;
 +        boolean ifNotExists = false;
 +
 +        List<ColumnIdentifier> argsNames = new ArrayList<>();
 +        List<CQL3Type.Raw> argsTypes = new ArrayList<>();
 +        boolean calledOnNullInput = false;
 +    }
 +    : K_CREATE (K_OR K_REPLACE { orReplace = true; })?
 +      K_FUNCTION
 +      (K_IF K_NOT K_EXISTS { ifNotExists = true; })?
 +      fn=functionName
 +      '('
 +        (
 +          k=noncol_ident v=comparatorType { argsNames.add(k); argsTypes.add(v); }
 +          ( ',' k=noncol_ident v=comparatorType { argsNames.add(k); argsTypes.add(v); } )*
 +        )?
 +      ')'
 +      ( (K_RETURNS K_NULL) | (K_CALLED { calledOnNullInput=true; })) K_ON K_NULL K_INPUT
 +      K_RETURNS rt = comparatorType
 +      K_LANGUAGE language = IDENT
 +      K_AS body = STRING_LITERAL
 +      { $expr = new CreateFunctionStatement(fn, $language.text.toLowerCase(), $body.text,
 +                                            argsNames, argsTypes, rt, calledOnNullInput, orReplace, ifNotExists); }
 +    ;
 +
 +dropFunctionStatement returns [DropFunctionStatement expr]
 +    @init {
 +        boolean ifExists = false;
 +        List<CQL3Type.Raw> argsTypes = new ArrayList<>();
 +        boolean argsPresent = false;
 +    }
 +    : K_DROP K_FUNCTION
 +      (K_IF K_EXISTS { ifExists = true; } )?
 +      fn=functionName
 +      (
 +        '('
 +          (
 +            v=comparatorType { argsTypes.add(v); }
 +            ( ',' v=comparatorType { argsTypes.add(v); } )*
 +          )?
 +        ')'
 +        { argsPresent = true; }
 +      )?
 +      { $expr = new DropFunctionStatement(fn, argsTypes, argsPresent, ifExists); }
 +    ;
 +
 +/**
 + * CREATE KEYSPACE [IF NOT EXISTS] <KEYSPACE> WITH attr1 = value1 AND attr2 = value2;
 + */
 +createKeyspaceStatement returns [CreateKeyspaceStatement expr]
 +    @init {
 +        KeyspaceAttributes attrs = new KeyspaceAttributes();
 +        boolean ifNotExists = false;
 +    }
 +    : K_CREATE K_KEYSPACE (K_IF K_NOT K_EXISTS { ifNotExists = true; } )? ks=keyspaceName
 +      K_WITH properties[attrs] { $expr = new CreateKeyspaceStatement(ks, attrs, ifNotExists); }
 +    ;
 +
 +/**
 + * CREATE COLUMNFAMILY [IF NOT EXISTS] <CF> (
 + *     <name1> <type>,
 + *     <name2> <type>,
 + *     <name3> <type>
 + * ) WITH <property> = <value> AND ...;
 + */
 +createTableStatement returns [CreateTableStatement.RawStatement expr]
 +    @init { boolean ifNotExists = false; }
 +    : K_CREATE K_COLUMNFAMILY (K_IF K_NOT K_EXISTS { ifNotExists = true; } )?
 +      cf=columnFamilyName { $expr = new CreateTableStatement.RawStatement(cf, ifNotExists); }
 +      cfamDefinition[expr]
 +    ;
 +
 +cfamDefinition[CreateTableStatement.RawStatement expr]
 +    : '(' cfamColumns[expr] ( ',' cfamColumns[expr]? )* ')'
 +      ( K_WITH cfamProperty[expr.properties] ( K_AND cfamProperty[expr.properties] )*)?
 +    ;
 +
 +cfamColumns[CreateTableStatement.RawStatement expr]
 +    : k=ident v=comparatorType { boolean isStatic=false; } (K_STATIC {isStatic = true;})? { $expr.addDefinition(k, v, isStatic); }
 +        (K_PRIMARY K_KEY { $expr.addKeyAliases(Collections.singletonList(k)); })?
 +    | K_PRIMARY K_KEY '(' pkDef[expr] (',' c=ident { $expr.addColumnAlias(c); } )* ')'
 +    ;
 +
 +pkDef[CreateTableStatement.RawStatement expr]
 +    : k=ident { $expr.addKeyAliases(Collections.singletonList(k)); }
 +    | '(' { List<ColumnIdentifier> l = new ArrayList<ColumnIdentifier>(); } k1=ident { l.add(k1); } ( ',' kn=ident { l.add(kn); } )* ')' { $expr.addKeyAliases(l); }
 +    ;
 +
 +cfamProperty[CFProperties props]
 +    : property[props.properties]
 +    | K_COMPACT K_STORAGE { $props.setCompactStorage(); }
 +    | K_CLUSTERING K_ORDER K_BY '(' cfamOrdering[props] (',' cfamOrdering[props])* ')'
 +    ;
 +
 +cfamOrdering[CFProperties props]
 +    @init{ boolean reversed=false; }
 +    : k=ident (K_ASC | K_DESC { reversed=true;} ) { $props.setOrdering(k, reversed); }
 +    ;
 +
 +
 +/**
 + * CREATE TYPE foo (
 + *    <name1> <type1>,
 + *    <name2> <type2>,
 + *    ....
 + * )
 + */
 +createTypeStatement returns [CreateTypeStatement expr]
 +    @init { boolean ifNotExists = false; }
 +    : K_CREATE K_TYPE (K_IF K_NOT K_EXISTS { ifNotExists = true; } )?
 +         tn=userTypeName { $expr = new CreateTypeStatement(tn, ifNotExists); }
 +         '(' typeColumns[expr] ( ',' typeColumns[expr]? )* ')'
 +    ;
 +
 +typeColumns[CreateTypeStatement expr]
 +    : k=fident v=comparatorType { $expr.addDefinition(k, v); }
 +    ;
 +
 +
 +/**
 + * CREATE INDEX [IF NOT EXISTS] [indexName] ON <columnFamily> (<columnName>);
 + * CREATE CUSTOM INDEX [IF NOT EXISTS] [indexName] ON <columnFamily> (<columnName>) USING <indexClass>;
 + */
 +createIndexStatement returns [CreateIndexStatement expr]
 +    @init {
 +        IndexPropDefs props = new IndexPropDefs();
 +        boolean ifNotExists = false;
 +        IndexName name = new IndexName();
 +        List<IndexTarget.Raw> targets = new ArrayList<>();
 +    }
 +    : K_CREATE (K_CUSTOM { props.isCustom = true; })? K_INDEX (K_IF K_NOT K_EXISTS { ifNotExists = true; } )?
 +        (idxName[name])? K_ON cf=columnFamilyName '(' (indexIdent[targets] (',' indexIdent[targets])*)? ')'
 +        (K_USING cls=STRING_LITERAL { props.customClass = $cls.text; })?
 +        (K_WITH properties[props])?
 +      { $expr = new CreateIndexStatement(cf, name, targets, props, ifNotExists); }
 +    ;
 +
 +indexIdent [List<IndexTarget.Raw> targets]
 +    : c=cident                   { $targets.add(IndexTarget.Raw.simpleIndexOn(c)); }
 +    | K_VALUES '(' c=cident ')'  { $targets.add(IndexTarget.Raw.valuesOf(c)); }
 +    | K_KEYS '(' c=cident ')'    { $targets.add(IndexTarget.Raw.keysOf(c)); }
 +    | K_ENTRIES '(' c=cident ')' { $targets.add(IndexTarget.Raw.keysAndValuesOf(c)); }
 +    | K_FULL '(' c=cident ')'    { $targets.add(IndexTarget.Raw.fullCollection(c)); }
 +    ;
 +
 +/**
 + * CREATE MATERIALIZED VIEW <viewName> AS
 + *  SELECT <columns>
 + *  FROM <CF>
 + *  WHERE <pkColumns> IS NOT NULL
 + *  PRIMARY KEY (<pkColumns>)
 + *  WITH <property> = <value> AND ...;
 + */
 +createMaterializedViewStatement returns [CreateViewStatement expr]
 +    @init {
 +        boolean ifNotExists = false;
 +        List<ColumnDefinition.Raw> partitionKeys = new ArrayList<>();
 +        List<ColumnDefinition.Raw> compositeKeys = new ArrayList<>();
 +    }
 +    : K_CREATE K_MATERIALIZED K_VIEW (K_IF K_NOT K_EXISTS { ifNotExists = true; })? cf=columnFamilyName K_AS
 +        K_SELECT sclause=selectClause K_FROM basecf=columnFamilyName
 +        (K_WHERE wclause=whereClause)?
 +        K_PRIMARY K_KEY (
 +        '(' '(' k1=cident { partitionKeys.add(k1); } ( ',' kn=cident { partitionKeys.add(kn); } )* ')' ( ',' c1=cident { compositeKeys.add(c1); } )* ')'
 +    |   '(' k1=cident { partitionKeys.add(k1); } ( ',' cn=cident { compositeKeys.add(cn); } )* ')'
 +        )
 +        {
 +             WhereClause where = wclause == null ? WhereClause.empty() : wclause.build();
 +             $expr = new CreateViewStatement(cf, basecf, sclause, where, partitionKeys, compositeKeys, ifNotExists);
 +        }
 +        ( K_WITH cfamProperty[expr.properties] ( K_AND cfamProperty[expr.properties] )*)?
 +    ;
 +
 +/**
 + * CREATE TRIGGER triggerName ON columnFamily USING 'triggerClass';
 + */
 +createTriggerStatement returns [CreateTriggerStatement expr]
 +    @init {
 +        boolean ifNotExists = false;
 +    }
-     : K_CREATE K_TRIGGER (K_IF K_NOT K_EXISTS { ifNotExists = true; } )? (name=cident)
++    : K_CREATE K_TRIGGER (K_IF K_NOT K_EXISTS { ifNotExists = true; } )? (name=ident)
 +        K_ON cf=columnFamilyName K_USING cls=STRING_LITERAL
-       { $expr = new CreateTriggerStatement(cf, name.rawText(), $cls.text, ifNotExists); }
++      { $expr = new CreateTriggerStatement(cf, name.toString(), $cls.text, ifNotExists); }
 +    ;
 +
 +/**
 + * DROP TRIGGER [IF EXISTS] triggerName ON columnFamily;
 + */
 +dropTriggerStatement returns [DropTriggerStatement expr]
 +     @init { boolean ifExists = false; }
-     : K_DROP K_TRIGGER (K_IF K_EXISTS { ifExists = true; } )? (name=cident) K_ON cf=columnFamilyName
-       { $expr = new DropTriggerStatement(cf, name.rawText(), ifExists); }
++    : K_DROP K_TRIGGER (K_IF K_EXISTS { ifExists = true; } )? (name=ident) K_ON cf=columnFamilyName
++      { $expr = new DropTriggerStatement(cf, name.toString(), ifExists); }
 +    ;
 +
 +/**
 + * ALTER KEYSPACE <KS> WITH <property> = <value>;
 + */
 +alterKeyspaceStatement returns [AlterKeyspaceStatement expr]
 +    @init { KeyspaceAttributes attrs = new KeyspaceAttributes(); }
 +    : K_ALTER K_KEYSPACE ks=keyspaceName
 +        K_WITH properties[attrs] { $expr = new AlterKeyspaceStatement(ks, attrs); }
 +    ;
 +
 +/**
 + * ALTER COLUMN FAMILY <CF> ALTER <column> TYPE <newtype>;
 + * ALTER COLUMN FAMILY <CF> ADD <column> <newtype>; | ALTER COLUMN FAMILY <CF> ADD (<column> <newtype>,<column1> <newtype1>..... <column n> <newtype n>)
 + * ALTER COLUMN FAMILY <CF> DROP <column>; | ALTER COLUMN FAMILY <CF> DROP ( <column>,<column1>.....<column n>)
 + * ALTER COLUMN FAMILY <CF> WITH <property> = <value>;
 + * ALTER COLUMN FAMILY <CF> RENAME <column> TO <column>;
 + */
 +alterTableStatement returns [AlterTableStatement expr]
 +    @init {
 +        AlterTableStatement.Type type = null;
 +        TableAttributes attrs = new TableAttributes();
 +        Map<ColumnDefinition.Raw, ColumnDefinition.Raw> renames = new HashMap<ColumnDefinition.Raw, ColumnDefinition.Raw>();
 +        List<AlterTableStatementColumn> colNameList = new ArrayList<AlterTableStatementColumn>();
 +        Long deleteTimestamp = null;
 +    }
 +    : K_ALTER K_COLUMNFAMILY cf=columnFamilyName
-           ( K_ALTER id=cident  K_TYPE v=comparatorType  { type = AlterTableStatement.Type.ALTER; } { colNameList.add(new AlterTableStatementColumn(id,v)); }
-           | K_ADD  (        (id=cident   v=comparatorType   b1=cfisStatic { colNameList.add(new AlterTableStatementColumn(id,v,b1)); })
-                      | ('('  id1=cident  v1=comparatorType  b1=cfisStatic { colNameList.add(new AlterTableStatementColumn(id1,v1,b1)); }
-                        ( ',' idn=cident  vn=comparatorType  bn=cfisStatic { colNameList.add(new AlterTableStatementColumn(idn,vn,bn)); } )* ')' ) ) { type = AlterTableStatement.Type.ADD; }
-           | K_DROP ( (         id=cident  { colNameList.add(new AlterTableStatementColumn(id)); }
-                       | ('('  id1=cident { colNameList.add(new AlterTableStatementColumn(id1)); }
-                         ( ',' idn=cident { colNameList.add(new AlterTableStatementColumn(idn)); } )* ')') )
++          ( K_ALTER id=schema_cident  K_TYPE v=comparatorType  { type = AlterTableStatement.Type.ALTER; } { colNameList.add(new AlterTableStatementColumn(id,v)); }
++          | K_ADD  (        (aid=schema_cident  v=comparatorType   b1=cfisStatic { colNameList.add(new AlterTableStatementColumn(aid,v,b1)); })
++                     | ('('  id1=schema_cident  v1=comparatorType  b1=cfisStatic { colNameList.add(new AlterTableStatementColumn(id1,v1,b1)); }
++                       ( ',' idn=schema_cident  vn=comparatorType  bn=cfisStatic { colNameList.add(new AlterTableStatementColumn(idn,vn,bn)); } )* ')' ) ) { type = AlterTableStatement.Type.ADD; }
++          | K_DROP K_COMPACT K_STORAGE          { type = AlterTableStatement.Type.DROP_COMPACT_STORAGE; }        
++          | K_DROP ( (        id=schema_cident  { colNameList.add(new AlterTableStatementColumn(id)); }
++                      | ('('  id1=schema_cident { colNameList.add(new AlterTableStatementColumn(id1)); }
++                        ( ',' idn=schema_cident { colNameList.add(new AlterTableStatementColumn(idn)); } )* ')') )
 +                     ( K_USING K_TIMESTAMP t=INTEGER { deleteTimestamp = Long.parseLong(Constants.Literal.integer($t.text).getText()); })? ) { type = AlterTableStatement.Type.DROP; }
 +          | K_WITH  properties[attrs]                 { type = AlterTableStatement.Type.OPTS; }
 +          | K_RENAME                                  { type = AlterTableStatement.Type.RENAME; }
-                id1=cident K_TO toId1=cident { renames.put(id1, toId1); }
-                ( K_AND idn=cident K_TO toIdn=cident { renames.put(idn, toIdn); } )*
++               id1=schema_cident K_TO toId1=schema_cident { renames.put(id1, toId1); }
++               ( K_AND idn=schema_cident K_TO toIdn=schema_cident { renames.put(idn, toIdn); } )*
 +          )
 +    {
 +        $expr = new AlterTableStatement(cf, type, colNameList, attrs, renames, deleteTimestamp);
 +    }
 +    ;
 +
 +cfisStatic returns [boolean isStaticColumn]
 +    @init{
 +        boolean isStatic = false;
 +    }
 +    : (K_STATIC { isStatic=true; })? { $isStaticColumn = isStatic;
 +    }
 +    ;
 +
 +alterMaterializedViewStatement returns [AlterViewStatement expr]
 +    @init {
 +        TableAttributes attrs = new TableAttributes();
 +    }
 +    : K_ALTER K_MATERIALIZED K_VIEW name=columnFamilyName
 +          K_WITH properties[attrs]
 +    {
 +        $expr = new AlterViewStatement(name, attrs);
 +    }
 +    ;
 +
 +
 +/**
 + * ALTER TYPE <name> ALTER <field> TYPE <newtype>;
 + * ALTER TYPE <name> ADD <field> <newtype>;
 + * ALTER TYPE <name> RENAME <field> TO <newtype> AND ...;
 + */
 +alterTypeStatement returns [AlterTypeStatement expr]
 +    : K_ALTER K_TYPE name=userTypeName
 +          ( K_ALTER f=fident K_TYPE v=comparatorType { $expr = AlterTypeStatement.alter(name, f, v); }
 +          | K_ADD   f=fident v=comparatorType        { $expr = AlterTypeStatement.addition(name, f, v); }
 +          | K_RENAME
 +               { Map<FieldIdentifier, FieldIdentifier> renames = new HashMap<>(); }
 +                 id1=fident K_TO toId1=fident { renames.put(id1, toId1); }
 +                 ( K_AND idn=fident K_TO toIdn=fident { renames.put(idn, toIdn); } )*
 +               { $expr = AlterTypeStatement.renames(name, renames); }
 +          )
 +    ;
 +
 +
 +/**
 + * DROP KEYSPACE [IF EXISTS] <KSP>;
 + */
 +dropKeyspaceStatement returns [DropKeyspaceStatement ksp]
 +    @init { boolean ifExists = false; }
 +    : K_DROP K_KEYSPACE (K_IF K_EXISTS { ifExists = true; } )? ks=keyspaceName { $ksp = new DropKeyspaceStatement(ks, ifExists); }
 +    ;
 +
 +/**
 + * DROP COLUMNFAMILY [IF EXISTS] <CF>;
 + */
 +dropTableStatement returns [DropTableStatement stmt]
 +    @init { boolean ifExists = false; }
 +    : K_DROP K_COLUMNFAMILY (K_IF K_EXISTS { ifExists = true; } )? cf=columnFamilyName { $stmt = new DropTableStatement(cf, ifExists); }
 +    ;
 +
 +/**
 + * DROP TYPE <name>;
 + */
 +dropTypeStatement returns [DropTypeStatement stmt]
 +    @init { boolean ifExists = false; }
 +    : K_DROP K_TYPE (K_IF K_EXISTS { ifExists = true; } )? name=userTypeName { $stmt = new DropTypeStatement(name, ifExists); }
 +    ;
 +
 +/**
 + * DROP INDEX [IF EXISTS] <INDEX_NAME>
 + */
 +dropIndexStatement returns [DropIndexStatement expr]
 +    @init { boolean ifExists = false; }
 +    : K_DROP K_INDEX (K_IF K_EXISTS { ifExists = true; } )? index=indexName
 +      { $expr = new DropIndexStatement(index, ifExists); }
 +    ;
 +
 +/**
 + * DROP MATERIALIZED VIEW [IF EXISTS] <view_name>
 + */
 +dropMaterializedViewStatement returns [DropViewStatement expr]
 +    @init { boolean ifExists = false; }
 +    : K_DROP K_MATERIALIZED K_VIEW (K_IF K_EXISTS { ifExists = true; } )? cf=columnFamilyName
 +      { $expr = new DropViewStatement(cf, ifExists); }
 +    ;
 +
 +/**
 +  * TRUNCATE <CF>;
 +  */
 +truncateStatement returns [TruncateStatement stmt]
 +    : K_TRUNCATE (K_COLUMNFAMILY)? cf=columnFamilyName { $stmt = new TruncateStatement(cf); }
 +    ;
 +
 +/**
 + * GRANT <permission> ON <resource> TO <rolename>
 + */
 +grantPermissionsStatement returns [GrantPermissionsStatement stmt]
 +    : K_GRANT
 +          permissionOrAll
 +      K_ON
 +          resource
 +      K_TO
 +          grantee=userOrRoleName
 +      { $stmt = new GrantPermissionsStatement(filterPermissions($permissionOrAll.perms, $resource.res), $resource.res, grantee); }
 +    ;
 +
 +/**
 + * REVOKE <permission> ON <resource> FROM <rolename>
 + */
 +revokePermissionsStatement returns [RevokePermissionsStatement stmt]
 +    : K_REVOKE
 +          permissionOrAll
 +      K_ON
 +          resource
 +      K_FROM
 +          revokee=userOrRoleName
 +      { $stmt = new RevokePermissionsStatement(filterPermissions($permissionOrAll.perms, $resource.res), $resource.res, revokee); }
 +    ;
 +
 +/**
 + * GRANT ROLE <rolename> TO <grantee>
 + */
 +grantRoleStatement returns [GrantRoleStatement stmt]
 +    : K_GRANT
 +          role=userOrRoleName
 +      K_TO
 +          grantee=userOrRoleName
 +      { $stmt = new GrantRoleStatement(role, grantee); }
 +    ;
 +
 +/**
 + * REVOKE ROLE <rolename> FROM <revokee>
 + */
 +revokeRoleStatement returns [RevokeRoleStatement stmt]
 +    : K_REVOKE
 +          role=userOrRoleName
 +      K_FROM
 +          revokee=userOrRoleName
 +      { $stmt = new RevokeRoleStatement(role, revokee); }
 +    ;
 +
 +listPermissionsStatement returns [ListPermissionsStatement stmt]
 +    @init {
 +        IResource resource = null;
 +        boolean recursive = true;
 +        RoleName grantee = new RoleName();
 +    }
 +    : K_LIST
 +          permissionOrAll
 +      ( K_ON resource { resource = $resource.res; } )?
 +      ( K_OF roleName[grantee] )?
 +      ( K_NORECURSIVE { recursive = false; } )?
 +      { $stmt = new ListPermissionsStatement($permissionOrAll.perms, resource, grantee, recursive); }
 +    ;
 +
 +permission returns [Permission perm]
 +    : p=(K_CREATE | K_ALTER | K_DROP | K_SELECT | K_MODIFY | K_AUTHORIZE | K_DESCRIBE | K_EXECUTE)
 +    { $perm = Permission.valueOf($p.text.toUpperCase()); }
 +    ;
 +
 +permissionOrAll returns [Set<Permission> perms]
 +    : K_ALL ( K_PERMISSIONS )?       { $perms = Permission.ALL; }
 +    | p=permission ( K_PERMISSION )? { $perms = EnumSet.of($p.perm); }
 +    ;
 +
 +resource returns [IResource res]
 +    : d=dataResource { $res = $d.res; }
 +    | r=roleResource { $res = $r.res; }
 +    | f=functionResource { $res = $f.res; }
 +    | j=jmxResource { $res = $j.res; }
 +    ;
 +
 +dataResource returns [DataResource res]
 +    : K_ALL K_KEYSPACES { $res = DataResource.root(); }
 +    | K_KEYSPACE ks = keyspaceName { $res = DataResource.keyspace($ks.id); }
 +    | ( K_COLUMNFAMILY )? cf = columnFamilyName
 +      { $res = DataResource.table($cf.name.getKeyspace(), $cf.name.getColumnFamily()); }
 +    ;
 +
 +jmxResource returns [JMXResource res]
 +    : K_ALL K_MBEANS { $res = JMXResource.root(); }
 +    // when a bean name (or pattern) is supplied, validate that it's a legal ObjectName
 +    // also, just to be picky, if the "MBEANS" form is used, only allow a pattern style names
 +    | K_MBEAN mbean { $res = JMXResource.mbean(canonicalizeObjectName($mbean.text, false)); }
 +    | K_MBEANS mbean { $res = JMXResource.mbean(canonicalizeObjectName($mbean.text, true)); }
 +    ;
 +
 +roleResource returns [RoleResource res]
 +    : K_ALL K_ROLES { $res = RoleResource.root(); }
 +    | K_ROLE role = userOrRoleName { $res = RoleResource.role($role.name.getName()); }
 +    ;
 +
 +functionResource returns [FunctionResource res]
 +    @init {
 +        List<CQL3Type.Raw> argsTypes = new ArrayList<>();
 +    }
 +    : K_ALL K_FUNCTIONS { $res = FunctionResource.root(); }
 +    | K_ALL K_FUNCTIONS K_IN K_KEYSPACE ks = keyspaceName { $res = FunctionResource.keyspace($ks.id); }
 +    // Arg types are mandatory for DCL statements on Functions
 +    | K_FUNCTION fn=functionName
 +      (
 +        '('
 +          (
 +            v=comparatorType { argsTypes.add(v); }
 +            ( ',' v=comparatorType { argsTypes.add(v); } )*
 +          )?
 +        ')'
 +      )
 +      { $res = FunctionResource.functionFromCql($fn.s.keyspace, $fn.s.name, argsTypes); }
 +    ;
 +
 +/**
 + * CREATE USER [IF NOT EXISTS] <username> [WITH PASSWORD <password>] [SUPERUSER|NOSUPERUSER]
 + */
 +createUserStatement returns [CreateRoleStatement stmt]
 +    @init {
 +        RoleOptions opts = new RoleOptions();
 +        opts.setOption(IRoleManager.Option.LOGIN, true);
 +        boolean superuser = false;
 +        boolean ifNotExists = false;
 +        RoleName name = new RoleName();
 +    }
 +    : K_CREATE K_USER (K_IF K_NOT K_EXISTS { ifNotExists = true; })? u=username { name.setName($u.text, true); }
 +      ( K_WITH userPassword[opts] )?
 +      ( K_SUPERUSER { superuser = true; } | K_NOSUPERUSER { superuser = false; } )?
 +      { opts.setOption(IRoleManager.Option.SUPERUSER, superuser);
 +        $stmt = new CreateRoleStatement(name, opts, ifNotExists); }
 +    ;
 +
 +/**
 + * ALTER USER <username> [WITH PASSWORD <password>] [SUPERUSER|NOSUPERUSER]
 + */
 +alterUserStatement returns [AlterRoleStatement stmt]
 +    @init {
 +        RoleOptions opts = new RoleOptions();
 +        RoleName name = new RoleName();
 +    }
 +    : K_ALTER K_USER u=username { name.setName($u.text, true); }
 +      ( K_WITH userPassword[opts] )?
 +      ( K_SUPERUSER { opts.setOption(IRoleManager.Option.SUPERUSER, true); }
 +        | K_NOSUPERUSER { opts.setOption(IRoleManager.Option.SUPERUSER, false); } ) ?
 +      {  $stmt = new AlterRoleStatement(name, opts); }
 +    ;
 +
 +/**
 + * DROP USER [IF EXISTS] <username>
 + */
 +dropUserStatement returns [DropRoleStatement stmt]
 +    @init {
 +        boolean ifExists = false;
 +        RoleName name = new RoleName();
 +    }
 +    : K_DROP K_USER (K_IF K_EXISTS { ifExists = true; })? u=username { name.setName($u.text, true); $stmt = new DropRoleStatement(name, ifExists); }
 +    ;
 +
 +/**
 + * LIST USERS
 + */
 +listUsersStatement returns [ListRolesStatement stmt]
 +    : K_LIST K_USERS { $stmt = new ListUsersStatement(); }
 +    ;
 +
 +/**
 + * CREATE ROLE [IF NOT EXISTS] <rolename> [ [WITH] option [ [AND] option ]* ]
 + *
 + * where option can be:
 + *  PASSWORD = '<password>'
 + *  SUPERUSER = (true|false)
 + *  LOGIN = (true|false)
 + *  OPTIONS = { 'k1':'v1', 'k2':'v2'}
 + */
 +createRoleStatement returns [CreateRoleStatement stmt]
 +    @init {
 +        RoleOptions opts = new RoleOptions();
 +        boolean ifNotExists = false;
 +    }
 +    : K_CREATE K_ROLE (K_IF K_NOT K_EXISTS { ifNotExists = true; })? name=userOrRoleName
 +      ( K_WITH roleOptions[opts] )?
 +      {
 +        // set defaults if they weren't explictly supplied
 +        if (!opts.getLogin().isPresent())
 +        {
 +            opts.setOption(IRoleManager.Option.LOGIN, false);
 +        }
 +        if (!opts.getSuperuser().isPresent())
 +        {
 +            opts.setOption(IRoleManager.Option.SUPERUSER, false);
 +        }
 +        $stmt = new CreateRoleStatement(name, opts, ifNotExists);
 +      }
 +    ;
 +
 +/**
 + * ALTER ROLE <rolename> [ [WITH] option [ [AND] option ]* ]
 + *
 + * where option can be:
 + *  PASSWORD = '<password>'
 + *  SUPERUSER = (true|false)
 + *  LOGIN = (true|false)
 + *  OPTIONS = { 'k1':'v1', 'k2':'v2'}
 + */
 +alterRoleStatement returns [AlterRoleStatement stmt]
 +    @init {
 +        RoleOptions opts = new RoleOptions();
 +    }
 +    : K_ALTER K_ROLE name=userOrRoleName
 +      ( K_WITH roleOptions[opts] )?
 +      {  $stmt = new AlterRoleStatement(name, opts); }
 +    ;
 +
 +/**
 + * DROP ROLE [IF EXISTS] <rolename>
 + */
 +dropRoleStatement returns [DropRoleStatement stmt]
 +    @init {
 +        boolean ifExists = false;
 +    }
 +    : K_DROP K_ROLE (K_IF K_EXISTS { ifExists = true; })? name=userOrRoleName
 +      { $stmt = new DropRoleStatement(name, ifExists); }
 +    ;
 +
 +/**
 + * LIST ROLES [OF <rolename>] [NORECURSIVE]
 + */
 +listRolesStatement returns [ListRolesStatement stmt]
 +    @init {
 +        boolean recursive = true;
 +        RoleName grantee = new RoleName();
 +    }
 +    : K_LIST K_ROLES
 +      ( K_OF roleName[grantee])?
 +      ( K_NORECURSIVE { recursive = false; } )?
 +      { $stmt = new ListRolesStatement(grantee, recursive); }
 +    ;
 +
 +roleOptions[RoleOptions opts]
 +    : roleOption[opts] (K_AND roleOption[opts])*
 +    ;
 +
 +roleOption[RoleOptions opts]
 +    :  K_PASSWORD '=' v=STRING_LITERAL { opts.setOption(IRoleManager.Option.PASSWORD, $v.text); }
 +    |  K_OPTIONS '=' m=mapLiteral { opts.setOption(IRoleManager.Option.OPTIONS, convertPropertyMap(m)); }
 +    |  K_SUPERUSER '=' b=BOOLEAN { opts.setOption(IRoleManager.Option.SUPERUSER, Boolean.valueOf($b.text)); }
 +    |  K_LOGIN '=' b=BOOLEAN { opts.setOption(IRoleManager.Option.LOGIN, Boolean.valueOf($b.text)); }
 +    ;
 +
 +// for backwards compatibility in CREATE/ALTER USER, this has no '='
 +userPassword[RoleOptions opts]
 +    :  K_PASSWORD v=STRING_LITERAL { opts.setOption(IRoleManager.Option.PASSWORD, $v.text); }
 +    ;
 +
 +/** DEFINITIONS **/
 +
 +// Column Identifiers.  These need to be treated differently from other
 +// identifiers because the underlying comparator is not necessarily text. See
 +// CASSANDRA-8178 for details.
++// Also, we need to support the internal of the super column map (for backward
++// compatibility) which is empty (we only want to allow this is in data manipulation
++// queries, not in schema defition etc).
 +cident returns [ColumnDefinition.Raw id]
++    : EMPTY_QUOTED_NAME    { $id = ColumnDefinition.Raw.forQuoted(""); }
++    | t=IDENT              { $id = ColumnDefinition.Raw.forUnquoted($t.text); }
++    | t=QUOTED_NAME        { $id = ColumnDefinition.Raw.forQuoted($t.text); }
++    | k=unreserved_keyword { $id = ColumnDefinition.Raw.forUnquoted(k); }
++    ;
++
++schema_cident returns [ColumnDefinition.Raw id]
 +    : t=IDENT              { $id = ColumnDefinition.Raw.forUnquoted($t.text); }
 +    | t=QUOTED_NAME        { $id = ColumnDefinition.Raw.forQuoted($t.text); }
 +    | k=unreserved_keyword { $id = ColumnDefinition.Raw.forUnquoted(k); }
 +    ;
 +
 +// Column identifiers where the comparator is known to be text
 +ident returns [ColumnIdentifier id]
 +    : t=IDENT              { $id = ColumnIdentifier.getInterned($t.text, false); }
 +    | t=QUOTED_NAME        { $id = ColumnIdentifier.getInterned($t.text, true); }
 +    | k=unreserved_keyword { $id = ColumnIdentifier.getInterned(k, false); }
 +    ;
 +
 +fident returns [FieldIdentifier id]
 +    : t=IDENT              { $id = FieldIdentifier.forUnquoted($t.text); }
 +    | t=QUOTED_NAME        { $id = FieldIdentifier.forQuoted($t.text); }
 +    | k=unreserved_keyword { $id = FieldIdentifier.forUnquoted(k); }
 +    ;
 +
 +// Identifiers that do not refer to columns
 +noncol_ident returns [ColumnIdentifier id]
 +    : t=IDENT              { $id = new ColumnIdentifier($t.text, false); }
 +    | t=QUOTED_NAME        { $id = new ColumnIdentifier($t.text, true); }
 +    | k=unreserved_keyword { $id = new ColumnIdentifier(k, false); }
 +    ;
 +
 +// Keyspace & Column family names
 +keyspaceName returns [String id]
 +    @init { CFName name = new CFName(); }
 +    : ksName[name] { $id = name.getKeyspace(); }
 +    ;
 +
 +indexName returns [IndexName name]
 +    @init { $name = new IndexName(); }
 +    : (ksName[name] '.')? idxName[name]
 +    ;
 +
 +columnFamilyName returns [CFName name]
 +    @init { $name = new CFName(); }
 +    : (ksName[name] '.')? cfName[name]
 +    ;
 +
 +userTypeName returns [UTName name]
 +    : (ks=noncol_ident '.')? ut=non_type_ident { $name = new UTName(ks, ut); }
 +    ;
 +
 +userOrRoleName returns [RoleName name]
 +    @init { RoleName role = new RoleName(); }
 +    : roleName[role] {$name = role;}
 +    ;
 +
 +ksName[KeyspaceElementName name]
 +    : t=IDENT              { $name.setKeyspace($t.text, false);}
 +    | t=QUOTED_NAME        { $name.setKeyspace($t.text, true);}
 +    | k=unreserved_keyword { $name.setKeyspace(k, false);}
 +    | QMARK {addRecognitionError("Bind variables cannot be used for keyspace names");}
 +    ;
 +
 +cfName[CFName name]
 +    : t=IDENT              { $name.setColumnFamily($t.text, false); }
 +    | t=QUOTED_NAME        { $name.setColumnFamily($t.text, true); }
 +    | k=unreserved_keyword { $name.setColumnFamily(k, false); }
 +    | QMARK {addRecognitionError("Bind variables cannot be used for table names");}
 +    ;
 +
 +idxName[IndexName name]
 +    : t=IDENT              { $name.setIndex($t.text, false); }
 +    | t=QUOTED_NAME        { $name.setIndex($t.text, true);}
 +    | k=unreserved_keyword { $name.setIndex(k, false); }
 +    | QMARK {addRecognitionError("Bind variables cannot be used for index names");}
 +    ;
 +
 +roleName[RoleName name]
 +    : t=IDENT              { $name.setName($t.text, false); }
 +    | s=STRING_LITERAL     { $name.setName($s.text, true); }
 +    | t=QUOTED_NAME        { $name.setName($t.text, true); }
 +    | k=unreserved_keyword { $name.setName(k, false); }
 +    | QMARK {addRecognitionError("Bind variables cannot be used for role names");}
 +    ;
 +
 +constant returns [Constants.Literal constant]
 +    : t=STRING_LITERAL { $constant = Constants.Literal.string($t.text); }
 +    | t=INTEGER        { $constant = Constants.Literal.integer($t.text); }
 +    | t=FLOAT          { $constant = Constants.Literal.floatingPoint($t.text); }
 +    | t=BOOLEAN        { $constant = Constants.Literal.bool($t.text); }
 +    | t=DURATION       { $constant = Constants.Literal.duration($t.text);}
 +    | t=UUID           { $constant = Constants.Literal.uuid($t.text); }
 +    | t=HEXNUMBER      { $constant = Constants.Literal.hex($t.text); }
 +    | { String sign=""; } ('-' {sign = "-"; } )? t=(K_NAN | K_INFINITY) { $constant = Constants.Literal.floatingPoint(sign + $t.text); }
 +    ;
 +
 +mapLiteral returns [Maps.Literal map]
 +    : '{' { List<Pair<Term.Raw, Term.Raw>> m = new ArrayList<Pair<Term.Raw, Term.Raw>>(); }
 +          ( k1=term ':' v1=term { m.add(Pair.create(k1, v1)); } ( ',' kn=term ':' vn=term { m.add(Pair.create(kn, vn)); } )* )?
 +      '}' { $map = new Maps.Literal(m); }
 +    ;
 +
 +setOrMapLiteral[Term.Raw t] returns [Term.Raw value]
 +    : ':' v=term { List<Pair<Term.Raw, Term.Raw>> m = new ArrayList<Pair<Term.Raw, Term.Raw>>(); m.add(Pair.create(t, v)); }
 +          ( ',' kn=term ':' vn=term { m.add(Pair.create(kn, vn)); } )*
 +      { $value = new Maps.Literal(m); }
 +    | { List<Term.Raw> s = new ArrayList<Term.Raw>(); s.add(t); }
 +          ( ',' tn=term { s.add(tn); } )*
 +      { $value = new Sets.Literal(s); }
 +    ;
 +
 +collectionLiteral returns [Term.Raw value]
 +    : '[' { List<Term.Raw> l = new ArrayList<Term.Raw>(); }
 +          ( t1=term { l.add(t1); } ( ',' tn=term { l.add(tn); } )* )?
 +      ']' { $value = new Lists.Literal(l); }
 +    | '{' t=term v=setOrMapLiteral[t] { $value = v; } '}'
 +    // Note that we have an ambiguity between maps and set for "{}". So we force it to a set literal,
 +    // and deal with it later based on the type of the column (SetLiteral.java).
 +    | '{' '}' { $value = new Sets.Literal(Collections.<Term.Raw>emptyList()); }
 +    ;
 +
 +usertypeLiteral returns [UserTypes.Literal ut]
 +    @init{ Map<FieldIdentifier, Term.Raw> m = new HashMap<>(); }
 +    @after{ $ut = new UserTypes.Literal(m); }
 +    // We don't allow empty literals because that conflicts with sets/maps and is currently useless since we don't allow empty user types
 +    : '{' k1=fident ':' v1=term { m.put(k1, v1); } ( ',' kn=fident ':' vn=term { m.put(kn, vn); } )* '}'
 +    ;
 +
 +tupleLiteral returns [Tuples.Literal tt]
 +    @init{ List<Term.Raw> l = new ArrayList<Term.Raw>(); }
 +    @after{ $tt = new Tuples.Literal(l); }
 +    : '(' t1=term { l.add(t1); } ( ',' tn=term { l.add(tn); } )* ')'
 +    ;
 +
 +value returns [Term.Raw value]
 +    : c=constant           { $value = c; }
 +    | l=collectionLiteral  { $value = l; }
 +    | u=usertypeLiteral    { $value = u; }
 +    | t=tupleLiteral       { $value = t; }
 +    | K_NULL               { $value = Constants.NULL_LITERAL; }
 +    | ':' id=noncol_ident  { $value = newBindVariables(id); }
 +    | QMARK                { $value = newBindVariables(null); }
 +    ;
 +
 +intValue returns [Term.Raw value]
 +    : t=INTEGER     { $value = Constants.Literal.integer($t.text); }
 +    | ':' id=noncol_ident  { $value = newBindVariables(id); }
 +    | QMARK         { $value = newBindVariables(null); }
 +    ;
 +
 +functionName returns [FunctionName s]
-     : (ks=keyspaceName '.')? f=allowedFunctionName   { $s = new FunctionName(ks, f); }
++     // antlr might try to recover and give a null for f. It will still error out in the end, but FunctionName
++     // wouldn't be happy with that so we should bypass this for now or we'll have a weird user-facing error
++    : (ks=keyspaceName '.')? f=allowedFunctionName   { $s = f == null ? null : new FunctionName(ks, f); }
 +    ;
 +
 +allowedFunctionName returns [String s]
 +    : f=IDENT                       { $s = $f.text.toLowerCase(); }
 +    | f=QUOTED_NAME                 { $s = $f.text; }
 +    | u=unreserved_function_keyword { $s = u; }
 +    | K_TOKEN                       { $s = "token"; }
 +    | K_COUNT                       { $s = "count"; }
 +    ;
 +
 +function returns [Term.Raw t]
 +    : f=functionName '(' ')'                   { $t = new FunctionCall.Raw(f, Collections.<Term.Raw>emptyList()); }
 +    | f=functionName '(' args=functionArgs ')' { $t = new FunctionCall.Raw(f, args); }
 +    ;
 +
 +functionArgs returns [List<Term.Raw> args]
 +    @init{ $args = new ArrayList<Term.Raw>(); }
 +    : t1=term {args.add(t1); } ( ',' tn=term { args.add(tn); } )*
 +    ;
 +
 +term returns [Term.Raw term]
 +    : v=value                          { $term = v; }
 +    | f=function                       { $term = f; }
 +    | '(' c=comparatorType ')' t=term  { $term = new TypeCast(c, t); }
 +    ;
 +
 +columnOperation[List<Pair<ColumnDefinition.Raw, Operation.RawUpdate>> operations]
 +    : key=cident columnOperationDifferentiator[operations, key]
 +    ;
 +
 +columnOperationDifferentiator[List<Pair<ColumnDefinition.Raw, Operation.RawUpdate>> operations, ColumnDefinition.Raw key]
 +    : '=' normalColumnOperation[operations, key]
 +    | shorthandColumnOperation[operations, key]
 +    | '[' k=term ']' collectionColumnOperation[operations, key, k]
 +    | '.' field=fident udtColumnOperation[operations, key, field]
 +    ;
 +
 +normalColumnOperation[List<Pair<ColumnDefinition.Raw, Operation.RawUpdate>> operations, ColumnDefinition.Raw key]
 +    : t=term ('+' c=cident )?
 +      {
 +          if (c == null)
 +          {
 +              addRawUpdate(operations, key, new Operation.SetValue(t));
 +          }
 +          else
 +          {
 +              if (!key.equals(c))
 +                  addRecognitionError("Only expressions of the form X = <value> + X are supported.");
 +              addRawUpdate(operations, key, new Operation.Prepend(t));
 +          }
 +      }
 +    | c=cident sig=('+' | '-') t=term
 +      {
 +          if (!key.equals(c))
 +              addRecognitionError("Only expressions of the form X = X " + $sig.text + "<value> are supported.");
 +          addRawUpdate(operations, key, $sig.text.equals("+") ? new Operation.Addition(t) : new Operation.Substraction(t));
 +      }
 +    | c=cident i=INTEGER
 +      {
 +          // Note that this production *is* necessary because X = X - 3 will in fact be lexed as [ X, '=', X, INTEGER].
 +          if (!key.equals(c))
 +              // We don't yet allow a '+' in front of an integer, but we could in the future really, so let's be future-proof in our error message
 +              addRecognitionError("Only expressions of the form X = X " + ($i.text.charAt(0) == '-' ? '-' : '+') + " <value> are supported.");
 +          addRawUpdate(operations, key, new Operation.Addition(Constants.Literal.integer($i.text)));
 +      }
 +    ;
 +
 +shorthandColumnOperation[List<Pair<ColumnDefinition.Raw, Operation.RawUpdate>> operations, ColumnDefinition.Raw key]
 +    : sig=('+=' | '-=') t=term
 +      {
 +          addRawUpdate(operations, key, $sig.text.equals("+=") ? new Operation.Addition(t) : new Operation.Substraction(t));
 +      }
 +    ;
 +
 +collectionColumnOperation[List<Pair<ColumnDefinition.Raw, Operation.RawUpdate>> operations, ColumnDefinition.Raw key, Term.Raw k]
 +    : '=' t=term
 +      {
 +          addRawUpdate(operations, key, new Operation.SetElement(k, t));
 +      }
 +    ;
 +
 +udtColumnOperation[List<Pair<ColumnDefinition.Raw, Operation.RawUpdate>> operations, ColumnDefinition.Raw key, FieldIdentifier field]
 +    : '=' t=term
 +      {
 +          addRawUpdate(operations, key, new Operation.SetField(field, t));
 +      }
 +    ;
 +
 +columnCondition[List<Pair<ColumnDefinition.Raw, ColumnCondition.Raw>> conditions]
 +    // Note: we'll reject duplicates later
 +    : key=cident
 +        ( op=relationType t=term { conditions.add(Pair.create(key, ColumnCondition.Raw.simpleCondition(t, op))); }
 +        | K_IN
 +            ( values=singleColumnInValues { conditions.add(Pair.create(key, ColumnCondition.Raw.simpleInCondition(values))); }
 +            | marker=inMarker { conditions.add(Pair.create(key, ColumnCondition.Raw.simpleInCondition(marker))); }
 +            )
 +        | '[' element=term ']'
 +            ( op=relationType t=term { conditions.add(Pair.create(key, ColumnCondition.Raw.collectionCondition(t, element, op))); }
 +            | K_IN
 +                ( values=singleColumnInValues { conditions.add(Pair.create(key, ColumnCondition.Raw.collectionInCondition(element, values))); }
 +                | marker=inMarker { conditions.add(Pair.create(key, ColumnCondition.Raw.collectionInCondition(element, marker))); }
 +                )
 +            )
 +        | '.' field=fident
 +            ( op=relationType t=term { conditions.add(Pair.create(key, ColumnCondition.Raw.udtFieldCondition(t, field, op))); }
 +            | K_IN
 +                ( values=singleColumnInValues { conditions.add(Pair.create(key, ColumnCondition.Raw.udtFieldInCondition(field, values))); }
 +                | marker=inMarker { conditions.add(Pair.create(key, ColumnCondition.Raw.udtFieldInCondition(field, marker))); }
 +                )
 +            )
 +        )
 +    ;
 +
 +properties[PropertyDefinitions props]
 +    : property[props] (K_AND property[props])*
 +    ;
 +
 +property[PropertyDefinitions props]
 +    : k=noncol_ident '=' simple=propertyValue { try { $props.addProperty(k.toString(), simple); } catch (SyntaxException e) { addRecognitionError(e.getMessage()); } }
 +    | k=noncol_ident '=' map=mapLiteral { try { $props.addProperty(k.toString(), convertPropertyMap(map)); } catch (SyntaxException e) { addRecognitionError(e.getMessage()); } }
 +    ;
 +
 +propertyValue returns [String str]
 +    : c=constant           { $str = c.getRawText(); }
 +    | u=unreserved_keyword { $str = u; }
 +    ;
 +
 +relationType returns [Operator op]
 +    : '='  { $op = Operator.EQ; }
 +    | '<'  { $op = Operator.LT; }
 +    | '<=' { $op = Operator.LTE; }
 +    | '>'  { $op = Operator.GT; }
 +    | '>=' { $op = Operator.GTE; }
 +    | '!=' { $op = Operator.NEQ; }
 +    ;
 +
 +relation[WhereClause.Builder clauses]
 +    : name=cident type=relationType t=term { $clauses.add(new SingleColumnRelation(name, type, t)); }
 +    | name=cident K_LIKE t=term { $clauses.add(new SingleColumnRelation(name, Operator.LIKE, t)); }
 +    | name=cident K_IS K_NOT K_NULL { $clauses.add(new SingleColumnRelation(name, Operator.IS_NOT, Constants.NULL_LITERAL)); }
 +    | K_TOKEN l=tupleOfIdentifiers type=relationType t=term
 +        { $clauses.add(new TokenRelation(l, type, t)); }
 +    | name=cident K_IN marker=inMarker
 +        { $clauses.add(new SingleColumnRelation(name, Operator.IN, marker)); }
 +    | name=cident K_IN inValues=singleColumnInValues
 +        { $clauses.add(SingleColumnRelation.createInRelation($name.id, inValues)); }
 +    | name=cident K_CONTAINS { Operator rt = Operator.CONTAINS; } (K_KEY { rt = Operator.CONTAINS_KEY; })?
 +        t=term { $clauses.add(new SingleColumnRelation(name, rt, t)); }
 +    | name=cident '[' key=term ']' type=relationType t=term { $clauses.add(new SingleColumnRelation(name, key, type, t)); }
 +    | ids=tupleOfIdentifiers
 +      ( K_IN
 +          ( '(' ')'
 +              { $clauses.add(MultiColumnRelation.createInRelation(ids, new ArrayList<Tuples.Literal>())); }
 +          | tupleInMarker=inMarkerForTuple /* (a, b, c) IN ? */
 +              { $clauses.add(MultiColumnRelation.createSingleMarkerInRelation(ids, tupleInMarker)); }
 +          | literals=tupleOfTupleLiterals /* (a, b, c) IN ((1, 2, 3), (4, 5, 6), ...) */
 +              {
 +                  $clauses.add(MultiColumnRelation.createInRelation(ids, literals));
 +              }
 +          | markers=tupleOfMarkersForTuples /* (a, b, c) IN (?, ?, ...) */
 +              { $clauses.add(MultiColumnRelation.createInRelation(ids, markers)); }
 +          )
 +      | type=relationType literal=tupleLiteral /* (a, b, c) > (1, 2, 3) or (a, b, c) > (?, ?, ?) */
 +          {
 +              $clauses.add(MultiColumnRelation.createNonInRelation(ids, type, literal));
 +          }
 +      | type=relationType tupleMarker=markerForTuple /* (a, b, c) >= ? */
 +          { $clauses.add(MultiColumnRelation.createNonInRelation(ids, type, tupleMarker)); }
 +      )
 +    | '(' relation[$clauses] ')'
 +    ;
 +
 +inMarker returns [AbstractMarker.INRaw marker]
 +    : QMARK { $marker = newINBindVariables(null); }
 +    | ':' name=noncol_ident { $marker = newINBindVariables(name); }
 +    ;
 +
 +tupleOfIdentifiers returns [List<ColumnDefinition.Raw> ids]
 +    @init { $ids = new ArrayList<ColumnDefinition.Raw>(); }
 +    : '(' n1=cident { $ids.add(n1); } (',' ni=cident { $ids.add(ni); })* ')'
 +    ;
 +
 +singleColumnInValues returns [List<Term.Raw> terms]
 +    @init { $terms = new ArrayList<Term.Raw>(); }
 +    : '(' ( t1 = term { $terms.add(t1); } (',' ti=term { $terms.add(ti); })* )? ')'
 +    ;
 +
 +tupleOfTupleLiterals returns [List<Tuples.Literal> literals]
 +    @init { $literals = new ArrayList<>(); }
 +    : '(' t1=tupleLiteral { $literals.add(t1); } (',' ti=tupleLiteral { $literals.add(ti); })* ')'
 +    ;
 +
 +markerForTuple returns [Tuples.Raw marker]
 +    : QMARK { $marker = newTupleBindVariables(null); }
 +    | ':' name=noncol_ident { $marker = newTupleBindVariables(name); }
 +    ;
 +
 +tupleOfMarkersForTuples returns [List<Tuples.Raw> markers]
 +    @init { $markers = new ArrayList<Tuples.Raw>(); }
 +    : '(' m1=markerForTuple { $markers.add(m1); } (',' mi=markerForTuple { $markers.add(mi); })* ')'
 +    ;
 +
 +inMarkerForTuple returns [Tuples.INRaw marker]
 +    : QMARK { $marker = newTupleINBindVariables(null); }
 +    | ':' name=noncol_ident { $marker = newTupleINBindVariables(name); }
 +    ;
 +
 +comparatorType returns [CQL3Type.Raw t]
 +    : n=native_type     { $t = CQL3Type.Raw.from(n); }
 +    | c=collection_type { $t = c; }
 +    | tt=tuple_type     { $t = tt; }
 +    | id=userTypeName   { $t = CQL3Type.Raw.userType(id); }
 +    | K_FROZEN '<' f=comparatorType '>'
 +      {
 +        try {
 +            $t = CQL3Type.Raw.frozen(f);
 +        } catch (InvalidRequestException e) {
 +            addRecognitionError(e.getMessage());
 +        }
 +      }
 +    | s=STRING_LITERAL
 +      {
 +        try {
 +            $t = CQL3Type.Raw.from(new CQL3Type.Custom($s.text));
 +        } catch (SyntaxException e) {
 +            addRecognitionError("Cannot parse type " + $s.text + ": " + e.getMessage());
 +        } catch (ConfigurationException e) {
 +            addRecognitionError("Error setting type " + $s.text + ": " + e.getMessage());
 +        }
 +      }
 +    ;
 +
 +native_type returns [CQL3Type t]
 +    : K_ASCII     { $t = CQL3Type.Native.ASCII; }
 +    | K_BIGINT    { $t = CQL3Type.Native.BIGINT; }
 +    | K_BLOB      { $t = CQL3Type.Native.BLOB; }
 +    | K_BOOLEAN   { $t = CQL3Type.Native.BOOLEAN; }
 +    | K_COUNTER   { $t = CQL3Type.Native.COUNTER; }
 +    | K_DECIMAL   { $t = CQL3Type.Native.DECIMAL; }
 +    | K_DOUBLE    { $t = CQL3Type.Native.DOUBLE; }
 +    | K_DURATION    { $t = CQL3Type.Native.DURATION; }
 +    | K_FLOAT     { $t = CQL3Type.Native.FLOAT; }
 +    | K_INET      { $t = CQL3Type.Native.INET;}
 +    | K_INT       { $t = CQL3Type.Native.INT; }
 +    | K_SMALLINT  { $t = CQL3Type.Native.SMALLINT; }
 +    | K_TEXT      { $t = CQL3Type.Native.TEXT; }
 +    | K_TIMESTAMP { $t = CQL3Type.Native.TIMESTAMP; }
 +    | K_TINYINT   { $t = CQL3Type.Native.TINYINT; }
 +    | K_UUID      { $t = CQL3Type.Native.UUID; }
 +    | K_VARCHAR   { $t = CQL3Type.Native.VARCHAR; }
 +    | K_VARINT    { $t = CQL3Type.Native.VARINT; }
 +    | K_TIMEUUID  { $t = CQL3Type.Native.TIMEUUID; }
 +    | K_DATE      { $t = CQL3Type.Native.DATE; }
 +    | K_TIME      { $t = CQL3Type.Native.TIME; }
 +    ;
 +
 +collection_type returns [CQL3Type.Raw pt]
 +    : K_MAP  '<' t1=comparatorType ',' t2=comparatorType '>'
 +        {
 +            // if we can't parse either t1 or t2, antlr will "recover" and we may have t1 or t2 null.
 +            if (t1 != null && t2 != null)
 +                $pt = CQL3Type.Raw.map(t1, t2);
 +        }
 +    | K_LIST '<' t=comparatorType '>'
 +        { if (t != null) $pt = CQL3Type.Raw.list(t); }
 +    | K_SET  '<' t=comparatorType '>'
 +        { if (t != null) $pt = CQL3Type.Raw.set(t); }
 +    ;
 +
 +tuple_type returns [CQL3Type.Raw t]
 +    : K_TUPLE '<' { List<CQL3Type.Raw> types = new ArrayList<>(); }
 +         t1=comparatorType { types.add(t1); } (',' tn=comparatorType { types.add(tn); })*
 +      '>' { $t = CQL3Type.Raw.tuple(types); }
 +    ;
 +
 +username
 +    : IDENT
 +    | STRING_LITERAL
 +    | QUOTED_NAME { addRecognitionError("Quoted strings are are not supported for user names and USER is deprecated, please use ROLE");}
 +    ;
 +
 +mbean
 +    : STRING_LITERAL
 +    ;
 +
 +// Basically the same as cident, but we need to exlude existing CQL3 types
 +// (which for some reason are not reserved otherwise)
 +non_type_ident returns [ColumnIdentifier id]
 +    : t=IDENT                    { if (reservedTypeNames.contains($t.text)) addRecognitionError("Invalid (reserved) user type name " + $t.text); $id = new ColumnIdentifier($t.text, false); }
 +    | t=QUOTED_NAME              { $id = new ColumnIdentifier($t.text, true); }
 +    | k=basic_unreserved_keyword { $id = new ColumnIdentifier(k, false); }
 +    | kk=K_KEY                   { $id = new ColumnIdentifier($kk.text, false); }
 +    ;
 +
 +unreserved_keyword returns [String str]
 +    : u=unreserved_function_keyword     { $str = u; }
 +    | k=(K_TTL | K_COUNT | K_WRITETIME | K_KEY | K_CAST | K_JSON | K_DISTINCT) { $str = $k.text; }
 +    ;
 +
 +unreserved_function_keyword returns [String str]
 +    : u=basic_unreserved_keyword { $str = u; }
 +    | t=native_type              { $str = t.toString(); }
 +    ;
 +
 +basic_unreserved_keyword returns [String str]
 +    : k=( K_KEYS
 +        | K_AS
 +        | K_CLUSTERING
 +        | K_COMPACT
 +        | K_STORAGE
 +        | K_TYPE
 +        | K_VALUES
 +        | K_MAP
 +        | K_LIST
 +        | K_FILTERING
 +        | K_PERMISSION
 +        | K_PERMISSIONS
 +        | K_KEYSPACES
 +        | K_ALL
 +        | K_USER
 +        | K_USERS
 +        | K_ROLE
 +        | K_ROLES
 +        | K_SUPERUSER
 +        | K_NOSUPERUSER
 +        | K_LOGIN
 +        | K_NOLOGIN
 +        | K_OPTIONS
 +        | K_PASSWORD
 +        | K_EXISTS
 +        | K_CUSTOM
 +        | K_TRIGGER
 +        | K_CONTAINS
 +        | K_STATIC
 +        | K_FROZEN
 +        | K_TUPLE
 +        | K_FUNCTION
 +        | K_FUNCTIONS
 +        | K_AGGREGATE
 +        | K_SFUNC
 +        | K_STYPE
 +        | K_FINALFUNC
 +        | K_INITCOND
 +        | K_RETURNS
 +        | K_LANGUAGE
 +        | K_CALLED
 +        | K_INPUT
 +        | K_LIKE
 +        | K_PER
 +        | K_PARTITION
 +        | K_GROUP
 +        ) { $str = $k.text; }
 +    ;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d429cd0/src/java/org/apache/cassandra/auth/CassandraRoleManager.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d429cd0/src/java/org/apache/cassandra/config/CFMetaData.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d429cd0/src/java/org/apache/cassandra/cql3/QueryProcessor.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/QueryProcessor.java
index 0e0ba3c,ddee6c7..ef0ff12
--- a/src/java/org/apache/cassandra/cql3/QueryProcessor.java
+++ b/src/java/org/apache/cassandra/cql3/QueryProcessor.java
@@@ -561,25 -517,9 +561,25 @@@ public class QueryProcessor implements 
              ((CFStatement)statement).prepareKeyspace(clientState);
  
          Tracing.trace("Preparing statement");
-         return statement.prepare();
+         return statement.prepare(clientState);
      }
  
 +    public static <T extends ParsedStatement> T parseStatement(String queryStr, Class<T> klass, String type) throws SyntaxException
 +    {
 +        try
 +        {
 +            ParsedStatement stmt = parseStatement(queryStr);
 +
 +            if (!klass.isAssignableFrom(stmt.getClass()))
 +                throw new IllegalArgumentException("Invalid query, must be a " + type + " statement but was: " + stmt.getClass());
 +
 +            return klass.cast(stmt);
 +        }
 +        catch (RequestValidationException e)
 +        {
 +            throw new IllegalArgumentException(e.getMessage(), e);
 +        }
 +    }
      public static ParsedStatement parseStatement(String queryStr) throws SyntaxException
      {
          try


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


[16/25] cassandra git commit: Merge branch 'cassandra-3.0' into cassandra-3.11

Posted by if...@apache.org.
Merge branch 'cassandra-3.0' into cassandra-3.11


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

Branch: refs/heads/cassandra-3.11
Commit: 6d429cd0315d3509c904d0e83f91f7d12ba12085
Parents: c8a3b58 6c29ee8
Author: Alex Petrov <ol...@gmail.com>
Authored: Mon Nov 6 15:52:31 2017 +0100
Committer: Alex Petrov <ol...@gmail.com>
Committed: Mon Nov 6 15:52:31 2017 +0100

----------------------------------------------------------------------
 NEWS.txt                                        |  20 +
 bin/cqlsh.py                                    |   7 +-
 doc/native_protocol_v4.spec                     |   4 +
 doc/source/cql/appendices.rst                   |  22 +
 ...dra-driver-internal-only-3.11.0-bb96859b.zip | Bin 0 -> 266661 bytes
 src/antlr/Lexer.g                               |   4 +
 src/antlr/Parser.g                              |  41 +-
 .../cassandra/auth/CassandraRoleManager.java    |   3 +-
 .../org/apache/cassandra/config/CFMetaData.java |  59 ++-
 .../apache/cassandra/cql3/QueryProcessor.java   |   2 +-
 .../cql3/statements/AlterTableStatement.java    |  26 +-
 .../statements/AuthenticationStatement.java     |   2 +-
 .../cql3/statements/AuthorizationStatement.java |   2 +-
 .../cql3/statements/BatchStatement.java         |   4 +-
 .../statements/CreateAggregateStatement.java    |   4 +-
 .../statements/CreateFunctionStatement.java     |   4 +-
 .../cql3/statements/CreateIndexStatement.java   |   9 +-
 .../cql3/statements/CreateTableStatement.java   |   2 +-
 .../cql3/statements/CreateViewStatement.java    |   2 +-
 .../cql3/statements/DropFunctionStatement.java  |   4 +-
 .../cql3/statements/ModificationStatement.java  |  12 +-
 .../cql3/statements/ParsedStatement.java        |   3 +-
 .../statements/SchemaAlteringStatement.java     |  32 +-
 .../cql3/statements/SelectStatement.java        |   8 +-
 .../cql3/statements/TruncateStatement.java      |   2 +-
 .../cassandra/cql3/statements/UseStatement.java |   2 +-
 src/java/org/apache/cassandra/db/view/View.java |  31 +-
 .../index/internal/keys/KeysSearcher.java       |   8 +-
 .../cassandra/io/sstable/CQLSSTableWriter.java  |   2 +-
 .../apache/cassandra/repair/RepairRunnable.java |   3 +-
 .../apache/cassandra/service/ClientState.java   |  16 +
 .../cassandra/thrift/ThriftValidation.java      |  10 +-
 .../transport/messages/StartupMessage.java      |   4 +
 .../org/apache/cassandra/cql3/ViewTest.java     |  16 +-
 .../cql3/validation/entities/UFTest.java        |  25 +
 .../cql3/validation/entities/UserTypesTest.java |  20 +
 .../cql3/validation/operations/AlterTest.java   |   8 +
 .../DropCompactStorageThriftTest.java           | 525 +++++++++++++++++++
 .../cassandra/index/sasi/SASIIndexTest.java     |   3 +-
 .../io/sstable/StressCQLSSTableWriter.java      |   2 +-
 40 files changed, 876 insertions(+), 77 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d429cd0/NEWS.txt
----------------------------------------------------------------------
diff --cc NEWS.txt
index f8dd93b,621866b..9812b22
--- a/NEWS.txt
+++ b/NEWS.txt
@@@ -33,16 -34,28 +33,36 @@@ Materialized View
  
  Upgrading
  ---------
 -   - Nothing specific to this release, but please see previous upgrading sections,
 -     especially if you are upgrading from 2.2.
 +    - Creating Materialized View with filtering on non-primary-key base column
 +      (added in CASSANDRA-10368) is disabled, because the liveness of view row
 +      is depending on multiple filtered base non-key columns and base non-key
 +      column used in view primary-key. This semantic cannot be supported without
 +      storage format change, see CASSANDRA-13826. For append-only use case, you
 +      may still use this feature with a startup flag: "-Dcassandra.mv.allow_filtering_nonkey_columns_unsafe=true"
  
 -Compact Storage
++Compact Storage (only when upgrading from 3.X or any version lower than 3.0.15)
+ ---------------
+     - Starting version 4.0, Thrift and COMPACT STORAGE is no longer supported.
+       'ALTER ... DROP COMPACT STORAGE' statement makes Compact Tables CQL-compatible,
+       exposing internal structure of Thrift/Compact Tables. You can find more details
+       on exposed internal structure under: 
+       http://cassandra.apache.org/doc/latest/cql/appendices.html#appendix-c-dropping-compact-storage
+ 
+       For uninterrupted cluster upgrades, drivers now support 'NO_COMPACT' startup option.
+       Supplying this flag will have same effect as 'DROP COMPACT STORAGE', but only for the
+       current connection.
+ 
+       In order to upgrade, clients supporting a non-compact schema view can be rolled out
+       gradually. When all the clients are updated 'ALTER ... DROP COMPACT STORAGE' can be
+       executed. After dropping compact storage, ’NO_COMPACT' option will have no effect
+       after that.
+ 
++
+ Materialized Views
+ -------------------
 +
 +Materialized Views (only when upgrading from 3.X or any version lower than 3.0.15)
 +---------------------------------------------------------------------------------------
      - Cassandra will no longer allow dropping columns on tables with Materialized Views.
      - A change was made in the way the Materialized View timestamp is computed, which
        may cause an old deletion to a base column which is view primary key (PK) column

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d429cd0/bin/cqlsh.py
----------------------------------------------------------------------
diff --cc bin/cqlsh.py
index 85a0489,8d05d9d..ca76e34
--- a/bin/cqlsh.py
+++ b/bin/cqlsh.py
@@@ -439,8 -701,9 +440,9 @@@ class Shell(cmd.Cmd)
      def __init__(self, hostname, port, color=False,
                   username=None, password=None, encoding=None, stdin=None, tty=True,
                   completekey=DEFAULT_COMPLETEKEY, browser=None, use_conn=None,
 -                 cqlver=DEFAULT_CQLVER, keyspace=None,
 +                 cqlver=None, keyspace=None,
                   tracing_enabled=False, expand_enabled=False,
+                  no_compact=False,
                   display_nanotime_format=DEFAULT_NANOTIME_FORMAT,
                   display_timestamp_format=DEFAULT_TIMESTAMP_FORMAT,
                   display_date_format=DEFAULT_DATE_FORMAT,
@@@ -470,11 -732,9 +472,12 @@@
          if use_conn:
              self.conn = use_conn
          else:
 +            kwargs = {}
 +            if protocol_version is not None:
 +                kwargs['protocol_version'] = protocol_version
              self.conn = Cluster(contact_points=(self.hostname,), port=self.port, cql_version=cqlver,
 -                                protocol_version=protocol_version,
 -                                auth_provider=self.auth_provider, no_compact=no_compact,
 +                                auth_provider=self.auth_provider,
++                                no_compact=no_compact,
                                  ssl_options=sslhandling.ssl_settings(hostname, CONFIG_FILE) if ssl else None,
                                  load_balancing_policy=WhiteListRoundRobinPolicy([self.hostname]),
                                  control_connection_timeout=connect_timeout,
@@@ -2396,9 -2644,9 +2400,10 @@@ def main(options, hostname, port)
                        tty=options.tty,
                        completekey=options.completekey,
                        browser=options.browser,
 +                      protocol_version=options.protocol_version,
                        cqlver=options.cqlversion,
                        keyspace=options.keyspace,
+                       no_compact=options.no_compact,
                        display_timestamp_format=options.time_format,
                        display_nanotime_format=options.nanotime_format,
                        display_date_format=options.date_format,

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d429cd0/doc/source/cql/appendices.rst
----------------------------------------------------------------------
diff --cc doc/source/cql/appendices.rst
index 8c63a32,0000000..480b78e
mode 100644,000000..100644
--- a/doc/source/cql/appendices.rst
+++ b/doc/source/cql/appendices.rst
@@@ -1,308 -1,0 +1,330 @@@
 +.. 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.
 +
 +.. highlight:: cql
 +
 +Appendices
 +----------
 +
 +.. _appendix-A:
 +
 +Appendix A: CQL Keywords
 +~~~~~~~~~~~~~~~~~~~~~~~~
 +
 +CQL distinguishes between *reserved* and *non-reserved* keywords.
 +Reserved keywords cannot be used as identifier, they are truly reserved
 +for the language (but one can enclose a reserved keyword by
 +double-quotes to use it as an identifier). Non-reserved keywords however
 +only have a specific meaning in certain context but can used as
 +identifier otherwise. The only *raison d’être* of these non-reserved
 +keywords is convenience: some keyword are non-reserved when it was
 +always easy for the parser to decide whether they were used as keywords
 +or not.
 +
 ++--------------------+-------------+
 +| Keyword            | Reserved?   |
 ++====================+=============+
 +| ``ADD``            | yes         |
 ++--------------------+-------------+
 +| ``AGGREGATE``      | no          |
 ++--------------------+-------------+
 +| ``ALL``            | no          |
 ++--------------------+-------------+
 +| ``ALLOW``          | yes         |
 ++--------------------+-------------+
 +| ``ALTER``          | yes         |
 ++--------------------+-------------+
 +| ``AND``            | yes         |
 ++--------------------+-------------+
 +| ``APPLY``          | yes         |
 ++--------------------+-------------+
 +| ``AS``             | no          |
 ++--------------------+-------------+
 +| ``ASC``            | yes         |
 ++--------------------+-------------+
 +| ``ASCII``          | no          |
 ++--------------------+-------------+
 +| ``AUTHORIZE``      | yes         |
 ++--------------------+-------------+
 +| ``BATCH``          | yes         |
 ++--------------------+-------------+
 +| ``BEGIN``          | yes         |
 ++--------------------+-------------+
 +| ``BIGINT``         | no          |
 ++--------------------+-------------+
 +| ``BLOB``           | no          |
 ++--------------------+-------------+
 +| ``BOOLEAN``        | no          |
 ++--------------------+-------------+
 +| ``BY``             | yes         |
 ++--------------------+-------------+
 +| ``CALLED``         | no          |
 ++--------------------+-------------+
 +| ``CLUSTERING``     | no          |
 ++--------------------+-------------+
 +| ``COLUMNFAMILY``   | yes         |
 ++--------------------+-------------+
 +| ``COMPACT``        | no          |
 ++--------------------+-------------+
 +| ``CONTAINS``       | no          |
 ++--------------------+-------------+
 +| ``COUNT``          | no          |
 ++--------------------+-------------+
 +| ``COUNTER``        | no          |
 ++--------------------+-------------+
 +| ``CREATE``         | yes         |
 ++--------------------+-------------+
 +| ``CUSTOM``         | no          |
 ++--------------------+-------------+
 +| ``DATE``           | no          |
 ++--------------------+-------------+
 +| ``DECIMAL``        | no          |
 ++--------------------+-------------+
 +| ``DELETE``         | yes         |
 ++--------------------+-------------+
 +| ``DESC``           | yes         |
 ++--------------------+-------------+
 +| ``DESCRIBE``       | yes         |
 ++--------------------+-------------+
 +| ``DISTINCT``       | no          |
 ++--------------------+-------------+
 +| ``DOUBLE``         | no          |
 ++--------------------+-------------+
 +| ``DROP``           | yes         |
 ++--------------------+-------------+
 +| ``ENTRIES``        | yes         |
 ++--------------------+-------------+
 +| ``EXECUTE``        | yes         |
 ++--------------------+-------------+
 +| ``EXISTS``         | no          |
 ++--------------------+-------------+
 +| ``FILTERING``      | no          |
 ++--------------------+-------------+
 +| ``FINALFUNC``      | no          |
 ++--------------------+-------------+
 +| ``FLOAT``          | no          |
 ++--------------------+-------------+
 +| ``FROM``           | yes         |
 ++--------------------+-------------+
 +| ``FROZEN``         | no          |
 ++--------------------+-------------+
 +| ``FULL``           | yes         |
 ++--------------------+-------------+
 +| ``FUNCTION``       | no          |
 ++--------------------+-------------+
 +| ``FUNCTIONS``      | no          |
 ++--------------------+-------------+
 +| ``GRANT``          | yes         |
 ++--------------------+-------------+
 +| ``IF``             | yes         |
 ++--------------------+-------------+
 +| ``IN``             | yes         |
 ++--------------------+-------------+
 +| ``INDEX``          | yes         |
 ++--------------------+-------------+
 +| ``INET``           | no          |
 ++--------------------+-------------+
 +| ``INFINITY``       | yes         |
 ++--------------------+-------------+
 +| ``INITCOND``       | no          |
 ++--------------------+-------------+
 +| ``INPUT``          | no          |
 ++--------------------+-------------+
 +| ``INSERT``         | yes         |
 ++--------------------+-------------+
 +| ``INT``            | no          |
 ++--------------------+-------------+
 +| ``INTO``           | yes         |
 ++--------------------+-------------+
 +| ``JSON``           | no          |
 ++--------------------+-------------+
 +| ``KEY``            | no          |
 ++--------------------+-------------+
 +| ``KEYS``           | no          |
 ++--------------------+-------------+
 +| ``KEYSPACE``       | yes         |
 ++--------------------+-------------+
 +| ``KEYSPACES``      | no          |
 ++--------------------+-------------+
 +| ``LANGUAGE``       | no          |
 ++--------------------+-------------+
 +| ``LIMIT``          | yes         |
 ++--------------------+-------------+
 +| ``LIST``           | no          |
 ++--------------------+-------------+
 +| ``LOGIN``          | no          |
 ++--------------------+-------------+
 +| ``MAP``            | no          |
 ++--------------------+-------------+
 +| ``MODIFY``         | yes         |
 ++--------------------+-------------+
 +| ``NAN``            | yes         |
 ++--------------------+-------------+
 +| ``NOLOGIN``        | no          |
 ++--------------------+-------------+
 +| ``NORECURSIVE``    | yes         |
 ++--------------------+-------------+
 +| ``NOSUPERUSER``    | no          |
 ++--------------------+-------------+
 +| ``NOT``            | yes         |
 ++--------------------+-------------+
 +| ``NULL``           | yes         |
 ++--------------------+-------------+
 +| ``OF``             | yes         |
 ++--------------------+-------------+
 +| ``ON``             | yes         |
 ++--------------------+-------------+
 +| ``OPTIONS``        | no          |
 ++--------------------+-------------+
 +| ``OR``             | yes         |
 ++--------------------+-------------+
 +| ``ORDER``          | yes         |
 ++--------------------+-------------+
 +| ``PASSWORD``       | no          |
 ++--------------------+-------------+
 +| ``PERMISSION``     | no          |
 ++--------------------+-------------+
 +| ``PERMISSIONS``    | no          |
 ++--------------------+-------------+
 +| ``PRIMARY``        | yes         |
 ++--------------------+-------------+
 +| ``RENAME``         | yes         |
 ++--------------------+-------------+
 +| ``REPLACE``        | yes         |
 ++--------------------+-------------+
 +| ``RETURNS``        | no          |
 ++--------------------+-------------+
 +| ``REVOKE``         | yes         |
 ++--------------------+-------------+
 +| ``ROLE``           | no          |
 ++--------------------+-------------+
 +| ``ROLES``          | no          |
 ++--------------------+-------------+
 +| ``SCHEMA``         | yes         |
 ++--------------------+-------------+
 +| ``SELECT``         | yes         |
 ++--------------------+-------------+
 +| ``SET``            | yes         |
 ++--------------------+-------------+
 +| ``SFUNC``          | no          |
 ++--------------------+-------------+
 +| ``SMALLINT``       | no          |
 ++--------------------+-------------+
 +| ``STATIC``         | no          |
 ++--------------------+-------------+
 +| ``STORAGE``        | no          |
 ++--------------------+-------------+
 +| ``STYPE``          | no          |
 ++--------------------+-------------+
 +| ``SUPERUSER``      | no          |
 ++--------------------+-------------+
 +| ``TABLE``          | yes         |
 ++--------------------+-------------+
 +| ``TEXT``           | no          |
 ++--------------------+-------------+
 +| ``TIME``           | no          |
 ++--------------------+-------------+
 +| ``TIMESTAMP``      | no          |
 ++--------------------+-------------+
 +| ``TIMEUUID``       | no          |
 ++--------------------+-------------+
 +| ``TINYINT``        | no          |
 ++--------------------+-------------+
 +| ``TO``             | yes         |
 ++--------------------+-------------+
 +| ``TOKEN``          | yes         |
 ++--------------------+-------------+
 +| ``TRIGGER``        | no          |
 ++--------------------+-------------+
 +| ``TRUNCATE``       | yes         |
 ++--------------------+-------------+
 +| ``TTL``            | no          |
 ++--------------------+-------------+
 +| ``TUPLE``          | no          |
 ++--------------------+-------------+
 +| ``TYPE``           | no          |
 ++--------------------+-------------+
 +| ``UNLOGGED``       | yes         |
 ++--------------------+-------------+
 +| ``UPDATE``         | yes         |
 ++--------------------+-------------+
 +| ``USE``            | yes         |
 ++--------------------+-------------+
 +| ``USER``           | no          |
 ++--------------------+-------------+
 +| ``USERS``          | no          |
 ++--------------------+-------------+
 +| ``USING``          | yes         |
 ++--------------------+-------------+
 +| ``UUID``           | no          |
 ++--------------------+-------------+
 +| ``VALUES``         | no          |
 ++--------------------+-------------+
 +| ``VARCHAR``        | no          |
 ++--------------------+-------------+
 +| ``VARINT``         | no          |
 ++--------------------+-------------+
 +| ``WHERE``          | yes         |
 ++--------------------+-------------+
 +| ``WITH``           | yes         |
 ++--------------------+-------------+
 +| ``WRITETIME``      | no          |
 ++--------------------+-------------+
 +
 +Appendix B: CQL Reserved Types
 +~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
 +
 +The following type names are not currently used by CQL, but are reserved
 +for potential future use. User-defined types may not use reserved type
 +names as their name.
 +
 ++-----------------+
 +| type            |
 ++=================+
 +| ``bitstring``   |
 ++-----------------+
 +| ``byte``        |
 ++-----------------+
 +| ``complex``     |
 ++-----------------+
 +| ``enum``        |
 ++-----------------+
 +| ``interval``    |
 ++-----------------+
 +| ``macaddr``     |
 ++-----------------+
++
++
++Appendix C: Dropping Compact Storage
++~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
++
++Starting version 4.0, Thrift and COMPACT STORAGE is no longer supported.
++
++'ALTER ... DROP COMPACT STORAGE' statement makes Compact Tables CQL-compatible,
++exposing internal structure of Thrift/Compact Tables:
++
++- CQL-created Compact Tables that have no clustering columns, will expose an
++  additional clustering column ``column1`` with ``UTF8Type``.
++- CQL-created Compact Tables that had no regular columns, will expose a
++  regular column ``value`` with ``BytesType``.
++- For CQL-Created Compact Tables, all columns originally defined as
++  ``regular`` will be come ``static``
++- CQL-created Compact Tables that have clustering but have no regular
++  columns will have an empty value column (of ``EmptyType``)
++- SuperColumn Tables (can only be created through Thrift) will expose
++  a compact value map with an empty name.
++- Thrift-created Compact Tables will have types corresponding to their
++  Thrift definition.

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d429cd0/src/antlr/Lexer.g
----------------------------------------------------------------------
diff --cc src/antlr/Lexer.g
index 23cbed6,0000000..1c52d4c
mode 100644,000000..100644
--- a/src/antlr/Lexer.g
+++ b/src/antlr/Lexer.g
@@@ -1,350 -1,0 +1,354 @@@
 +/*
 + * 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.
 + */
 +
 +lexer grammar Lexer;
 +
 +@lexer::members {
 +    List<Token> tokens = new ArrayList<Token>();
 +
 +    public void emit(Token token)
 +    {
 +        state.token = token;
 +        tokens.add(token);
 +    }
 +
 +    public Token nextToken()
 +    {
 +        super.nextToken();
 +        if (tokens.size() == 0)
 +            return new CommonToken(Token.EOF);
 +        return tokens.remove(0);
 +    }
 +
 +    private final List<ErrorListener> listeners = new ArrayList<ErrorListener>();
 +
 +    public void addErrorListener(ErrorListener listener)
 +    {
 +        this.listeners.add(listener);
 +    }
 +
 +    public void removeErrorListener(ErrorListener listener)
 +    {
 +        this.listeners.remove(listener);
 +    }
 +
 +    public void displayRecognitionError(String[] tokenNames, RecognitionException e)
 +    {
 +        for (int i = 0, m = listeners.size(); i < m; i++)
 +            listeners.get(i).syntaxError(this, tokenNames, e);
 +    }
 +}
 +
 +// Case-insensitive keywords
 +// When adding a new reserved keyword, add entry to o.a.c.cql3.ReservedKeywords as well
 +// When adding a new unreserved keyword, add entry to unreserved keywords in Parser.g
 +K_SELECT:      S E L E C T;
 +K_FROM:        F R O M;
 +K_AS:          A S;
 +K_WHERE:       W H E R E;
 +K_AND:         A N D;
 +K_KEY:         K E Y;
 +K_KEYS:        K E Y S;
 +K_ENTRIES:     E N T R I E S;
 +K_FULL:        F U L L;
 +K_INSERT:      I N S E R T;
 +K_UPDATE:      U P D A T E;
 +K_WITH:        W I T H;
 +K_LIMIT:       L I M I T;
 +K_PER:         P E R;
 +K_PARTITION:   P A R T I T I O N;
 +K_USING:       U S I N G;
 +K_USE:         U S E;
 +K_DISTINCT:    D I S T I N C T;
 +K_COUNT:       C O U N T;
 +K_SET:         S E T;
 +K_BEGIN:       B E G I N;
 +K_UNLOGGED:    U N L O G G E D;
 +K_BATCH:       B A T C H;
 +K_APPLY:       A P P L Y;
 +K_TRUNCATE:    T R U N C A T E;
 +K_DELETE:      D E L E T E;
 +K_IN:          I N;
 +K_CREATE:      C R E A T E;
 +K_KEYSPACE:    ( K E Y S P A C E
 +                 | S C H E M A );
 +K_KEYSPACES:   K E Y S P A C E S;
 +K_COLUMNFAMILY:( C O L U M N F A M I L Y
 +                 | T A B L E );
 +K_MATERIALIZED:M A T E R I A L I Z E D;
 +K_VIEW:        V I E W;
 +K_INDEX:       I N D E X;
 +K_CUSTOM:      C U S T O M;
 +K_ON:          O N;
 +K_TO:          T O;
 +K_DROP:        D R O P;
 +K_PRIMARY:     P R I M A R Y;
 +K_INTO:        I N T O;
 +K_VALUES:      V A L U E S;
 +K_TIMESTAMP:   T I M E S T A M P;
 +K_TTL:         T T L;
 +K_CAST:        C A S T;
 +K_ALTER:       A L T E R;
 +K_RENAME:      R E N A M E;
 +K_ADD:         A D D;
 +K_TYPE:        T Y P E;
 +K_COMPACT:     C O M P A C T;
 +K_STORAGE:     S T O R A G E;
 +K_ORDER:       O R D E R;
 +K_BY:          B Y;
 +K_ASC:         A S C;
 +K_DESC:        D E S C;
 +K_ALLOW:       A L L O W;
 +K_FILTERING:   F I L T E R I N G;
 +K_IF:          I F;
 +K_IS:          I S;
 +K_CONTAINS:    C O N T A I N S;
 +K_GROUP:       G R O U P;
 +
 +K_GRANT:       G R A N T;
 +K_ALL:         A L L;
 +K_PERMISSION:  P E R M I S S I O N;
 +K_PERMISSIONS: P E R M I S S I O N S;
 +K_OF:          O F;
 +K_REVOKE:      R E V O K E;
 +K_MODIFY:      M O D I F Y;
 +K_AUTHORIZE:   A U T H O R I Z E;
 +K_DESCRIBE:    D E S C R I B E;
 +K_EXECUTE:     E X E C U T E;
 +K_NORECURSIVE: N O R E C U R S I V E;
 +K_MBEAN:       M B E A N;
 +K_MBEANS:      M B E A N S;
 +
 +K_USER:        U S E R;
 +K_USERS:       U S E R S;
 +K_ROLE:        R O L E;
 +K_ROLES:       R O L E S;
 +K_SUPERUSER:   S U P E R U S E R;
 +K_NOSUPERUSER: N O S U P E R U S E R;
 +K_PASSWORD:    P A S S W O R D;
 +K_LOGIN:       L O G I N;
 +K_NOLOGIN:     N O L O G I N;
 +K_OPTIONS:     O P T I O N S;
 +
 +K_CLUSTERING:  C L U S T E R I N G;
 +K_ASCII:       A S C I I;
 +K_BIGINT:      B I G I N T;
 +K_BLOB:        B L O B;
 +K_BOOLEAN:     B O O L E A N;
 +K_COUNTER:     C O U N T E R;
 +K_DECIMAL:     D E C I M A L;
 +K_DOUBLE:      D O U B L E;
 +K_DURATION:    D U R A T I O N;
 +K_FLOAT:       F L O A T;
 +K_INET:        I N E T;
 +K_INT:         I N T;
 +K_SMALLINT:    S M A L L I N T;
 +K_TINYINT:     T I N Y I N T;
 +K_TEXT:        T E X T;
 +K_UUID:        U U I D;
 +K_VARCHAR:     V A R C H A R;
 +K_VARINT:      V A R I N T;
 +K_TIMEUUID:    T I M E U U I D;
 +K_TOKEN:       T O K E N;
 +K_WRITETIME:   W R I T E T I M E;
 +K_DATE:        D A T E;
 +K_TIME:        T I M E;
 +
 +K_NULL:        N U L L;
 +K_NOT:         N O T;
 +K_EXISTS:      E X I S T S;
 +
 +K_MAP:         M A P;
 +K_LIST:        L I S T;
 +K_NAN:         N A N;
 +K_INFINITY:    I N F I N I T Y;
 +K_TUPLE:       T U P L E;
 +
 +K_TRIGGER:     T R I G G E R;
 +K_STATIC:      S T A T I C;
 +K_FROZEN:      F R O Z E N;
 +
 +K_FUNCTION:    F U N C T I O N;
 +K_FUNCTIONS:   F U N C T I O N S;
 +K_AGGREGATE:   A G G R E G A T E;
 +K_SFUNC:       S F U N C;
 +K_STYPE:       S T Y P E;
 +K_FINALFUNC:   F I N A L F U N C;
 +K_INITCOND:    I N I T C O N D;
 +K_RETURNS:     R E T U R N S;
 +K_CALLED:      C A L L E D;
 +K_INPUT:       I N P U T;
 +K_LANGUAGE:    L A N G U A G E;
 +K_OR:          O R;
 +K_REPLACE:     R E P L A C E;
 +
 +K_JSON:        J S O N;
 +K_DEFAULT:     D E F A U L T;
 +K_UNSET:       U N S E T;
 +K_LIKE:        L I K E;
 +
 +// Case-insensitive alpha characters
 +fragment A: ('a'|'A');
 +fragment B: ('b'|'B');
 +fragment C: ('c'|'C');
 +fragment D: ('d'|'D');
 +fragment E: ('e'|'E');
 +fragment F: ('f'|'F');
 +fragment G: ('g'|'G');
 +fragment H: ('h'|'H');
 +fragment I: ('i'|'I');
 +fragment J: ('j'|'J');
 +fragment K: ('k'|'K');
 +fragment L: ('l'|'L');
 +fragment M: ('m'|'M');
 +fragment N: ('n'|'N');
 +fragment O: ('o'|'O');
 +fragment P: ('p'|'P');
 +fragment Q: ('q'|'Q');
 +fragment R: ('r'|'R');
 +fragment S: ('s'|'S');
 +fragment T: ('t'|'T');
 +fragment U: ('u'|'U');
 +fragment V: ('v'|'V');
 +fragment W: ('w'|'W');
 +fragment X: ('x'|'X');
 +fragment Y: ('y'|'Y');
 +fragment Z: ('z'|'Z');
 +
 +STRING_LITERAL
 +    @init{
 +        StringBuilder txt = new StringBuilder(); // temporary to build pg-style-string
 +    }
 +    @after{ setText(txt.toString()); }
 +    :
 +      /* pg-style string literal */
 +      (
 +        '\$' '\$'
 +        ( /* collect all input until '$$' is reached again */
 +          {  (input.size() - input.index() > 1)
 +               && !"$$".equals(input.substring(input.index(), input.index() + 1)) }?
 +             => c=. { txt.appendCodePoint(c); }
 +        )*
 +        '\$' '\$'
 +      )
 +      |
 +      /* conventional quoted string literal */
 +      (
 +        '\'' (c=~('\'') { txt.appendCodePoint(c);} | '\'' '\'' { txt.appendCodePoint('\''); })* '\''
 +      )
 +    ;
 +
 +QUOTED_NAME
 +    @init{ StringBuilder b = new StringBuilder(); }
 +    @after{ setText(b.toString()); }
 +    : '\"' (c=~('\"') { b.appendCodePoint(c); } | '\"' '\"' { b.appendCodePoint('\"'); })+ '\"'
 +    ;
 +
++EMPTY_QUOTED_NAME
++    : '\"' '\"'
++    ;
++
 +fragment DIGIT
 +    : '0'..'9'
 +    ;
 +
 +fragment LETTER
 +    : ('A'..'Z' | 'a'..'z')
 +    ;
 +
 +fragment HEX
 +    : ('A'..'F' | 'a'..'f' | '0'..'9')
 +    ;
 +
 +fragment EXPONENT
 +    : E ('+' | '-')? DIGIT+
 +    ;
 +
 +fragment DURATION_UNIT
 +    : Y
 +    | M O
 +    | W
 +    | D
 +    | H
 +    | M
 +    | S
 +    | M S
 +    | U S
 +    | '\u00B5' S
 +    | N S
 +    ;
 +
 +INTEGER
 +    : '-'? DIGIT+
 +    ;
 +
 +QMARK
 +    : '?'
 +    ;
 +
 +/*
 + * Normally a lexer only emits one token at a time, but ours is tricked out
 + * to support multiple (see @lexer::members near the top of the grammar).
 + */
 +FLOAT
 +    : INTEGER EXPONENT
 +    | INTEGER '.' DIGIT* EXPONENT?
 +    ;
 +
 +/*
 + * This has to be before IDENT so it takes precendence over it.
 + */
 +BOOLEAN
 +    : T R U E | F A L S E
 +    ;
 +
 +DURATION
 +    : '-'? DIGIT+ DURATION_UNIT (DIGIT+ DURATION_UNIT)*
 +    | '-'? 'P' (DIGIT+ 'Y')? (DIGIT+ 'M')? (DIGIT+ 'D')? ('T' (DIGIT+ 'H')? (DIGIT+ 'M')? (DIGIT+ 'S')?)? // ISO 8601 "format with designators"
 +    | '-'? 'P' DIGIT+ 'W'
 +    | '-'? 'P' DIGIT DIGIT DIGIT DIGIT '-' DIGIT DIGIT '-' DIGIT DIGIT 'T' DIGIT DIGIT ':' DIGIT DIGIT ':' DIGIT DIGIT // ISO 8601 "alternative format"
 +    ;
 +
 +IDENT
 +    : LETTER (LETTER | DIGIT | '_')*
 +    ;
 +
 +HEXNUMBER
 +    : '0' X HEX*
 +    ;
 +
 +UUID
 +    : HEX HEX HEX HEX HEX HEX HEX HEX '-'
 +      HEX HEX HEX HEX '-'
 +      HEX HEX HEX HEX '-'
 +      HEX HEX HEX HEX '-'
 +      HEX HEX HEX HEX HEX HEX HEX HEX HEX HEX HEX HEX
 +    ;
 +
 +WS
 +    : (' ' | '\t' | '\n' | '\r')+ { $channel = HIDDEN; }
 +    ;
 +
 +COMMENT
 +    : ('--' | '//') .* ('\n'|'\r') { $channel = HIDDEN; }
 +    ;
 +
 +MULTILINE_COMMENT
 +    : '/*' .* '*/' { $channel = HIDDEN; }
 +    ;


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


[07/25] cassandra git commit: Merge branch 'cassandra-3.0' into cassandra-3.11

Posted by if...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d429cd0/tools/stress/src/org/apache/cassandra/io/sstable/StressCQLSSTableWriter.java
----------------------------------------------------------------------
diff --cc tools/stress/src/org/apache/cassandra/io/sstable/StressCQLSSTableWriter.java
index 56f0297,0000000..5a285e1
mode 100644,000000..100644
--- a/tools/stress/src/org/apache/cassandra/io/sstable/StressCQLSSTableWriter.java
+++ b/tools/stress/src/org/apache/cassandra/io/sstable/StressCQLSSTableWriter.java
@@@ -1,676 -1,0 +1,676 @@@
 +/*
 + * 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.io.sstable;
 +
 +import java.io.Closeable;
 +import java.io.File;
 +import java.io.IOException;
 +import java.nio.ByteBuffer;
 +import java.util.*;
 +import java.util.stream.Collectors;
 +
 +import com.datastax.driver.core.ProtocolVersion;
 +import com.datastax.driver.core.TypeCodec;
 +import org.antlr.runtime.RecognitionException;
 +import org.apache.cassandra.config.CFMetaData;
 +import org.apache.cassandra.config.DatabaseDescriptor;
 +import org.apache.cassandra.config.Schema;
 +import org.apache.cassandra.cql3.CQLFragmentParser;
 +import org.apache.cassandra.cql3.ColumnSpecification;
 +import org.apache.cassandra.cql3.CqlParser;
 +import org.apache.cassandra.cql3.QueryOptions;
 +import org.apache.cassandra.cql3.UpdateParameters;
 +import org.apache.cassandra.cql3.functions.UDHelper;
 +import org.apache.cassandra.cql3.statements.CreateTableStatement;
 +import org.apache.cassandra.cql3.statements.CreateTypeStatement;
 +import org.apache.cassandra.cql3.statements.ParsedStatement;
 +import org.apache.cassandra.cql3.statements.UpdateStatement;
 +import org.apache.cassandra.db.*;
 +import org.apache.cassandra.db.marshal.UserType;
 +import org.apache.cassandra.db.partitions.Partition;
 +import org.apache.cassandra.dht.IPartitioner;
 +import org.apache.cassandra.dht.Murmur3Partitioner;
 +import org.apache.cassandra.exceptions.InvalidRequestException;
 +import org.apache.cassandra.exceptions.RequestValidationException;
 +import org.apache.cassandra.exceptions.SyntaxException;
 +import org.apache.cassandra.io.sstable.format.SSTableFormat;
 +import org.apache.cassandra.schema.KeyspaceMetadata;
 +import org.apache.cassandra.schema.KeyspaceParams;
 +import org.apache.cassandra.schema.Types;
 +import org.apache.cassandra.service.ClientState;
 +import org.apache.cassandra.utils.ByteBufferUtil;
 +import org.apache.cassandra.utils.Pair;
 +
 +/**
 + * Utility to write SSTables.
 + * <p>
 + * Typical usage looks like:
 + * <pre>
 + *   String type = CREATE TYPE myKs.myType (a int, b int)";
 + *   String schema = "CREATE TABLE myKs.myTable ("
 + *                 + "  k int PRIMARY KEY,"
 + *                 + "  v1 text,"
 + *                 + "  v2 int,"
 + *                 + "  v3 myType,"
 + *                 + ")";
 + *   String insert = "INSERT INTO myKs.myTable (k, v1, v2, v3) VALUES (?, ?, ?, ?)";
 + *
 + *   // Creates a new writer. You need to provide at least the directory where to write the created sstable,
 + *   // the schema for the sstable to write and a (prepared) insert statement to use. If you do not use the
 + *   // default partitioner (Murmur3Partitioner), you will also need to provide the partitioner in use, see
 + *   // StressCQLSSTableWriter.Builder for more details on the available options.
 + *   StressCQLSSTableWriter writer = StressCQLSSTableWriter.builder()
 + *                                             .inDirectory("path/to/directory")
 + *                                             .withType(type)
 + *                                             .forTable(schema)
 + *                                             .using(insert).build();
 + *
 + *   UserType myType = writer.getUDType("myType");
 + *   // Adds a nember of rows to the resulting sstable
 + *   writer.addRow(0, "test1", 24, myType.newValue().setInt("a", 10).setInt("b", 20));
 + *   writer.addRow(1, "test2", null, null);
 + *   writer.addRow(2, "test3", 42, myType.newValue().setInt("a", 30).setInt("b", 40));
 + *
 + *   // Close the writer, finalizing the sstable
 + *   writer.close();
 + * </pre>
 + *
 + * Please note that {@code StressCQLSSTableWriter} is <b>not</b> thread-safe (multiple threads cannot access the
 + * same instance). It is however safe to use multiple instances in parallel (even if those instance write
 + * sstables for the same table).
 + */
 +public class StressCQLSSTableWriter implements Closeable
 +{
 +    public static final ByteBuffer UNSET_VALUE = ByteBufferUtil.UNSET_BYTE_BUFFER;
 +
 +    static
 +    {
 +        DatabaseDescriptor.clientInitialization(false);
 +        // Partitioner is not set in client mode.
 +        if (DatabaseDescriptor.getPartitioner() == null)
 +            DatabaseDescriptor.setPartitionerUnsafe(Murmur3Partitioner.instance);
 +    }
 +
 +    private final AbstractSSTableSimpleWriter writer;
 +    private final UpdateStatement insert;
 +    private final List<ColumnSpecification> boundNames;
 +    private final List<TypeCodec> typeCodecs;
 +    private final ColumnFamilyStore cfs;
 +
 +    private StressCQLSSTableWriter(ColumnFamilyStore cfs, AbstractSSTableSimpleWriter writer, UpdateStatement insert, List<ColumnSpecification> boundNames)
 +    {
 +        this.cfs = cfs;
 +        this.writer = writer;
 +        this.insert = insert;
 +        this.boundNames = boundNames;
 +        this.typeCodecs = boundNames.stream().map(bn ->  UDHelper.codecFor(UDHelper.driverType(bn.type)))
 +                                             .collect(Collectors.toList());
 +    }
 +
 +    /**
 +     * Returns a new builder for a StressCQLSSTableWriter.
 +     *
 +     * @return the new builder.
 +     */
 +    public static Builder builder()
 +    {
 +        return new Builder();
 +    }
 +
 +    /**
 +     * Adds a new row to the writer.
 +     * <p>
 +     * This is a shortcut for {@code addRow(Arrays.asList(values))}.
 +     *
 +     * @param values the row values (corresponding to the bind variables of the
 +     * insertion statement used when creating by this writer).
 +     * @return this writer.
 +     */
 +    public StressCQLSSTableWriter addRow(Object... values)
 +    throws InvalidRequestException, IOException
 +    {
 +        return addRow(Arrays.asList(values));
 +    }
 +
 +    /**
 +     * Adds a new row to the writer.
 +     * <p>
 +     * Each provided value type should correspond to the types of the CQL column
 +     * the value is for. The correspondance between java type and CQL type is the
 +     * same one than the one documented at
 +     * www.datastax.com/drivers/java/2.0/apidocs/com/datastax/driver/core/DataType.Name.html#asJavaClass().
 +     * <p>
 +     * If you prefer providing the values directly as binary, use
 +     * {@link #rawAddRow} instead.
 +     *
 +     * @param values the row values (corresponding to the bind variables of the
 +     * insertion statement used when creating by this writer).
 +     * @return this writer.
 +     */
 +    public StressCQLSSTableWriter addRow(List<Object> values)
 +    throws InvalidRequestException, IOException
 +    {
 +        int size = Math.min(values.size(), boundNames.size());
 +        List<ByteBuffer> rawValues = new ArrayList<>(size);
 +
 +        for (int i = 0; i < size; i++)
 +        {
 +            Object value = values.get(i);
 +            rawValues.add(serialize(value, typeCodecs.get(i)));
 +        }
 +
 +        return rawAddRow(rawValues);
 +    }
 +
 +    /**
 +     * Adds a new row to the writer.
 +     * <p>
 +     * This is equivalent to the other addRow methods, but takes a map whose
 +     * keys are the names of the columns to add instead of taking a list of the
 +     * values in the order of the insert statement used during construction of
 +     * this write.
 +     * <p>
 +     * Please note that the column names in the map keys must be in lowercase unless
 +     * the declared column name is a
 +     * <a href="http://cassandra.apache.org/doc/cql3/CQL.html#identifiers">case-sensitive quoted identifier</a>
 +     * (in which case the map key must use the exact case of the column).
 +     *
 +     * @param values a map of colum name to column values representing the new
 +     * row to add. Note that if a column is not part of the map, it's value will
 +     * be {@code null}. If the map contains keys that does not correspond to one
 +     * of the column of the insert statement used when creating this writer, the
 +     * the corresponding value is ignored.
 +     * @return this writer.
 +     */
 +    public StressCQLSSTableWriter addRow(Map<String, Object> values)
 +    throws InvalidRequestException, IOException
 +    {
 +        int size = boundNames.size();
 +        List<ByteBuffer> rawValues = new ArrayList<>(size);
 +        for (int i = 0; i < size; i++)
 +        {
 +            ColumnSpecification spec = boundNames.get(i);
 +            Object value = values.get(spec.name.toString());
 +            rawValues.add(serialize(value, typeCodecs.get(i)));
 +        }
 +        return rawAddRow(rawValues);
 +    }
 +
 +    /**
 +     * Adds a new row to the writer given already serialized values.
 +     *
 +     * @param values the row values (corresponding to the bind variables of the
 +     * insertion statement used when creating by this writer) as binary.
 +     * @return this writer.
 +     */
 +    public StressCQLSSTableWriter rawAddRow(ByteBuffer... values)
 +    throws InvalidRequestException, IOException
 +    {
 +        return rawAddRow(Arrays.asList(values));
 +    }
 +
 +    /**
 +     * Adds a new row to the writer given already serialized values.
 +     * <p>
 +     * This is a shortcut for {@code rawAddRow(Arrays.asList(values))}.
 +     *
 +     * @param values the row values (corresponding to the bind variables of the
 +     * insertion statement used when creating by this writer) as binary.
 +     * @return this writer.
 +     */
 +    public StressCQLSSTableWriter rawAddRow(List<ByteBuffer> values)
 +    throws InvalidRequestException, IOException
 +    {
 +        if (values.size() != boundNames.size())
 +            throw new InvalidRequestException(String.format("Invalid number of arguments, expecting %d values but got %d", boundNames.size(), values.size()));
 +
 +        QueryOptions options = QueryOptions.forInternalCalls(null, values);
 +        List<ByteBuffer> keys = insert.buildPartitionKeyNames(options);
 +        SortedSet<Clustering> clusterings = insert.createClustering(options);
 +
 +        long now = System.currentTimeMillis() * 1000;
 +        // Note that we asks indexes to not validate values (the last 'false' arg below) because that triggers a 'Keyspace.open'
 +        // and that forces a lot of initialization that we don't want.
 +        UpdateParameters params = new UpdateParameters(insert.cfm,
 +                                                       insert.updatedColumns(),
 +                                                       options,
 +                                                       insert.getTimestamp(now, options),
 +                                                       insert.getTimeToLive(options),
 +                                                       Collections.<DecoratedKey, Partition>emptyMap());
 +
 +        try
 +        {
 +            for (ByteBuffer key : keys)
 +            {
 +                for (Clustering clustering : clusterings)
 +                    insert.addUpdateForKey(writer.getUpdateFor(key), clustering, params);
 +            }
 +            return this;
 +        }
 +        catch (SSTableSimpleUnsortedWriter.SyncException e)
 +        {
 +            // If we use a BufferedWriter and had a problem writing to disk, the IOException has been
 +            // wrapped in a SyncException (see BufferedWriter below). We want to extract that IOE.
 +            throw (IOException)e.getCause();
 +        }
 +    }
 +
 +    /**
 +     * Adds a new row to the writer given already serialized values.
 +     * <p>
 +     * This is equivalent to the other rawAddRow methods, but takes a map whose
 +     * keys are the names of the columns to add instead of taking a list of the
 +     * values in the order of the insert statement used during construction of
 +     * this write.
 +     *
 +     * @param values a map of colum name to column values representing the new
 +     * row to add. Note that if a column is not part of the map, it's value will
 +     * be {@code null}. If the map contains keys that does not correspond to one
 +     * of the column of the insert statement used when creating this writer, the
 +     * the corresponding value is ignored.
 +     * @return this writer.
 +     */
 +    public StressCQLSSTableWriter rawAddRow(Map<String, ByteBuffer> values)
 +    throws InvalidRequestException, IOException
 +    {
 +        int size = Math.min(values.size(), boundNames.size());
 +        List<ByteBuffer> rawValues = new ArrayList<>(size);
 +        for (int i = 0; i < size; i++) 
 +        {
 +            ColumnSpecification spec = boundNames.get(i);
 +            rawValues.add(values.get(spec.name.toString()));
 +        }
 +        return rawAddRow(rawValues);
 +    }
 +
 +    /**
 +     * Returns the User Defined type, used in this SSTable Writer, that can
 +     * be used to create UDTValue instances.
 +     *
 +     * @param dataType name of the User Defined type
 +     * @return user defined type
 +     */
 +    public com.datastax.driver.core.UserType getUDType(String dataType)
 +    {
 +        KeyspaceMetadata ksm = Schema.instance.getKSMetaData(insert.keyspace());
 +        UserType userType = ksm.types.getNullable(ByteBufferUtil.bytes(dataType));
 +        return (com.datastax.driver.core.UserType) UDHelper.driverType(userType);
 +    }
 +
 +    /**
 +     * Close this writer.
 +     * <p>
 +     * This method should be called, otherwise the produced sstables are not
 +     * guaranteed to be complete (and won't be in practice).
 +     */
 +    public void close() throws IOException
 +    {
 +        writer.close();
 +    }
 +
 +    private ByteBuffer serialize(Object value, TypeCodec codec)
 +    {
 +        if (value == null || value == UNSET_VALUE)
 +            return (ByteBuffer) value;
 +
 +        return codec.serialize(value, ProtocolVersion.NEWEST_SUPPORTED);
 +    }
 +    /**
 +     * The writer loads data in directories corresponding to how they laid out on the server.
 +     * <p>
 +     * {keyspace}/{table-cfid}/
 +     *
 +     * This method can be used to fetch the innermost directory with the sstable components
 +     * @return The directory containing the sstable components
 +     */
 +    public File getInnermostDirectory()
 +    {
 +        return cfs.getDirectories().getDirectoryForNewSSTables();
 +    }
 +
 +    /**
 +     * A Builder for a StressCQLSSTableWriter object.
 +     */
 +    public static class Builder
 +    {
 +        private final List<File> directoryList;
 +        private ColumnFamilyStore cfs;
 +
 +        protected SSTableFormat.Type formatType = null;
 +
 +        private Boolean makeRangeAware = false;
 +
 +        private CreateTableStatement.RawStatement schemaStatement;
 +        private final List<CreateTypeStatement> typeStatements;
 +        private UpdateStatement.ParsedInsert insertStatement;
 +        private IPartitioner partitioner;
 +
 +        private boolean sorted = false;
 +        private long bufferSizeInMB = 128;
 +
 +        protected Builder()
 +        {
 +            this.typeStatements = new ArrayList<>();
 +            this.directoryList = new ArrayList<>();
 +        }
 +
 +        /**
 +         * The directory where to write the sstables.
 +         * <p>
 +         * This is a mandatory option.
 +         *
 +         * @param directory the directory to use, which should exists and be writable.
 +         * @return this builder.
 +         *
 +         * @throws IllegalArgumentException if {@code directory} doesn't exist or is not writable.
 +         */
 +        public Builder inDirectory(String directory)
 +        {
 +            return inDirectory(new File(directory));
 +        }
 +
 +        /**
 +         * The directory where to write the sstables (mandatory option).
 +         * <p>
 +         * This is a mandatory option.
 +         *
 +         * @param directory the directory to use, which should exist and be writable.
 +         * @return this builder.
 +         *
 +         * @throws IllegalArgumentException if {@code directory} doesn't exist or is not writable.
 +         */
 +        public Builder inDirectory(File directory)
 +        {
 +            if (!directory.exists())
 +                throw new IllegalArgumentException(directory + " doesn't exists");
 +            if (!directory.canWrite())
 +                throw new IllegalArgumentException(directory + " exists but is not writable");
 +
 +            directoryList.add(directory);
 +            return this;
 +        }
 +
 +        /**
 +         * A pre-instanciated ColumnFamilyStore
 +         * <p>
 +         * This is can be used in place of inDirectory and forTable
 +         *
 +         * @see #inDirectory(File)
 +         *
 +         * @param cfs the list of directories to use, which should exist and be writable.
 +         * @return this builder.
 +         *
 +         * @throws IllegalArgumentException if a directory doesn't exist or is not writable.
 +         */
 +        public Builder withCfs(ColumnFamilyStore cfs)
 +        {
 +            this.cfs = cfs;
 +            return this;
 +        }
 +
 +
 +        public Builder withType(String typeDefinition) throws SyntaxException
 +        {
 +            typeStatements.add(parseStatement(typeDefinition, CreateTypeStatement.class, "CREATE TYPE"));
 +            return this;
 +        }
 +
 +        /**
 +         * The schema (CREATE TABLE statement) for the table for which sstable are to be created.
 +         * <p>
 +         * Please note that the provided CREATE TABLE statement <b>must</b> use a fully-qualified
 +         * table name, one that include the keyspace name.
 +         * <p>
 +         * This is a mandatory option.
 +         *
 +         * @param schema the schema of the table for which sstables are to be created.
 +         * @return this builder.
 +         *
 +         * @throws IllegalArgumentException if {@code schema} is not a valid CREATE TABLE statement
 +         * or does not have a fully-qualified table name.
 +         */
 +        public Builder forTable(String schema)
 +        {
 +            this.schemaStatement = parseStatement(schema, CreateTableStatement.RawStatement.class, "CREATE TABLE");
 +            return this;
 +        }
 +
 +        /**
 +         * The partitioner to use.
 +         * <p>
 +         * By default, {@code Murmur3Partitioner} will be used. If this is not the partitioner used
 +         * by the cluster for which the SSTables are created, you need to use this method to
 +         * provide the correct partitioner.
 +         *
 +         * @param partitioner the partitioner to use.
 +         * @return this builder.
 +         */
 +        public Builder withPartitioner(IPartitioner partitioner)
 +        {
 +            this.partitioner = partitioner;
 +            return this;
 +        }
 +
 +
 +        /**
 +         * Specify if the sstable writer should be vnode range aware.
 +         * This will create a sstable per vnode range.
 +         *
 +         * @param makeRangeAware
 +         * @return
 +         */
 +        public Builder rangeAware(boolean makeRangeAware)
 +        {
 +            this.makeRangeAware = makeRangeAware;
 +            return this;
 +        }
 +
 +        /**
 +         * The INSERT statement defining the order of the values to add for a given CQL row.
 +         * <p>
 +         * Please note that the provided INSERT statement <b>must</b> use a fully-qualified
 +         * table name, one that include the keyspace name. Morewover, said statement must use
 +         * bind variables since it is those bind variables that will be bound to values by the
 +         * resulting writer.
 +         * <p>
 +         * This is a mandatory option, and this needs to be called after foTable().
 +         *
 +         * @param insert an insertion statement that defines the order
 +         * of column values to use.
 +         * @return this builder.
 +         *
 +         * @throws IllegalArgumentException if {@code insertStatement} is not a valid insertion
 +         * statement, does not have a fully-qualified table name or have no bind variables.
 +         */
 +        public Builder using(String insert)
 +        {
 +            this.insertStatement = parseStatement(insert, UpdateStatement.ParsedInsert.class, "INSERT");
 +            return this;
 +        }
 +
 +        /**
 +         * The size of the buffer to use.
 +         * <p>
 +         * This defines how much data will be buffered before being written as
 +         * a new SSTable. This correspond roughly to the data size that will have the created
 +         * sstable.
 +         * <p>
 +         * The default is 128MB, which should be reasonable for a 1GB heap. If you experience
 +         * OOM while using the writer, you should lower this value.
 +         *
 +         * @param size the size to use in MB.
 +         * @return this builder.
 +         */
 +        public Builder withBufferSizeInMB(int size)
 +        {
 +            this.bufferSizeInMB = size;
 +            return this;
 +        }
 +
 +        /**
 +         * Creates a StressCQLSSTableWriter that expects sorted inputs.
 +         * <p>
 +         * If this option is used, the resulting writer will expect rows to be
 +         * added in SSTable sorted order (and an exception will be thrown if that
 +         * is not the case during insertion). The SSTable sorted order means that
 +         * rows are added such that their partition key respect the partitioner
 +         * order.
 +         * <p>
 +         * You should thus only use this option is you know that you can provide
 +         * the rows in order, which is rarely the case. If you can provide the
 +         * rows in order however, using this sorted might be more efficient.
 +         * <p>
 +         * Note that if used, some option like withBufferSizeInMB will be ignored.
 +         *
 +         * @return this builder.
 +         */
 +        public Builder sorted()
 +        {
 +            this.sorted = true;
 +            return this;
 +        }
 +
 +        @SuppressWarnings("resource")
 +        public StressCQLSSTableWriter build()
 +        {
 +            if (directoryList.isEmpty() && cfs == null)
 +                throw new IllegalStateException("No output directories specified, you should provide a directory with inDirectory()");
 +            if (schemaStatement == null && cfs == null)
 +                throw new IllegalStateException("Missing schema, you should provide the schema for the SSTable to create with forTable()");
 +            if (insertStatement == null)
 +                throw new IllegalStateException("No insert statement specified, you should provide an insert statement through using()");
 +
 +            synchronized (StressCQLSSTableWriter.class)
 +            {
 +                if (cfs == null)
 +                    cfs = createOfflineTable(schemaStatement, typeStatements, directoryList);
 +
 +                if (partitioner == null)
 +                    partitioner = cfs.getPartitioner();
 +
 +                Pair<UpdateStatement, List<ColumnSpecification>> preparedInsert = prepareInsert();
 +                AbstractSSTableSimpleWriter writer = sorted
 +                                                     ? new SSTableSimpleWriter(cfs.getDirectories().getDirectoryForNewSSTables(), cfs.metadata, preparedInsert.left.updatedColumns())
 +                                                     : new SSTableSimpleUnsortedWriter(cfs.getDirectories().getDirectoryForNewSSTables(), cfs.metadata, preparedInsert.left.updatedColumns(), bufferSizeInMB);
 +
 +                if (formatType != null)
 +                    writer.setSSTableFormatType(formatType);
 +
 +                writer.setRangeAwareWriting(makeRangeAware);
 +
 +                return new StressCQLSSTableWriter(cfs, writer, preparedInsert.left, preparedInsert.right);
 +            }
 +        }
 +
 +        private static void createTypes(String keyspace, List<CreateTypeStatement> typeStatements)
 +        {
 +            KeyspaceMetadata ksm = Schema.instance.getKSMetaData(keyspace);
 +            Types.RawBuilder builder = Types.rawBuilder(keyspace);
 +            for (CreateTypeStatement st : typeStatements)
 +                st.addToRawBuilder(builder);
 +
 +            ksm = ksm.withSwapped(builder.build());
 +            Schema.instance.setKeyspaceMetadata(ksm);
 +        }
 +
 +        public static ColumnFamilyStore createOfflineTable(String schema, List<File> directoryList)
 +        {
 +            return createOfflineTable(parseStatement(schema, CreateTableStatement.RawStatement.class, "CREATE TABLE"), Collections.EMPTY_LIST, directoryList);
 +        }
 +
 +        /**
 +         * Creates the table according to schema statement
 +         * with specified data directories
 +         */
 +        public static ColumnFamilyStore createOfflineTable(CreateTableStatement.RawStatement schemaStatement, List<CreateTypeStatement> typeStatements, List<File> directoryList)
 +        {
 +            String keyspace = schemaStatement.keyspace();
 +
 +            if (Schema.instance.getKSMetaData(keyspace) == null)
 +                Schema.instance.load(KeyspaceMetadata.create(keyspace, KeyspaceParams.simple(1)));
 +
 +            createTypes(keyspace, typeStatements);
 +
 +            KeyspaceMetadata ksm = Schema.instance.getKSMetaData(keyspace);
 +
 +            CFMetaData cfMetaData = ksm.tables.getNullable(schemaStatement.columnFamily());
 +
 +            if (cfMetaData != null)
 +                return Schema.instance.getColumnFamilyStoreInstance(cfMetaData.cfId);
 +
 +            CreateTableStatement statement = (CreateTableStatement) schemaStatement.prepare(ksm.types).statement;
 +            statement.validate(ClientState.forInternalCalls());
 +
 +            //Build metatdata with a portable cfId
 +            cfMetaData = statement.metadataBuilder()
 +                                  .withId(CFMetaData.generateLegacyCfId(keyspace, statement.columnFamily()))
 +                                  .build()
 +                                  .params(statement.params());
 +
 +            Keyspace.setInitialized();
 +            Directories directories = new Directories(cfMetaData, directoryList.stream().map(Directories.DataDirectory::new).collect(Collectors.toList()));
 +
 +            Keyspace ks = Keyspace.openWithoutSSTables(keyspace);
 +            ColumnFamilyStore cfs =  ColumnFamilyStore.createColumnFamilyStore(ks, cfMetaData.cfName, cfMetaData, directories, false, false, true);
 +
 +            ks.initCfCustom(cfs);
 +            Schema.instance.load(cfs.metadata);
 +            Schema.instance.setKeyspaceMetadata(ksm.withSwapped(ksm.tables.with(cfs.metadata)));
 +
 +            return cfs;
 +        }
 +
 +        /**
 +         * Prepares insert statement for writing data to SSTable
 +         *
 +         * @return prepared Insert statement and it's bound names
 +         */
 +        private Pair<UpdateStatement, List<ColumnSpecification>> prepareInsert()
 +        {
-             ParsedStatement.Prepared cqlStatement = insertStatement.prepare();
++            ParsedStatement.Prepared cqlStatement = insertStatement.prepare(ClientState.forInternalCalls());
 +            UpdateStatement insert = (UpdateStatement) cqlStatement.statement;
 +            insert.validate(ClientState.forInternalCalls());
 +
 +            if (insert.hasConditions())
 +                throw new IllegalArgumentException("Conditional statements are not supported");
 +            if (insert.isCounter())
 +                throw new IllegalArgumentException("Counter update statements are not supported");
 +            if (cqlStatement.boundNames.isEmpty())
 +                throw new IllegalArgumentException("Provided insert statement has no bind variables");
 +
 +            return Pair.create(insert, cqlStatement.boundNames);
 +        }
 +    }
 +
 +    public static <T extends ParsedStatement> T parseStatement(String query, Class<T> klass, String type)
 +    {
 +        try
 +        {
 +            ParsedStatement stmt = CQLFragmentParser.parseAnyUnhandled(CqlParser::query, query);
 +
 +            if (!stmt.getClass().equals(klass))
 +                throw new IllegalArgumentException("Invalid query, must be a " + type + " statement but was: " + stmt.getClass());
 +
 +            return klass.cast(stmt);
 +        }
 +        catch (RecognitionException | RequestValidationException e)
 +        {
 +            throw new IllegalArgumentException(e.getMessage(), e);
 +        }
 +    }
 +}


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


[09/25] cassandra git commit: Merge branch 'cassandra-3.0' into cassandra-3.11

Posted by if...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d429cd0/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java
index fdbcf7a,a5fa12d..de42647
--- a/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java
@@@ -87,12 -93,16 +85,12 @@@ public class AlterTableStatement extend
          if (meta.isView())
              throw new InvalidRequestException("Cannot use ALTER TABLE on Materialized View");
  
-         CFMetaData cfm = meta.copy();
+         CFMetaData cfm;
 -
 -        CQL3Type validator = this.validator == null ? null : this.validator.prepare(keyspace());
          ColumnIdentifier columnName = null;
          ColumnDefinition def = null;
 -        if (rawColumnName != null)
 -        {
 -            columnName = rawColumnName.prepare(meta);
 -            def = meta.getColumnDefinition(columnName);
 -        }
 +        CQL3Type.Raw dataType = null;
 +        boolean isStatic = false;
 +        CQL3Type validator = null;
  
          List<ViewDefinition> viewUpdates = null;
          Iterable<ViewDefinition> views = View.findAll(keyspace(), columnFamily());
@@@ -102,146 -112,141 +100,156 @@@
              case ALTER:
                  throw new InvalidRequestException("Altering of types is not allowed");
              case ADD:
-                 if (cfm.isDense())
 -                assert columnName != null;
+                 if (meta.isDense())
                      throw new InvalidRequestException("Cannot add new column to a COMPACT STORAGE table");
  
+                 cfm = meta.copy();
+ 
 -                if (isStatic)
 +                for (AlterTableStatementColumn colData : colNameList)
                  {
 -                    if (!cfm.isCompound())
 -                        throw new InvalidRequestException("Static columns are not allowed in COMPACT STORAGE tables");
 -                    if (cfm.clusteringColumns().isEmpty())
 -                        throw new InvalidRequestException("Static columns are only useful (and thus allowed) if the table has at least one clustering column");
 -                }
 +                    columnName = colData.getColumnName().getIdentifier(cfm);
 +                    def = cfm.getColumnDefinition(columnName);
 +                    dataType = colData.getColumnType();
 +                    assert dataType != null;
 +                    isStatic = colData.getStaticType();
 +                    validator = dataType.prepare(keyspace());
  
 -                if (def != null)
 -                {
 -                    switch (def.kind)
 +
 +                    if (isStatic)
                      {
 -                        case PARTITION_KEY:
 -                        case CLUSTERING:
 -                            throw new InvalidRequestException(String.format("Invalid column name %s because it conflicts with a PRIMARY KEY part", columnName));
 -                        default:
 -                            throw new InvalidRequestException(String.format("Invalid column name %s because it conflicts with an existing column", columnName));
 +                        if (!cfm.isCompound())
 +                            throw new InvalidRequestException("Static columns are not allowed in COMPACT STORAGE tables");
 +                        if (cfm.clusteringColumns().isEmpty())
 +                            throw new InvalidRequestException("Static columns are only useful (and thus allowed) if the table has at least one clustering column");
                      }
 -                }
  
 -                // Cannot re-add a dropped counter column. See #7831.
 -                if (meta.isCounter() && meta.getDroppedColumns().containsKey(columnName.bytes))
 -                    throw new InvalidRequestException(String.format("Cannot re-add previously dropped counter column %s", columnName));
 +                    if (def != null)
 +                    {
 +                        switch (def.kind)
 +                        {
 +                            case PARTITION_KEY:
 +                            case CLUSTERING:
 +                                throw new InvalidRequestException(String.format("Invalid column name %s because it conflicts with a PRIMARY KEY part", columnName));
 +                            default:
 +                                throw new InvalidRequestException(String.format("Invalid column name %s because it conflicts with an existing column", columnName));
 +                        }
 +                    }
  
 -                AbstractType<?> type = validator.getType();
 -                if (type.isCollection() && type.isMultiCell())
 -                {
 -                    if (!cfm.isCompound())
 -                        throw new InvalidRequestException("Cannot use non-frozen collections in COMPACT STORAGE tables");
 -                    if (cfm.isSuper())
 -                        throw new InvalidRequestException("Cannot use non-frozen collections with super column families");
 -
 -                    // If there used to be a non-frozen collection column with the same name (that has been dropped),
 -                    // we could still have some data using the old type, and so we can't allow adding a collection
 -                    // with the same name unless the types are compatible (see #6276).
 -                    CFMetaData.DroppedColumn dropped = cfm.getDroppedColumns().get(columnName.bytes);
 -                    if (dropped != null && dropped.type instanceof CollectionType
 -                        && dropped.type.isMultiCell() && !type.isCompatibleWith(dropped.type))
 +                    // Cannot re-add a dropped counter column. See #7831.
 +                    if (meta.isCounter() && meta.getDroppedColumns().containsKey(columnName.bytes))
 +                        throw new InvalidRequestException(String.format("Cannot re-add previously dropped counter column %s", columnName));
 +
 +                    AbstractType<?> type = validator.getType();
 +                    if (type.isCollection() && type.isMultiCell())
                      {
 -                        String message =
 -                            String.format("Cannot add a collection with the name %s because a collection with the same name"
 -                                          + " and a different type (%s) has already been used in the past",
 -                                          columnName,
 -                                          dropped.type.asCQL3Type());
 -                        throw new InvalidRequestException(message);
 +                        if (!cfm.isCompound())
 +                            throw new InvalidRequestException("Cannot use non-frozen collections in COMPACT STORAGE tables");
 +                        if (cfm.isSuper())
 +                            throw new InvalidRequestException("Cannot use non-frozen collections with super column families");
 +
 +                        // If there used to be a non-frozen collection column with the same name (that has been dropped),
 +                        // we could still have some data using the old type, and so we can't allow adding a collection
 +                        // with the same name unless the types are compatible (see #6276).
 +                        CFMetaData.DroppedColumn dropped = cfm.getDroppedColumns().get(columnName.bytes);
 +                        if (dropped != null && dropped.type instanceof CollectionType
 +                            && dropped.type.isMultiCell() && !type.isCompatibleWith(dropped.type))
 +                        {
 +                            String message =
 +                                String.format("Cannot add a collection with the name %s because a collection with the same name"
 +                                              + " and a different type (%s) has already been used in the past",
 +                                              columnName,
 +                                              dropped.type.asCQL3Type());
 +                            throw new InvalidRequestException(message);
 +                        }
                      }
 -                }
  
 -                cfm.addColumnDefinition(isStatic
 -                                        ? ColumnDefinition.staticDef(cfm, columnName.bytes, type)
 -                                        : ColumnDefinition.regularDef(cfm, columnName.bytes, type));
 +                    cfm.addColumnDefinition(isStatic
 +                                            ? ColumnDefinition.staticDef(cfm, columnName.bytes, type)
 +                                            : ColumnDefinition.regularDef(cfm, columnName.bytes, type));
  
 -                // Adding a column to a table which has an include all view requires the column to be added to the view
 -                // as well
 -                if (!isStatic)
 -                {
 -                    for (ViewDefinition view : views)
 +                    // Adding a column to a table which has an include all view requires the column to be added to the view
 +                    // as well
 +                    if (!isStatic)
                      {
 -                        if (view.includeAllColumns)
 +                        for (ViewDefinition view : views)
                          {
 -                            ViewDefinition viewCopy = view.copy();
 -                            viewCopy.metadata.addColumnDefinition(ColumnDefinition.regularDef(viewCopy.metadata, columnName.bytes, type));
 -                            if (viewUpdates == null)
 -                                viewUpdates = new ArrayList<>();
 -                            viewUpdates.add(viewCopy);
 +                            if (view.includeAllColumns)
 +                            {
 +                                ViewDefinition viewCopy = view.copy();
 +                                viewCopy.metadata.addColumnDefinition(ColumnDefinition.regularDef(viewCopy.metadata, columnName.bytes, type));
 +                                if (viewUpdates == null)
 +                                    viewUpdates = new ArrayList<>();
 +                                viewUpdates.add(viewCopy);
 +                            }
                          }
                      }
                  }
                  break;
  
              case DROP:
-                 if (!cfm.isCQLTable())
 -                assert columnName != null;
+                 if (!meta.isCQLTable())
                      throw new InvalidRequestException("Cannot drop columns from a non-CQL3 table");
  
 -                if (def == null)
 -                    throw new InvalidRequestException(String.format("Column %s was not found in table %s", columnName, columnFamily()));
 -
+                 cfm = meta.copy();
+ 
 -                switch (def.kind)
 +                for (AlterTableStatementColumn colData : colNameList)
                  {
 -                    case PARTITION_KEY:
 -                    case CLUSTERING:
 -                        throw new InvalidRequestException(String.format("Cannot drop PRIMARY KEY part %s", columnName));
 -                    case REGULAR:
 -                    case STATIC:
 -                        ColumnDefinition toDelete = null;
 -                        for (ColumnDefinition columnDef : cfm.partitionColumns())
 -                        {
 -                            if (columnDef.name.equals(columnName))
 -                            {
 -                                toDelete = columnDef;
 -                                break;
 -                            }
 -                        }
 -                        assert toDelete != null;
 -                        cfm.removeColumnDefinition(toDelete);
 -                        cfm.recordColumnDrop(toDelete, deleteTimestamp == null ? queryState.getTimestamp() : deleteTimestamp);
 -                        break;
 -                }
 +                    columnName = colData.getColumnName().getIdentifier(cfm);
 +                    def = cfm.getColumnDefinition(columnName);
  
 -                // If the dropped column is required by any secondary indexes
 -                // we reject the operation, as the indexes must be dropped first
 -                Indexes allIndexes = cfm.getIndexes();
 -                if (!allIndexes.isEmpty())
 -                {
 -                    ColumnFamilyStore store = Keyspace.openAndGetStore(cfm);
 -                    Set<IndexMetadata> dependentIndexes = store.indexManager.getDependentIndexes(def);
 -                    if (!dependentIndexes.isEmpty())
 -                        throw new InvalidRequestException(String.format("Cannot drop column %s because it has " +
 -                                                                        "dependent secondary indexes (%s)",
 -                                                                        def,
 -                                                                        dependentIndexes.stream()
 -                                                                                        .map(i -> i.name)
 -                                                                                        .collect(Collectors.joining(","))));
 -                }
 +                    if (def == null)
 +                        throw new InvalidRequestException(String.format("Column %s was not found in table %s", columnName, columnFamily()));
  
 -                if (!Iterables.isEmpty(views))
 -                    throw new InvalidRequestException(String.format("Cannot drop column %s on base table with materialized views.",
 -                                                                    columnName.toString(),
 -                                                                    keyspace()));
 +                    switch (def.kind)
 +                    {
 +                         case PARTITION_KEY:
 +                         case CLUSTERING:
 +                              throw new InvalidRequestException(String.format("Cannot drop PRIMARY KEY part %s", columnName));
 +                         case REGULAR:
 +                         case STATIC:
 +                              ColumnDefinition toDelete = null;
 +                              for (ColumnDefinition columnDef : cfm.partitionColumns())
 +                              {
 +                                   if (columnDef.name.equals(columnName))
 +                                   {
 +                                       toDelete = columnDef;
 +                                       break;
 +                                   }
 +                               }
 +                             assert toDelete != null;
 +                             cfm.removeColumnDefinition(toDelete);
 +                             cfm.recordColumnDrop(toDelete, deleteTimestamp  == null ? queryState.getTimestamp() : deleteTimestamp);
 +                             break;
 +                    }
 +
 +                    // If the dropped column is required by any secondary indexes
 +                    // we reject the operation, as the indexes must be dropped first
 +                    Indexes allIndexes = cfm.getIndexes();
 +                    if (!allIndexes.isEmpty())
 +                    {
 +                        ColumnFamilyStore store = Keyspace.openAndGetStore(cfm);
 +                        Set<IndexMetadata> dependentIndexes = store.indexManager.getDependentIndexes(def);
 +                        if (!dependentIndexes.isEmpty())
 +                            throw new InvalidRequestException(String.format("Cannot drop column %s because it has " +
 +                                                                            "dependent secondary indexes (%s)",
 +                                                                            def,
 +                                                                            dependentIndexes.stream()
 +                                                                                            .map(i -> i.name)
 +                                                                                            .collect(Collectors.joining(","))));
 +                    }
 +
 +                    if (!Iterables.isEmpty(views))
 +                        throw new InvalidRequestException(String.format("Cannot drop column %s on base table with materialized views.",
 +                                                                        columnName.toString(),
 +                                                                        keyspace()));
 +                }
                  break;
+             case DROP_COMPACT_STORAGE:
+                 if (!meta.isCompactTable())
+                     throw new InvalidRequestException("Cannot DROP COMPACT STORAGE on table without COMPACT STORAGE");
+ 
+                 cfm = meta.asNonCompact();
+                 break;
              case OPTS:
                  if (attrs == null)
                      throw new InvalidRequestException("ALTER TABLE WITH invoked, but no parameters found");
@@@ -265,10 -272,13 +275,12 @@@
  
                  break;
              case RENAME:
+                 cfm = meta.copy();
+ 
 -                for (Map.Entry<ColumnIdentifier.Raw, ColumnIdentifier> entry : renames.entrySet())
 +                for (Map.Entry<ColumnDefinition.Raw, ColumnDefinition.Raw> entry : renames.entrySet())
                  {
 -                    ColumnIdentifier from = entry.getKey().prepare(cfm);
 -                    ColumnIdentifier to = entry.getValue();
 -
 +                    ColumnIdentifier from = entry.getKey().getIdentifier(cfm);
 +                    ColumnIdentifier to = entry.getValue().getIdentifier(cfm);
                      cfm.renameColumn(from, to);
  
                      // If the view includes a renamed column, it must be renamed in the view table and the definition.

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d429cd0/src/java/org/apache/cassandra/cql3/statements/AuthenticationStatement.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d429cd0/src/java/org/apache/cassandra/cql3/statements/AuthorizationStatement.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d429cd0/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d429cd0/src/java/org/apache/cassandra/cql3/statements/CreateAggregateStatement.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d429cd0/src/java/org/apache/cassandra/cql3/statements/CreateIndexStatement.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/statements/CreateIndexStatement.java
index 204edf4,47d54fe..88afc6b
--- a/src/java/org/apache/cassandra/cql3/statements/CreateIndexStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/CreateIndexStatement.java
@@@ -107,18 -102,24 +107,23 @@@ public class CreateIndexStatement exten
              if (cd == null)
                  throw new InvalidRequestException("No column definition found for column " + target.column);
  
 +            if (cd.type.referencesDuration())
 +            {
 +                checkFalse(cd.type.isCollection(), "Secondary indexes are not supported on collections containing durations");
 +                checkFalse(cd.type.isTuple(), "Secondary indexes are not supported on tuples containing durations");
 +                checkFalse(cd.type.isUDT(), "Secondary indexes are not supported on UDTs containing durations");
 +                throw invalidRequest("Secondary indexes are not supported on duration columns");
 +            }
 +
              // TODO: we could lift that limitation
-             if (cfm.isCompactTable() && cd.isPrimaryKeyColumn())
-                 throw new InvalidRequestException("Secondary indexes are not supported on PRIMARY KEY columns in COMPACT STORAGE tables");
+             if (cfm.isCompactTable())
+             {
+                 if (cd.isPrimaryKeyColumn())
+                     throw new InvalidRequestException("Secondary indexes are not supported on PRIMARY KEY columns in COMPACT STORAGE tables");
+                 if (cfm.compactValueColumn().equals(cd))
+                     throw new InvalidRequestException("Secondary indexes are not supported on compact value column of COMPACT STORAGE tables");
+             }
  
 -            // It would be possible to support 2ndary index on static columns (but not without modifications of at least ExtendedFilter and
 -            // CompositesIndex) and maybe we should, but that means a query like:
 -            //     SELECT * FROM foo WHERE static_column = 'bar'
 -            // would pull the full partition every time the static column of partition is 'bar', which sounds like offering a
 -            // fair potential for foot-shooting, so I prefer leaving that to a follow up ticket once we have identified cases where
 -            // such indexing is actually useful.
 -            if (!cfm.isCompactTable() && cd.isStatic())
 -                throw new InvalidRequestException("Secondary indexes are not allowed on static columns");
 -
              if (cd.kind == ColumnDefinition.Kind.PARTITION_KEY && cfm.getKeyValidatorAsClusteringComparator().size() == 1)
                  throw new InvalidRequestException(String.format("Cannot create secondary index on partition key column %s", target.column));
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d429cd0/src/java/org/apache/cassandra/cql3/statements/CreateTableStatement.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d429cd0/src/java/org/apache/cassandra/cql3/statements/CreateViewStatement.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d429cd0/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
index c8c7570,8ae4d64..56d47b3
--- a/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
@@@ -27,9 -27,9 +27,10 @@@ import org.slf4j.LoggerFactory
  import org.apache.cassandra.auth.Permission;
  import org.apache.cassandra.config.CFMetaData;
  import org.apache.cassandra.config.ColumnDefinition;
++import org.apache.cassandra.config.Schema;
 +import org.apache.cassandra.config.ColumnDefinition.Raw;
  import org.apache.cassandra.config.ViewDefinition;
  import org.apache.cassandra.cql3.*;
 -import org.apache.cassandra.cql3.ColumnIdentifier.Raw;
  import org.apache.cassandra.cql3.functions.Function;
  import org.apache.cassandra.cql3.restrictions.StatementRestrictions;
  import org.apache.cassandra.cql3.selection.Selection;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d429cd0/src/java/org/apache/cassandra/cql3/statements/ParsedStatement.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d429cd0/src/java/org/apache/cassandra/cql3/statements/SchemaAlteringStatement.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d429cd0/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
index 33a69e7,1e867bc..d86a47d
--- a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
@@@ -945,17 -885,16 +945,17 @@@ public class SelectStatement implement
              this.selectClause = selectClause;
              this.whereClause = whereClause;
              this.limit = limit;
 +            this.perPartitionLimit = perPartitionLimit;
          }
  
-         public ParsedStatement.Prepared prepare() throws InvalidRequestException
+         public ParsedStatement.Prepared prepare(ClientState clientState) throws InvalidRequestException
          {
-             return prepare(false);
+             return prepare(false, clientState);
          }
  
-         public ParsedStatement.Prepared prepare(boolean forView) throws InvalidRequestException
+         public ParsedStatement.Prepared prepare(boolean forView, ClientState clientState) throws InvalidRequestException
          {
-             CFMetaData cfm = ThriftValidation.validateColumnFamily(keyspace(), columnFamily());
+             CFMetaData cfm = ThriftValidation.validateColumnFamilyWithCompactMode(keyspace(), columnFamily(), clientState.isNoCompactMode());
              VariableSpecifications boundNames = getBoundVariables();
  
              Selection selection = selectClause.isEmpty()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d429cd0/src/java/org/apache/cassandra/cql3/statements/TruncateStatement.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d429cd0/src/java/org/apache/cassandra/cql3/statements/UseStatement.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d429cd0/src/java/org/apache/cassandra/db/view/View.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d429cd0/src/java/org/apache/cassandra/index/internal/keys/KeysSearcher.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d429cd0/src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java
index 9e42101,39f7339..694fe37
--- a/src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java
@@@ -555,51 -527,24 +555,51 @@@ public class CQLSSTableWriter implement
              }
          }
  
 -        @SuppressWarnings("resource")
 -        public CQLSSTableWriter build()
 +        private Types createTypes(String keyspace)
          {
 -            if (directory == null)
 -                throw new IllegalStateException("No ouptut directory specified, you should provide a directory with inDirectory()");
 -            if (schema == null)
 -                throw new IllegalStateException("Missing schema, you should provide the schema for the SSTable to create with forTable()");
 -            if (insert == null)
 -                throw new IllegalStateException("No insert statement specified, you should provide an insert statement through using()");
 +            Types.RawBuilder builder = Types.rawBuilder(keyspace);
 +            for (CreateTypeStatement st : typeStatements)
 +                st.addToRawBuilder(builder);
 +            return builder.build();
 +        }
  
 -            AbstractSSTableSimpleWriter writer = sorted
 -                                               ? new SSTableSimpleWriter(directory, schema, insert.updatedColumns())
 -                                               : new SSTableSimpleUnsortedWriter(directory, schema, insert.updatedColumns(), bufferSizeInMB);
 +        /**
 +         * Creates the table according to schema statement
 +         *
 +         * @param types types this table should be created with
 +         */
 +        private CFMetaData createTable(Types types)
 +        {
 +            CreateTableStatement statement = (CreateTableStatement) schemaStatement.prepare(types).statement;
 +            statement.validate(ClientState.forInternalCalls());
  
 -            if (formatType != null)
 -                writer.setSSTableFormatType(formatType);
 +            CFMetaData cfMetaData = statement.getCFMetaData();
 +
 +            if (partitioner != null)
 +                return cfMetaData.copy(partitioner);
 +            else
 +                return cfMetaData;
 +        }
 +
 +        /**
 +         * Prepares insert statement for writing data to SSTable
 +         *
 +         * @return prepared Insert statement and it's bound names
 +         */
 +        private Pair<UpdateStatement, List<ColumnSpecification>> prepareInsert()
 +        {
-             ParsedStatement.Prepared cqlStatement = insertStatement.prepare();
++            ParsedStatement.Prepared cqlStatement = insertStatement.prepare(ClientState.forInternalCalls());
 +            UpdateStatement insert = (UpdateStatement) cqlStatement.statement;
 +            insert.validate(ClientState.forInternalCalls());
 +
 +            if (insert.hasConditions())
 +                throw new IllegalArgumentException("Conditional statements are not supported");
 +            if (insert.isCounter())
 +                throw new IllegalArgumentException("Counter update statements are not supported");
 +            if (cqlStatement.boundNames.isEmpty())
 +                throw new IllegalArgumentException("Provided insert statement has no bind variables");
  
 -            return new CQLSSTableWriter(writer, insert, boundNames);
 +            return Pair.create(insert, cqlStatement.boundNames);
          }
      }
  }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d429cd0/src/java/org/apache/cassandra/repair/RepairRunnable.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/repair/RepairRunnable.java
index c9eed54,77726d4..52cc29d
--- a/src/java/org/apache/cassandra/repair/RepairRunnable.java
+++ b/src/java/org/apache/cassandra/repair/RepairRunnable.java
@@@ -389,8 -385,8 +390,8 @@@ public class RepairRunnable extends Wra
                      throw new Exception("no tracestate");
  
                  String format = "select event_id, source, activity from %s.%s where session_id = ? and event_id > ? and event_id < ?;";
 -                String query = String.format(format, TraceKeyspace.NAME, TraceKeyspace.EVENTS);
 +                String query = String.format(format, SchemaConstants.TRACE_KEYSPACE_NAME, TraceKeyspace.EVENTS);
-                 SelectStatement statement = (SelectStatement) QueryProcessor.parseStatement(query).prepare().statement;
+                 SelectStatement statement = (SelectStatement) QueryProcessor.parseStatement(query).prepare(ClientState.forInternalCalls()).statement;
  
                  ByteBuffer sessionIdBytes = ByteBufferUtil.bytes(sessionId);
                  InetAddress source = FBUtilities.getBroadcastAddress();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d429cd0/src/java/org/apache/cassandra/service/ClientState.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d429cd0/src/java/org/apache/cassandra/thrift/ThriftValidation.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d429cd0/src/java/org/apache/cassandra/transport/messages/StartupMessage.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/transport/messages/StartupMessage.java
index bf4a619,774be6a..eb82292
--- a/src/java/org/apache/cassandra/transport/messages/StartupMessage.java
+++ b/src/java/org/apache/cassandra/transport/messages/StartupMessage.java
@@@ -35,7 -35,7 +35,8 @@@ public class StartupMessage extends Mes
  {
      public static final String CQL_VERSION = "CQL_VERSION";
      public static final String COMPRESSION = "COMPRESSION";
 +    public static final String PROTOCOL_VERSIONS = "PROTOCOL_VERSIONS";
+     public static final String NO_COMPACT = "NO_COMPACT";
  
      public static final Message.Codec<StartupMessage> codec = new Message.Codec<StartupMessage>()
      {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d429cd0/test/unit/org/apache/cassandra/cql3/ViewTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/cql3/ViewTest.java
index 7717a4d,136ae1c..eb9c855
--- a/test/unit/org/apache/cassandra/cql3/ViewTest.java
+++ b/test/unit/org/apache/cassandra/cql3/ViewTest.java
@@@ -48,8 -48,9 +48,10 @@@ import org.apache.cassandra.db.Keyspace
  import org.apache.cassandra.db.SystemKeyspace;
  import org.apache.cassandra.db.compaction.CompactionManager;
  import org.apache.cassandra.db.marshal.AsciiType;
+ import org.apache.cassandra.exceptions.InvalidRequestException;
+ import org.apache.cassandra.exceptions.SyntaxException;
  import org.apache.cassandra.schema.KeyspaceParams;
 +import org.apache.cassandra.transport.ProtocolVersion;
  import org.apache.cassandra.utils.FBUtilities;
  
  import static org.junit.Assert.assertEquals;
@@@ -1407,4 -1344,49 +1408,17 @@@ public class ViewTest extends CQLTeste
  
          assertRows(execute("SELECT count(*) FROM mv_test"), row(1024L));
      }
+ 
 -    @Test
 -    public void testFrozenCollectionsWithComplicatedInnerType() throws Throwable
 -    {
 -        createTable("CREATE TABLE %s (k int, intval int,  listval frozen<list<tuple<text,text>>>, PRIMARY KEY (k))");
 -
 -        execute("USE " + keyspace());
 -        executeNet(protocolVersion, "USE " + keyspace());
 -
 -        createView("mv",
 -                   "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE k IS NOT NULL AND listval IS NOT NULL PRIMARY KEY (k, listval)");
 -
 -        updateView("INSERT INTO %s (k, intval, listval) VALUES (?, ?, fromJson(?))",
 -                   0,
 -                   0,
 -                   "[[\"a\", \"1\"], [\"b\", \"2\"], [\"c\", \"3\"]]");
 -
 -        // verify input
 -        assertRows(execute("SELECT k, toJson(listval) FROM %s WHERE k = ?", 0),
 -                   row(0, "[[\"a\", \"1\"], [\"b\", \"2\"], [\"c\", \"3\"]]"));
 -        assertRows(execute("SELECT k, toJson(listval) from mv"),
 -                   row(0, "[[\"a\", \"1\"], [\"b\", \"2\"], [\"c\", \"3\"]]"));
 -
 -        // update listval with the same value and it will be compared in view generator
 -        updateView("INSERT INTO %s (k, listval) VALUES (?, fromJson(?))",
 -                   0,
 -                   "[[\"a\", \"1\"], [\"b\", \"2\"], [\"c\", \"3\"]]");
 -        // verify result
 -        assertRows(execute("SELECT k, toJson(listval) FROM %s WHERE k = ?", 0),
 -                   row(0, "[[\"a\", \"1\"], [\"b\", \"2\"], [\"c\", \"3\"]]"));
 -        assertRows(execute("SELECT k, toJson(listval) from mv"),
 -                   row(0, "[[\"a\", \"1\"], [\"b\", \"2\"], [\"c\", \"3\"]]"));
 -    }
+ 
+     @Test(expected = SyntaxException.class)
+     public void emptyViewNameTest() throws Throwable
+     {
+         execute("CREATE MATERIALIZED VIEW \"\" AS SELECT a, b FROM tbl WHERE b IS NOT NULL PRIMARY KEY (b, a)");
+     }
+ 
+      @Test(expected = SyntaxException.class)
+      public void emptyBaseTableNameTest() throws Throwable
+      {
+          execute("CREATE MATERIALIZED VIEW myview AS SELECT a, b FROM \"\" WHERE b IS NOT NULL PRIMARY KEY (b, a)");
+      }
  }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d429cd0/test/unit/org/apache/cassandra/cql3/validation/entities/UFTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/cql3/validation/entities/UFTest.java
index af9ec1a,6e6af19..6f3616c
--- a/test/unit/org/apache/cassandra/cql3/validation/entities/UFTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/UFTest.java
@@@ -36,11 -34,13 +36,12 @@@ import org.apache.cassandra.cql3.functi
  import org.apache.cassandra.cql3.functions.UDFunction;
  import org.apache.cassandra.db.marshal.CollectionType;
  import org.apache.cassandra.exceptions.InvalidRequestException;
+ import org.apache.cassandra.exceptions.SyntaxException;
  import org.apache.cassandra.schema.KeyspaceMetadata;
  import org.apache.cassandra.service.ClientState;
 -import org.apache.cassandra.transport.Event;
 -import org.apache.cassandra.transport.Server;
 +import org.apache.cassandra.transport.*;
 +import org.apache.cassandra.transport.ProtocolVersion;
  import org.apache.cassandra.transport.messages.ResultMessage;
 -import org.apache.cassandra.utils.ByteBufferUtil;
  
  public class UFTest extends CQLTester
  {
@@@ -872,222 -863,134 +873,246 @@@
      }
  
      @Test
 -    public void testEmptyString() throws Throwable
 +    public void testArgumentGenerics() throws Throwable
      {
          createTable("CREATE TABLE %s (key int primary key, sval text, aval ascii, bval blob, empty_int int)");
 -        execute("INSERT INTO %s (key, sval, aval, bval, empty_int) VALUES (?, ?, ?, ?, blobAsInt(0x))", 1, "", "", ByteBuffer.allocate(0));
  
 -        String fNameSRC = createFunction(KEYSPACE_PER_TEST, "text",
 -                                         "CREATE OR REPLACE FUNCTION %s(val text) " +
 -                                         "CALLED ON NULL INPUT " +
 -                                         "RETURNS text " +
 -                                         "LANGUAGE JAVA\n" +
 -                                         "AS 'return val;'");
 +        String typeName = createType("CREATE TYPE %s (txt text, i int)");
  
 -        String fNameSCC = createFunction(KEYSPACE_PER_TEST, "text",
 -                                         "CREATE OR REPLACE FUNCTION %s(val text) " +
 -                                         "CALLED ON NULL INPUT " +
 -                                         "RETURNS text " +
 -                                         "LANGUAGE JAVA\n" +
 -                                         "AS 'return \"\";'");
 +        createFunction(KEYSPACE, "map<text,bigint>,list<text>",
 +                       "CREATE FUNCTION IF NOT EXISTS %s(state map<text,bigint>, styles list<text>)\n" +
 +                       "  RETURNS NULL ON NULL INPUT\n" +
 +                       "  RETURNS map<text,bigint>\n" +
 +                       "  LANGUAGE java\n" +
 +                       "  AS $$\n" +
 +                       "    for (String style : styles) {\n" +
 +                       "      if (state.containsKey(style)) {\n" +
 +                       "        state.put(style, state.get(style) + 1L);\n" +
 +                       "      } else {\n" +
 +                       "        state.put(style, 1L);\n" +
 +                       "      }\n" +
 +                       "    }\n" +
 +                       "    return state;\n" +
 +                       "  $$");
 +
 +        createFunction(KEYSPACE, "text",
 +                                  "CREATE OR REPLACE FUNCTION %s("                 +
 +                                  "  listText list<text>,"                         +
 +                                  "  setText set<text>,"                           +
 +                                  "  mapTextInt map<text, int>,"                   +
 +                                  "  mapListTextSetInt map<frozen<list<text>>, frozen<set<int>>>," +
 +                                  "  mapTextTuple map<text, frozen<tuple<int, text>>>," +
 +                                  "  mapTextType map<text, frozen<" + typeName + ">>" +
 +                                  ") "                                             +
 +                                  "CALLED ON NULL INPUT "                          +
 +                                  "RETURNS map<frozen<list<text>>, frozen<set<int>>> " +
 +                                  "LANGUAGE JAVA\n"                                +
 +                                  "AS $$" +
 +                                  "     for (String s : listtext) {};" +
 +                                  "     for (String s : settext) {};" +
 +                                  "     for (String s : maptextint.keySet()) {};" +
 +                                  "     for (Integer s : maptextint.values()) {};" +
 +                                  "     for (java.util.List<String> l : maplisttextsetint.keySet()) {};" +
 +                                  "     for (java.util.Set<Integer> s : maplisttextsetint.values()) {};" +
 +                                  "     for (com.datastax.driver.core.TupleValue t : maptexttuple.values()) {};" +
 +                                  "     for (com.datastax.driver.core.UDTValue u : maptexttype.values()) {};" +
 +                                  "     return maplisttextsetint;" +
 +                                  "$$");
 +    }
  
 -        String fNameSRN = createFunction(KEYSPACE_PER_TEST, "text",
 -                                         "CREATE OR REPLACE FUNCTION %s(val text) " +
 -                                         "RETURNS NULL ON NULL INPUT " +
 -                                         "RETURNS text " +
 -                                         "LANGUAGE JAVA\n" +
 -                                         "AS 'return val;'");
 +    @Test
 +    public void testArgAndReturnTypes() throws Throwable
 +    {
  
 -        String fNameSCN = createFunction(KEYSPACE_PER_TEST, "text",
 -                                         "CREATE OR REPLACE FUNCTION %s(val text) " +
 -                                         "RETURNS NULL ON NULL INPUT " +
 -                                         "RETURNS text " +
 -                                         "LANGUAGE JAVA\n" +
 -                                         "AS 'return \"\";'");
 -
 -        String fNameBRC = createFunction(KEYSPACE_PER_TEST, "blob",
 -                                         "CREATE OR REPLACE FUNCTION %s(val blob) " +
 -                                         "CALLED ON NULL INPUT " +
 -                                         "RETURNS blob " +
 -                                         "LANGUAGE JAVA\n" +
 -                                         "AS 'return val;'");
 -
 -        String fNameBCC = createFunction(KEYSPACE_PER_TEST, "blob",
 -                                         "CREATE OR REPLACE FUNCTION %s(val blob) " +
 -                                         "CALLED ON NULL INPUT " +
 -                                         "RETURNS blob " +
 -                                         "LANGUAGE JAVA\n" +
 -                                         "AS 'return ByteBuffer.allocate(0);'");
 -
 -        String fNameBRN = createFunction(KEYSPACE_PER_TEST, "blob",
 -                                         "CREATE OR REPLACE FUNCTION %s(val blob) " +
 -                                         "RETURNS NULL ON NULL INPUT " +
 -                                         "RETURNS blob " +
 -                                         "LANGUAGE JAVA\n" +
 -                                         "AS 'return val;'");
 +        String type = KEYSPACE + '.' + createType("CREATE TYPE %s (txt text, i int)");
  
 -        String fNameBCN = createFunction(KEYSPACE_PER_TEST, "blob",
 -                                         "CREATE OR REPLACE FUNCTION %s(val blob) " +
 -                                         "RETURNS NULL ON NULL INPUT " +
 -                                         "RETURNS blob " +
 -                                         "LANGUAGE JAVA\n" +
 -                                         "AS 'return ByteBuffer.allocate(0);'");
 +        createTable("CREATE TABLE %s (key int primary key, udt frozen<" + type + ">)");
 +        execute("INSERT INTO %s (key, udt) VALUES (1, {txt: 'foo', i: 42})");
  
 -        String fNameIRC = createFunction(KEYSPACE_PER_TEST, "int",
 -                                         "CREATE OR REPLACE FUNCTION %s(val int) " +
 -                                         "CALLED ON NULL INPUT " +
 -                                         "RETURNS int " +
 -                                         "LANGUAGE JAVA\n" +
 -                                         "AS 'return val;'");
 +        // Java UDFs
  
 -        String fNameICC = createFunction(KEYSPACE_PER_TEST, "int",
 -                                         "CREATE OR REPLACE FUNCTION %s(val int) " +
 -                                         "CALLED ON NULL INPUT " +
 -                                         "RETURNS int " +
 -                                         "LANGUAGE JAVA\n" +
 -                                         "AS 'return 0;'");
 +        String f = createFunction(KEYSPACE, "int",
 +                                  "CREATE OR REPLACE FUNCTION %s(val int) " +
 +                                  "RETURNS NULL ON NULL INPUT " +
 +                                  "RETURNS " + type + ' ' +
 +                                  "LANGUAGE JAVA\n" +
 +                                  "AS 'return udfContext.newReturnUDTValue();';");
 +
 +        assertRows(execute("SELECT " + f + "(key) FROM %s"),
 +                   row(userType("txt", null, "i", null)));
 +
 +        f = createFunction(KEYSPACE, "int",
 +                           "CREATE OR REPLACE FUNCTION %s(val " + type + ") " +
 +                           "RETURNS NULL ON NULL INPUT " +
 +                           "RETURNS " + type + ' ' +
 +                           "LANGUAGE JAVA\n" +
 +                           "AS $$" +
 +                           "   com.datastax.driver.core.UDTValue udt = udfContext.newArgUDTValue(\"val\");" +
 +                           "   udt.setString(\"txt\", \"baz\");" +
 +                           "   udt.setInt(\"i\", 88);" +
 +                           "   return udt;" +
 +                           "$$;");
 +
 +        assertRows(execute("SELECT " + f + "(udt) FROM %s"),
 +                   row(userType("txt", "baz", "i", 88)));
 +
 +        f = createFunction(KEYSPACE, "int",
 +                           "CREATE OR REPLACE FUNCTION %s(val " + type + ") " +
 +                           "RETURNS NULL ON NULL INPUT " +
 +                           "RETURNS tuple<text, int>" +
 +                           "LANGUAGE JAVA\n" +
 +                           "AS $$" +
 +                           "   com.datastax.driver.core.TupleValue tv = udfContext.newReturnTupleValue();" +
 +                           "   tv.setString(0, \"baz\");" +
 +                           "   tv.setInt(1, 88);" +
 +                           "   return tv;" +
 +                           "$$;");
 +
 +        assertRows(execute("SELECT " + f + "(udt) FROM %s"),
 +                   row(tuple("baz", 88)));
 +
 +        // JavaScript UDFs
 +
 +        f = createFunction(KEYSPACE, "int",
 +                           "CREATE OR REPLACE FUNCTION %s(val int) " +
 +                           "RETURNS NULL ON NULL INPUT " +
 +                           "RETURNS " + type + ' ' +
 +                           "LANGUAGE JAVASCRIPT\n" +
 +                           "AS $$" +
 +                           "   udt = udfContext.newReturnUDTValue();" +
 +                           "   udt;" +
 +                           "$$;");
 +
 +        assertRows(execute("SELECT " + f + "(key) FROM %s"),
 +                   row(userType("txt", null, "i", null)));
 +
 +        f = createFunction(KEYSPACE, "int",
 +                           "CREATE OR REPLACE FUNCTION %s(val " + type + ") " +
 +                           "RETURNS NULL ON NULL INPUT " +
 +                           "RETURNS " + type + ' ' +
 +                           "LANGUAGE JAVASCRIPT\n" +
 +                           "AS $$" +
 +                           "   udt = udfContext.newArgUDTValue(0);" +
 +                           "   udt.setString(\"txt\", \"baz\");" +
 +                           "   udt.setInt(\"i\", 88);" +
 +                           "   udt;" +
 +                           "$$;");
 +
 +        assertRows(execute("SELECT " + f + "(udt) FROM %s"),
 +                   row(userType("txt", "baz", "i", 88)));
 +
 +        f = createFunction(KEYSPACE, "int",
 +                           "CREATE OR REPLACE FUNCTION %s(val " + type + ") " +
 +                           "RETURNS NULL ON NULL INPUT " +
 +                           "RETURNS tuple<text, int>" +
 +                           "LANGUAGE JAVASCRIPT\n" +
 +                           "AS $$" +
 +                           "   tv = udfContext.newReturnTupleValue();" +
 +                           "   tv.setString(0, \"baz\");" +
 +                           "   tv.setInt(1, 88);" +
 +                           "   tv;" +
 +                           "$$;");
 +
 +        assertRows(execute("SELECT " + f + "(udt) FROM %s"),
 +                   row(tuple("baz", 88)));
 +
 +        createFunction(KEYSPACE, "map",
 +                       "CREATE FUNCTION %s(my_map map<text, text>)\n" +
 +                       "         CALLED ON NULL INPUT\n" +
 +                       "         RETURNS text\n" +
 +                       "         LANGUAGE java\n" +
 +                       "         AS $$\n" +
 +                       "             String buffer = \"\";\n" +
 +                       "             for(java.util.Map.Entry<String, String> entry: my_map.entrySet()) {\n" +
 +                       "                 buffer = buffer + entry.getKey() + \": \" + entry.getValue() + \", \";\n" +
 +                       "             }\n" +
 +                       "             return buffer;\n" +
 +                       "         $$;\n");
 +    }
  
 -        String fNameIRN = createFunction(KEYSPACE_PER_TEST, "int",
 -                                         "CREATE OR REPLACE FUNCTION %s(val int) " +
 -                                         "RETURNS NULL ON NULL INPUT " +
 -                                         "RETURNS int " +
 -                                         "LANGUAGE JAVA\n" +
 -                                         "AS 'return val;'");
 +    @Test
 +    public void testImportJavaUtil() throws Throwable
 +    {
 +        createFunction(KEYSPACE, "list<text>",
 +                "CREATE OR REPLACE FUNCTION %s(listText list<text>) "                                             +
 +                        "CALLED ON NULL INPUT "                          +
 +                        "RETURNS set<text> " +
 +                        "LANGUAGE JAVA\n"                                +
 +                        "AS $$\n" +
 +                        "     Set<String> set = new HashSet<String>(); " +
 +                        "     for (String s : listtext) {" +
 +                        "            set.add(s);" +
 +                        "     }" +
 +                        "     return set;" +
 +                        "$$");
  
 -        String fNameICN = createFunction(KEYSPACE_PER_TEST, "int",
 -                                         "CREATE OR REPLACE FUNCTION %s(val int) " +
 -                                         "RETURNS NULL ON NULL INPUT " +
 -                                         "RETURNS int " +
 -                                         "LANGUAGE JAVA\n" +
 -                                         "AS 'return 0;'");
 -
 -        assertRows(execute("SELECT " + fNameSRC + "(sval) FROM %s"), row(""));
 -        assertRows(execute("SELECT " + fNameSRN + "(sval) FROM %s"), row(""));
 -        assertRows(execute("SELECT " + fNameSCC + "(sval) FROM %s"), row(""));
 -        assertRows(execute("SELECT " + fNameSCN + "(sval) FROM %s"), row(""));
 -        assertRows(execute("SELECT " + fNameSRC + "(aval) FROM %s"), row(""));
 -        assertRows(execute("SELECT " + fNameSRN + "(aval) FROM %s"), row(""));
 -        assertRows(execute("SELECT " + fNameSCC + "(aval) FROM %s"), row(""));
 -        assertRows(execute("SELECT " + fNameSCN + "(aval) FROM %s"), row(""));
 -        assertRows(execute("SELECT " + fNameBRC + "(bval) FROM %s"), row(ByteBufferUtil.EMPTY_BYTE_BUFFER));
 -        assertRows(execute("SELECT " + fNameBRN + "(bval) FROM %s"), row(ByteBufferUtil.EMPTY_BYTE_BUFFER));
 -        assertRows(execute("SELECT " + fNameBCC + "(bval) FROM %s"), row(ByteBufferUtil.EMPTY_BYTE_BUFFER));
 -        assertRows(execute("SELECT " + fNameBCN + "(bval) FROM %s"), row(ByteBufferUtil.EMPTY_BYTE_BUFFER));
 -        assertRows(execute("SELECT " + fNameIRC + "(empty_int) FROM %s"), row(new Object[]{ null }));
 -        assertRows(execute("SELECT " + fNameIRN + "(empty_int) FROM %s"), row(new Object[]{ null }));
 -        assertRows(execute("SELECT " + fNameICC + "(empty_int) FROM %s"), row(0));
 -        assertRows(execute("SELECT " + fNameICN + "(empty_int) FROM %s"), row(new Object[]{ null }));
 +    }
 +
 +    @Test
 +    public void testAnyUserTupleType() throws Throwable
 +    {
 +        createTable("CREATE TABLE %s (key int primary key, sval text)");
 +        execute("INSERT INTO %s (key, sval) VALUES (1, 'foo')");
 +
 +        String udt = createType("CREATE TYPE %s (a int, b text, c bigint)");
 +
 +        String fUdt = createFunction(KEYSPACE, "text",
 +                                     "CREATE OR REPLACE FUNCTION %s(arg text) " +
 +                                     "CALLED ON NULL INPUT " +
 +                                     "RETURNS " + udt + " " +
 +                                     "LANGUAGE JAVA\n" +
 +                                     "AS $$\n" +
 +                                     "    UDTValue udt = udfContext.newUDTValue(\"" + udt + "\");" +
 +                                     "    udt.setInt(\"a\", 42);" +
 +                                     "    udt.setString(\"b\", \"42\");" +
 +                                     "    udt.setLong(\"c\", 4242);" +
 +                                     "    return udt;" +
 +                                     "$$");
 +
 +        assertRows(execute("SELECT " + fUdt + "(sval) FROM %s"),
 +                   row(userType("a", 42, "b", "42", "c", 4242L)));
 +
 +        String fTup = createFunction(KEYSPACE, "text",
 +                                     "CREATE OR REPLACE FUNCTION %s(arg text) " +
 +                                     "CALLED ON NULL INPUT " +
 +                                     "RETURNS tuple<int, " + udt + "> " +
 +                                     "LANGUAGE JAVA\n" +
 +                                     "AS $$\n" +
 +                                     "    UDTValue udt = udfContext.newUDTValue(\"" + udt + "\");" +
 +                                     "    udt.setInt(\"a\", 42);" +
 +                                     "    udt.setString(\"b\", \"42\");" +
 +                                     "    udt.setLong(\"c\", 4242);" +
 +                                     "    TupleValue tup = udfContext.newTupleValue(\"tuple<int," + udt + ">\");" +
 +                                     "    tup.setInt(0, 88);" +
 +                                     "    tup.setUDTValue(1, udt);" +
 +                                     "    return tup;" +
 +                                     "$$");
 +
 +        assertRows(execute("SELECT " + fTup + "(sval) FROM %s"),
 +                   row(tuple(88, userType("a", 42, "b", "42", "c", 4242L))));
      }
+ 
+     @Test(expected = SyntaxException.class)
+     public void testEmptyFunctionName() throws Throwable
+     {
+         execute("CREATE FUNCTION IF NOT EXISTS " + KEYSPACE + ".\"\" (arg int)\n" +
+                 "  RETURNS NULL ON NULL INPUT\n" +
+                 "  RETURNS int\n" +
+                 "  LANGUAGE java\n" +
+                 "  AS $$\n" +
+                 "    return a;\n" +
+                 "  $$");
+     }
+ 
+     @Test(expected = SyntaxException.class)
+     public void testEmptyArgName() throws Throwable
+     {
+         execute("CREATE FUNCTION IF NOT EXISTS " + KEYSPACE + ".myfn (\"\" int)\n" +
+                 "  RETURNS NULL ON NULL INPUT\n" +
+                 "  RETURNS int\n" +
+                 "  LANGUAGE java\n" +
+                 "  AS $$\n" +
+                 "    return a;\n" +
+                 "  $$");
+     }
  }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d429cd0/test/unit/org/apache/cassandra/cql3/validation/entities/UserTypesTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/cql3/validation/entities/UserTypesTest.java
index 9ea5572,68c0b8c..646484c
--- a/test/unit/org/apache/cassandra/cql3/validation/entities/UserTypesTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/UserTypesTest.java
@@@ -872,12 -708,31 +873,31 @@@ public class UserTypesTest extends CQLT
  
          execute("ALTER TYPE " + columnType + " ADD b int");
          execute("UPDATE %s SET s = s + ?, v = ? WHERE pk = ? AND c = ?",
 -                set(userType(1, 1), userType(1, 2), userType(2, 1)), 2, 1, 1);
 +                set(userType("a", 1, "b", 1), userType("a", 1, "b", 2), userType("a", 2, "b", 1)), 2, 1, 1);
  
          assertRows(execute("SELECT * FROM %s WHERE pk = ? AND c = ?", 1, 1),
 -                       row(1, 1,set(userType(1), userType(1, 1), userType(1, 2), userType(2), userType(2, 1)), 2));
 +                       row(1, 1,set(userType("a", 1), userType("a", 1, "b", 1), userType("a", 1, "b", 2), userType("a", 2), userType("a", 2, "b", 1)), 2));
      }
  
+     @Test(expected = SyntaxException.class)
+     public void emptyTypeNameTest() throws Throwable
+     {
+         execute("CREATE TYPE \"\" (a int, b int)");
+     }
+ 
+     @Test(expected = SyntaxException.class)
+     public void emptyFieldNameTest() throws Throwable
+     {
+         execute("CREATE TYPE mytype (\"\" int, b int)");
+     }
+ 
+     @Test(expected = SyntaxException.class)
+     public void renameColumnToEmpty() throws Throwable
+     {
+         String typeName = createType("CREATE TYPE %s (a int, b int)");
+         execute(String.format("ALTER TYPE %s.%s RENAME b TO \"\"", keyspace(), typeName));
+     }
+ 
      private String typeWithKs(String type1)
      {
          return keyspace() + '.' + type1;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d429cd0/test/unit/org/apache/cassandra/cql3/validation/operations/AlterTest.java
----------------------------------------------------------------------


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


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

Posted by if...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/07fbd8ee/test/unit/org/apache/cassandra/cql3/validation/operations/SelectMultiColumnRelationTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/SelectMultiColumnRelationTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/SelectMultiColumnRelationTest.java
index b91650d..52a7f47 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/SelectMultiColumnRelationTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/SelectMultiColumnRelationTest.java
@@ -32,781 +32,754 @@ public class SelectMultiColumnRelationTest extends CQLTester
     @Test
     public void testSingleClusteringInvalidQueries() throws Throwable
     {
-        for (String compactOption : new String[] { "", " WITH COMPACT STORAGE" })
-        {
-            createTable("CREATE TABLE %s (a int, b int, c int, PRIMARY KEY (a, b))" + compactOption);
-
-            assertInvalidSyntax("SELECT * FROM %s WHERE () = (?, ?)", 1, 2);
-            assertInvalidMessage("b cannot be restricted by more than one relation if it includes an Equal",
-                                 "SELECT * FROM %s WHERE a = 0 AND (b) = (?) AND (b) > (?)", 0, 0);
-            assertInvalidMessage("More than one restriction was found for the start bound on b",
-                                 "SELECT * FROM %s WHERE a = 0 AND (b) > (?) AND (b) > (?)", 0, 1);
-            assertInvalidMessage("More than one restriction was found for the start bound on b",
-                                 "SELECT * FROM %s WHERE a = 0 AND (b) > (?) AND b > ?", 0, 1);
-            assertInvalidMessage("Multi-column relations can only be applied to clustering columns but was applied to: a",
-                                 "SELECT * FROM %s WHERE (a, b) = (?, ?)", 0, 0);
-        }
+        createTable("CREATE TABLE %s (a int, b int, c int, PRIMARY KEY (a, b))");
+
+        assertInvalidSyntax("SELECT * FROM %s WHERE () = (?, ?)", 1, 2);
+        assertInvalidMessage("b cannot be restricted by more than one relation if it includes an Equal",
+                             "SELECT * FROM %s WHERE a = 0 AND (b) = (?) AND (b) > (?)", 0, 0);
+        assertInvalidMessage("More than one restriction was found for the start bound on b",
+                             "SELECT * FROM %s WHERE a = 0 AND (b) > (?) AND (b) > (?)", 0, 1);
+        assertInvalidMessage("More than one restriction was found for the start bound on b",
+                             "SELECT * FROM %s WHERE a = 0 AND (b) > (?) AND b > ?", 0, 1);
+        assertInvalidMessage("Multi-column relations can only be applied to clustering columns but was applied to: a",
+                             "SELECT * FROM %s WHERE (a, b) = (?, ?)", 0, 0);
     }
 
     @Test
     public void testMultiClusteringInvalidQueries() throws Throwable
     {
-        for (String compactOption : new String[] { "", " WITH COMPACT STORAGE" })
-        {
-            createTable("CREATE TABLE %s (a int, b int, c int, d int, PRIMARY KEY (a, b, c, d))" + compactOption);
-
-            assertInvalidSyntax("SELECT * FROM %s WHERE a = 0 AND (b, c) > ()");
-            assertInvalidMessage("Expected 2 elements in value tuple, but got 3: (?, ?, ?)",
-                                 "SELECT * FROM %s WHERE a = 0 AND (b, c) > (?, ?, ?)", 1, 2, 3);
-            assertInvalidMessage("Invalid null value in condition for column c",
-                                 "SELECT * FROM %s WHERE a = 0 AND (b, c) > (?, ?)", 1, null);
-
-            // Wrong order of columns
-            assertInvalidMessage("Clustering columns must appear in the PRIMARY KEY order in multi-column relations: (d, c, b) = (?, ?, ?)",
-                                 "SELECT * FROM %s WHERE a = 0 AND (d, c, b) = (?, ?, ?)", 0, 0, 0);
-            assertInvalidMessage("Clustering columns must appear in the PRIMARY KEY order in multi-column relations: (d, c, b) > (?, ?, ?)",
-                                 "SELECT * FROM %s WHERE a = 0 AND (d, c, b) > (?, ?, ?)", 0, 0, 0);
-
-            // Wrong number of values
-            assertInvalidMessage("Expected 3 elements in value tuple, but got 2: (?, ?)",
-                                 "SELECT * FROM %s WHERE a=0 AND (b, c, d) IN ((?, ?))", 0, 1);
-            assertInvalidMessage("Expected 3 elements in value tuple, but got 5: (?, ?, ?, ?, ?)",
-                                 "SELECT * FROM %s WHERE a=0 AND (b, c, d) IN ((?, ?, ?, ?, ?))", 0, 1, 2, 3, 4);
-
-            // Missing first clustering column
-            assertInvalidMessage("PRIMARY KEY column \"c\" cannot be restricted as preceding column \"b\" is not restricted",
-                                 "SELECT * FROM %s WHERE a = 0 AND (c, d) = (?, ?)", 0, 0);
-            assertInvalidMessage("PRIMARY KEY column \"c\" cannot be restricted as preceding column \"b\" is not restricted",
-                                 "SELECT * FROM %s WHERE a = 0 AND (c, d) > (?, ?)", 0, 0);
-
-            // Nulls
-            assertInvalidMessage("Invalid null value for column d",
-                                 "SELECT * FROM %s WHERE a = 0 AND (b, c, d) = (?, ?, ?)", 1, 2, null);
-            assertInvalidMessage("Invalid null value for column d",
-                                 "SELECT * FROM %s WHERE a = 0 AND (b, c, d) IN ((?, ?, ?))", 1, 2, null);
-            assertInvalidMessage("Invalid null value in condition for columns: [b, c, d]",
-                                 "SELECT * FROM %s WHERE a = 0 AND (b, c, d) IN ((?, ?, ?), (?, ?, ?))", 1, 2, null, 2, 1 ,4);
-
-            // Wrong type for 'd'
-            assertInvalid("SELECT * FROM %s WHERE a = 0 AND (b, c, d) = (?, ?, ?)", 1, 2, "foobar");
-            assertInvalid("SELECT * FROM %s WHERE a = 0 AND b = (?, ?, ?)", 1, 2, 3);
-
-            // Mix single and tuple inequalities
-             assertInvalidMessage("Column \"c\" cannot be restricted by two inequalities not starting with the same column",
-                                 "SELECT * FROM %s WHERE a = 0 AND (b, c, d) > (?, ?, ?) AND c < ?", 0, 1, 0, 1);
-            assertInvalidMessage("Column \"c\" cannot be restricted by two inequalities not starting with the same column",
-                                 "SELECT * FROM %s WHERE a = 0 AND c > ? AND (b, c, d) < (?, ?, ?)", 1, 1, 1, 0);
-
-            assertInvalidMessage("Multi-column relations can only be applied to clustering columns but was applied to: a",
-                                 "SELECT * FROM %s WHERE (a, b, c, d) IN ((?, ?, ?, ?))", 0, 1, 2, 3);
-            assertInvalidMessage("PRIMARY KEY column \"c\" cannot be restricted as preceding column \"b\" is not restricted",
-                                 "SELECT * FROM %s WHERE (c, d) IN ((?, ?))", 0, 1);
-
-            assertInvalidMessage("Clustering column \"c\" cannot be restricted (preceding column \"b\" is restricted by a non-EQ relation)",
-                                 "SELECT * FROM %s WHERE a = ? AND b > ?  AND (c, d) IN ((?, ?))", 0, 0, 0, 0);
-
-            assertInvalidMessage("Clustering column \"c\" cannot be restricted (preceding column \"b\" is restricted by a non-EQ relation)",
-                                 "SELECT * FROM %s WHERE a = ? AND b > ?  AND (c, d) > (?, ?)", 0, 0, 0, 0);
-            assertInvalidMessage("PRIMARY KEY column \"c\" cannot be restricted (preceding column \"b\" is restricted by a non-EQ relation)",
-                                 "SELECT * FROM %s WHERE a = ? AND (c, d) > (?, ?) AND b > ?  ", 0, 0, 0, 0);
-
-            assertInvalidMessage("Column \"c\" cannot be restricted by two inequalities not starting with the same column",
-                                 "SELECT * FROM %s WHERE a = ? AND (b, c) > (?, ?) AND (b) < (?) AND (c) < (?)", 0, 0, 0, 0, 0);
-            assertInvalidMessage("Column \"c\" cannot be restricted by two inequalities not starting with the same column",
-                                 "SELECT * FROM %s WHERE a = ? AND (c) < (?) AND (b, c) > (?, ?) AND (b) < (?)", 0, 0, 0, 0, 0);
-            assertInvalidMessage("Clustering column \"c\" cannot be restricted (preceding column \"b\" is restricted by a non-EQ relation)",
-                                 "SELECT * FROM %s WHERE a = ? AND (b) < (?) AND (c) < (?) AND (b, c) > (?, ?)", 0, 0, 0, 0, 0);
-            assertInvalidMessage("Clustering column \"c\" cannot be restricted (preceding column \"b\" is restricted by a non-EQ relation)",
-                                 "SELECT * FROM %s WHERE a = ? AND (b) < (?) AND c < ? AND (b, c) > (?, ?)", 0, 0, 0, 0, 0);
-
-            assertInvalidMessage("Column \"c\" cannot be restricted by two inequalities not starting with the same column",
-                                 "SELECT * FROM %s WHERE a = ? AND (b, c) > (?, ?) AND (c) < (?)", 0, 0, 0, 0);
-        }
+        createTable("CREATE TABLE %s (a int, b int, c int, d int, PRIMARY KEY (a, b, c, d))");
+
+        assertInvalidSyntax("SELECT * FROM %s WHERE a = 0 AND (b, c) > ()");
+        assertInvalidMessage("Expected 2 elements in value tuple, but got 3: (?, ?, ?)",
+                             "SELECT * FROM %s WHERE a = 0 AND (b, c) > (?, ?, ?)", 1, 2, 3);
+        assertInvalidMessage("Invalid null value in condition for column c",
+                             "SELECT * FROM %s WHERE a = 0 AND (b, c) > (?, ?)", 1, null);
+
+        // Wrong order of columns
+        assertInvalidMessage("Clustering columns must appear in the PRIMARY KEY order in multi-column relations: (d, c, b) = (?, ?, ?)",
+                             "SELECT * FROM %s WHERE a = 0 AND (d, c, b) = (?, ?, ?)", 0, 0, 0);
+        assertInvalidMessage("Clustering columns must appear in the PRIMARY KEY order in multi-column relations: (d, c, b) > (?, ?, ?)",
+                             "SELECT * FROM %s WHERE a = 0 AND (d, c, b) > (?, ?, ?)", 0, 0, 0);
+
+        // Wrong number of values
+        assertInvalidMessage("Expected 3 elements in value tuple, but got 2: (?, ?)",
+                             "SELECT * FROM %s WHERE a=0 AND (b, c, d) IN ((?, ?))", 0, 1);
+        assertInvalidMessage("Expected 3 elements in value tuple, but got 5: (?, ?, ?, ?, ?)",
+                             "SELECT * FROM %s WHERE a=0 AND (b, c, d) IN ((?, ?, ?, ?, ?))", 0, 1, 2, 3, 4);
+
+        // Missing first clustering column
+        assertInvalidMessage("PRIMARY KEY column \"c\" cannot be restricted as preceding column \"b\" is not restricted",
+                             "SELECT * FROM %s WHERE a = 0 AND (c, d) = (?, ?)", 0, 0);
+        assertInvalidMessage("PRIMARY KEY column \"c\" cannot be restricted as preceding column \"b\" is not restricted",
+                             "SELECT * FROM %s WHERE a = 0 AND (c, d) > (?, ?)", 0, 0);
+
+        // Nulls
+        assertInvalidMessage("Invalid null value for column d",
+                             "SELECT * FROM %s WHERE a = 0 AND (b, c, d) = (?, ?, ?)", 1, 2, null);
+        assertInvalidMessage("Invalid null value for column d",
+                             "SELECT * FROM %s WHERE a = 0 AND (b, c, d) IN ((?, ?, ?))", 1, 2, null);
+        assertInvalidMessage("Invalid null value in condition for columns: [b, c, d]",
+                             "SELECT * FROM %s WHERE a = 0 AND (b, c, d) IN ((?, ?, ?), (?, ?, ?))", 1, 2, null, 2, 1, 4);
+
+        // Wrong type for 'd'
+        assertInvalid("SELECT * FROM %s WHERE a = 0 AND (b, c, d) = (?, ?, ?)", 1, 2, "foobar");
+        assertInvalid("SELECT * FROM %s WHERE a = 0 AND b = (?, ?, ?)", 1, 2, 3);
+
+        // Mix single and tuple inequalities
+        assertInvalidMessage("Column \"c\" cannot be restricted by two inequalities not starting with the same column",
+                             "SELECT * FROM %s WHERE a = 0 AND (b, c, d) > (?, ?, ?) AND c < ?", 0, 1, 0, 1);
+        assertInvalidMessage("Column \"c\" cannot be restricted by two inequalities not starting with the same column",
+                             "SELECT * FROM %s WHERE a = 0 AND c > ? AND (b, c, d) < (?, ?, ?)", 1, 1, 1, 0);
+
+        assertInvalidMessage("Multi-column relations can only be applied to clustering columns but was applied to: a",
+                             "SELECT * FROM %s WHERE (a, b, c, d) IN ((?, ?, ?, ?))", 0, 1, 2, 3);
+        assertInvalidMessage("PRIMARY KEY column \"c\" cannot be restricted as preceding column \"b\" is not restricted",
+                             "SELECT * FROM %s WHERE (c, d) IN ((?, ?))", 0, 1);
+
+        assertInvalidMessage("Clustering column \"c\" cannot be restricted (preceding column \"b\" is restricted by a non-EQ relation)",
+                             "SELECT * FROM %s WHERE a = ? AND b > ?  AND (c, d) IN ((?, ?))", 0, 0, 0, 0);
+
+        assertInvalidMessage("Clustering column \"c\" cannot be restricted (preceding column \"b\" is restricted by a non-EQ relation)",
+                             "SELECT * FROM %s WHERE a = ? AND b > ?  AND (c, d) > (?, ?)", 0, 0, 0, 0);
+        assertInvalidMessage("PRIMARY KEY column \"c\" cannot be restricted (preceding column \"b\" is restricted by a non-EQ relation)",
+                             "SELECT * FROM %s WHERE a = ? AND (c, d) > (?, ?) AND b > ?  ", 0, 0, 0, 0);
+
+        assertInvalidMessage("Column \"c\" cannot be restricted by two inequalities not starting with the same column",
+                             "SELECT * FROM %s WHERE a = ? AND (b, c) > (?, ?) AND (b) < (?) AND (c) < (?)", 0, 0, 0, 0, 0);
+        assertInvalidMessage("Column \"c\" cannot be restricted by two inequalities not starting with the same column",
+                             "SELECT * FROM %s WHERE a = ? AND (c) < (?) AND (b, c) > (?, ?) AND (b) < (?)", 0, 0, 0, 0, 0);
+        assertInvalidMessage("Clustering column \"c\" cannot be restricted (preceding column \"b\" is restricted by a non-EQ relation)",
+                             "SELECT * FROM %s WHERE a = ? AND (b) < (?) AND (c) < (?) AND (b, c) > (?, ?)", 0, 0, 0, 0, 0);
+        assertInvalidMessage("Clustering column \"c\" cannot be restricted (preceding column \"b\" is restricted by a non-EQ relation)",
+                             "SELECT * FROM %s WHERE a = ? AND (b) < (?) AND c < ? AND (b, c) > (?, ?)", 0, 0, 0, 0, 0);
+
+        assertInvalidMessage("Column \"c\" cannot be restricted by two inequalities not starting with the same column",
+                             "SELECT * FROM %s WHERE a = ? AND (b, c) > (?, ?) AND (c) < (?)", 0, 0, 0, 0);
     }
 
     @Test
     public void testMultiAndSingleColumnRelationMix() throws Throwable
     {
-        for (String compactOption : new String[]{"", " WITH COMPACT STORAGE"})
-        {
-            createTable("CREATE TABLE %s (a int, b int, c int, d int, PRIMARY KEY (a, b, c, d))" + compactOption);
+        createTable("CREATE TABLE %s (a int, b int, c int, d int, PRIMARY KEY (a, b, c, d))");
 
-            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 0, 0);
-            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 1, 0);
-            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 1, 1);
+        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 0, 0);
+        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 1, 0);
+        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 1, 1);
 
-            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 0, 0);
-            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 1, 0);
-            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 1, 1);
+        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 0, 0);
+        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 1, 0);
+        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 1, 1);
 
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and b = ? and (c, d) = (?, ?)", 0, 1, 0, 0),
-                       row(0, 1, 0, 0));
+        assertRows(execute("SELECT * FROM %s WHERE a = ? and b = ? and (c, d) = (?, ?)", 0, 1, 0, 0),
+                   row(0, 1, 0, 0));
 
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and b IN (?, ?) and (c, d) = (?, ?)", 0, 0, 1, 0, 0),
-                       row(0, 0, 0, 0),
-                       row(0, 1, 0, 0));
+        assertRows(execute("SELECT * FROM %s WHERE a = ? and b IN (?, ?) and (c, d) = (?, ?)", 0, 0, 1, 0, 0),
+                   row(0, 0, 0, 0),
+                   row(0, 1, 0, 0));
 
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and b = ? and (c) IN ((?))", 0, 1, 0),
-                       row(0, 1, 0, 0));
+        assertRows(execute("SELECT * FROM %s WHERE a = ? and b = ? and (c) IN ((?))", 0, 1, 0),
+                   row(0, 1, 0, 0));
 
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and b IN (?, ?) and (c) IN ((?))", 0, 0, 1, 0),
-                       row(0, 0, 0, 0),
-                       row(0, 1, 0, 0));
+        assertRows(execute("SELECT * FROM %s WHERE a = ? and b IN (?, ?) and (c) IN ((?))", 0, 0, 1, 0),
+                   row(0, 0, 0, 0),
+                   row(0, 1, 0, 0));
 
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and b = ? and (c) IN ((?), (?))", 0, 1, 0, 1),
-                       row(0, 1, 0, 0),
-                       row(0, 1, 1, 0),
-                       row(0, 1, 1, 1));
+        assertRows(execute("SELECT * FROM %s WHERE a = ? and b = ? and (c) IN ((?), (?))", 0, 1, 0, 1),
+                   row(0, 1, 0, 0),
+                   row(0, 1, 1, 0),
+                   row(0, 1, 1, 1));
 
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and b = ? and (c, d) IN ((?, ?))", 0, 1, 0, 0),
-                       row(0, 1, 0, 0));
+        assertRows(execute("SELECT * FROM %s WHERE a = ? and b = ? and (c, d) IN ((?, ?))", 0, 1, 0, 0),
+                   row(0, 1, 0, 0));
 
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and b = ? and (c, d) IN ((?, ?), (?, ?))", 0, 1, 0, 0, 1, 1),
-                       row(0, 1, 0, 0),
-                       row(0, 1, 1, 1));
+        assertRows(execute("SELECT * FROM %s WHERE a = ? and b = ? and (c, d) IN ((?, ?), (?, ?))", 0, 1, 0, 0, 1, 1),
+                   row(0, 1, 0, 0),
+                   row(0, 1, 1, 1));
 
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and b IN (?, ?) and (c, d) IN ((?, ?), (?, ?))", 0, 0, 1, 0, 0, 1, 1),
-                       row(0, 0, 0, 0),
-                       row(0, 0, 1, 1),
-                       row(0, 1, 0, 0),
-                       row(0, 1, 1, 1));
+        assertRows(execute("SELECT * FROM %s WHERE a = ? and b IN (?, ?) and (c, d) IN ((?, ?), (?, ?))", 0, 0, 1, 0, 0, 1, 1),
+                   row(0, 0, 0, 0),
+                   row(0, 0, 1, 1),
+                   row(0, 1, 0, 0),
+                   row(0, 1, 1, 1));
 
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and b = ? and (c, d) > (?, ?)", 0, 1, 0, 0),
-                       row(0, 1, 1, 0),
-                       row(0, 1, 1, 1));
+        assertRows(execute("SELECT * FROM %s WHERE a = ? and b = ? and (c, d) > (?, ?)", 0, 1, 0, 0),
+                   row(0, 1, 1, 0),
+                   row(0, 1, 1, 1));
 
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and b IN (?, ?) and (c, d) > (?, ?)", 0, 0, 1, 0, 0),
-                       row(0, 0, 1, 0),
-                       row(0, 0, 1, 1),
-                       row(0, 1, 1, 0),
-                       row(0, 1, 1, 1));
+        assertRows(execute("SELECT * FROM %s WHERE a = ? and b IN (?, ?) and (c, d) > (?, ?)", 0, 0, 1, 0, 0),
+                   row(0, 0, 1, 0),
+                   row(0, 0, 1, 1),
+                   row(0, 1, 1, 0),
+                   row(0, 1, 1, 1));
 
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and b = ? and (c, d) > (?, ?) and (c) <= (?) ", 0, 1, 0, 0, 1),
-                       row(0, 1, 1, 0),
-                       row(0, 1, 1, 1));
+        assertRows(execute("SELECT * FROM %s WHERE a = ? and b = ? and (c, d) > (?, ?) and (c) <= (?) ", 0, 1, 0, 0, 1),
+                   row(0, 1, 1, 0),
+                   row(0, 1, 1, 1));
 
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and b = ? and (c, d) > (?, ?) and c <= ? ", 0, 1, 0, 0, 1),
-                       row(0, 1, 1, 0),
-                       row(0, 1, 1, 1));
+        assertRows(execute("SELECT * FROM %s WHERE a = ? and b = ? and (c, d) > (?, ?) and c <= ? ", 0, 1, 0, 0, 1),
+                   row(0, 1, 1, 0),
+                   row(0, 1, 1, 1));
 
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and b = ? and (c, d) >= (?, ?) and (c, d) < (?, ?)", 0, 1, 0, 0, 1, 1),
-                       row(0, 1, 0, 0),
-                       row(0, 1, 1, 0));
+        assertRows(execute("SELECT * FROM %s WHERE a = ? and b = ? and (c, d) >= (?, ?) and (c, d) < (?, ?)", 0, 1, 0, 0, 1, 1),
+                   row(0, 1, 0, 0),
+                   row(0, 1, 1, 0));
 
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b, c) = (?, ?) and d = ?", 0, 0, 1, 0),
-                       row(0, 0, 1, 0));
+        assertRows(execute("SELECT * FROM %s WHERE a = ? and (b, c) = (?, ?) and d = ?", 0, 0, 1, 0),
+                   row(0, 0, 1, 0));
 
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b, c) IN ((?, ?), (?, ?)) and d = ?", 0, 0, 1, 0, 0, 0),
-                       row(0, 0, 0, 0),
-                       row(0, 0, 1, 0));
+        assertRows(execute("SELECT * FROM %s WHERE a = ? and (b, c) IN ((?, ?), (?, ?)) and d = ?", 0, 0, 1, 0, 0, 0),
+                   row(0, 0, 0, 0),
+                   row(0, 0, 1, 0));
 
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and b = ? and (c) = (?) and d = ?", 0, 0, 1, 0),
-                       row(0, 0, 1, 0));
+        assertRows(execute("SELECT * FROM %s WHERE a = ? and b = ? and (c) = (?) and d = ?", 0, 0, 1, 0),
+                   row(0, 0, 1, 0));
 
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b, c) = (?, ?) and d IN (?, ?)", 0, 0, 1, 0, 2),
-                       row(0, 0, 1, 0));
+        assertRows(execute("SELECT * FROM %s WHERE a = ? and (b, c) = (?, ?) and d IN (?, ?)", 0, 0, 1, 0, 2),
+                   row(0, 0, 1, 0));
 
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and b = ? and (c) = (?) and d IN (?, ?)", 0, 0, 1, 0, 2),
-                       row(0, 0, 1, 0));
+        assertRows(execute("SELECT * FROM %s WHERE a = ? and b = ? and (c) = (?) and d IN (?, ?)", 0, 0, 1, 0, 2),
+                   row(0, 0, 1, 0));
 
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b, c) = (?, ?) and d >= ?", 0, 0, 1, 0),
-                       row(0, 0, 1, 0),
-                       row(0, 0, 1, 1));
+        assertRows(execute("SELECT * FROM %s WHERE a = ? and (b, c) = (?, ?) and d >= ?", 0, 0, 1, 0),
+                   row(0, 0, 1, 0),
+                   row(0, 0, 1, 1));
 
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and d < 1 and (b, c) = (?, ?) and d >= ?", 0, 0, 1, 0),
-                       row(0, 0, 1, 0));
+        assertRows(execute("SELECT * FROM %s WHERE a = ? and d < 1 and (b, c) = (?, ?) and d >= ?", 0, 0, 1, 0),
+                   row(0, 0, 1, 0));
 
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and d < 1 and (b, c) IN ((?, ?), (?, ?)) and d >= ?", 0, 0, 1, 0, 0, 0),
-                       row(0, 0, 0, 0),
-                       row(0, 0, 1, 0));
-        }
+        assertRows(execute("SELECT * FROM %s WHERE a = ? and d < 1 and (b, c) IN ((?, ?), (?, ?)) and d >= ?", 0, 0, 1, 0, 0, 0),
+                   row(0, 0, 0, 0),
+                   row(0, 0, 1, 0));
     }
 
     @Test
     public void testSeveralMultiColumnRelation() throws Throwable
     {
-        for (String compactOption : new String[]{"", " WITH COMPACT STORAGE"})
-        {
-            createTable("CREATE TABLE %s (a int, b int, c int, d int, PRIMARY KEY (a, b, c, d))" + compactOption);
+        createTable("CREATE TABLE %s (a int, b int, c int, d int, PRIMARY KEY (a, b, c, d))");
 
-            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 0, 0);
-            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 1, 0);
-            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 1, 1);
+        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 0, 0);
+        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 1, 0);
+        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 1, 1);
 
-            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 0, 0);
-            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 1, 0);
-            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 1, 1);
+        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 0, 0);
+        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 1, 0);
+        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 1, 1);
 
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b) = (?) and (c, d) = (?, ?)", 0, 1, 0, 0),
-                       row(0, 1, 0, 0));
+        assertRows(execute("SELECT * FROM %s WHERE a = ? and (b) = (?) and (c, d) = (?, ?)", 0, 1, 0, 0),
+                   row(0, 1, 0, 0));
 
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b) IN ((?), (?)) and (c, d) = (?, ?)", 0, 0, 1, 0, 0),
-                       row(0, 0, 0, 0),
-                       row(0, 1, 0, 0));
+        assertRows(execute("SELECT * FROM %s WHERE a = ? and (b) IN ((?), (?)) and (c, d) = (?, ?)", 0, 0, 1, 0, 0),
+                   row(0, 0, 0, 0),
+                   row(0, 1, 0, 0));
 
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b) = (?) and (c) IN ((?))", 0, 1, 0),
-                       row(0, 1, 0, 0));
+        assertRows(execute("SELECT * FROM %s WHERE a = ? and (b) = (?) and (c) IN ((?))", 0, 1, 0),
+                   row(0, 1, 0, 0));
 
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b) IN ((?),(?)) and (c) IN ((?))", 0, 0, 1, 0),
-                       row(0, 0, 0, 0),
-                       row(0, 1, 0, 0));
+        assertRows(execute("SELECT * FROM %s WHERE a = ? and (b) IN ((?),(?)) and (c) IN ((?))", 0, 0, 1, 0),
+                   row(0, 0, 0, 0),
+                   row(0, 1, 0, 0));
 
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b) = (?) and (c) IN ((?), (?))", 0, 1, 0, 1),
-                       row(0, 1, 0, 0),
-                       row(0, 1, 1, 0),
-                       row(0, 1, 1, 1));
+        assertRows(execute("SELECT * FROM %s WHERE a = ? and (b) = (?) and (c) IN ((?), (?))", 0, 1, 0, 1),
+                   row(0, 1, 0, 0),
+                   row(0, 1, 1, 0),
+                   row(0, 1, 1, 1));
 
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b) = (?) and (c, d) IN ((?, ?))", 0, 1, 0, 0),
-                       row(0, 1, 0, 0));
+        assertRows(execute("SELECT * FROM %s WHERE a = ? and (b) = (?) and (c, d) IN ((?, ?))", 0, 1, 0, 0),
+                   row(0, 1, 0, 0));
 
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b) = (?) and (c, d) IN ((?, ?), (?, ?))", 0, 1, 0, 0, 1, 1),
-                       row(0, 1, 0, 0),
-                       row(0, 1, 1, 1));
+        assertRows(execute("SELECT * FROM %s WHERE a = ? and (b) = (?) and (c, d) IN ((?, ?), (?, ?))", 0, 1, 0, 0, 1, 1),
+                   row(0, 1, 0, 0),
+                   row(0, 1, 1, 1));
 
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b) IN ((?), (?)) and (c, d) IN ((?, ?), (?, ?))", 0, 0, 1, 0, 0, 1, 1),
-                       row(0, 0, 0, 0),
-                       row(0, 0, 1, 1),
-                       row(0, 1, 0, 0),
-                       row(0, 1, 1, 1));
+        assertRows(execute("SELECT * FROM %s WHERE a = ? and (b) IN ((?), (?)) and (c, d) IN ((?, ?), (?, ?))", 0, 0, 1, 0, 0, 1, 1),
+                   row(0, 0, 0, 0),
+                   row(0, 0, 1, 1),
+                   row(0, 1, 0, 0),
+                   row(0, 1, 1, 1));
 
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b) = (?) and (c, d) > (?, ?)", 0, 1, 0, 0),
-                       row(0, 1, 1, 0),
-                       row(0, 1, 1, 1));
+        assertRows(execute("SELECT * FROM %s WHERE a = ? and (b) = (?) and (c, d) > (?, ?)", 0, 1, 0, 0),
+                   row(0, 1, 1, 0),
+                   row(0, 1, 1, 1));
 
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b) IN ((?),(?)) and (c, d) > (?, ?)", 0, 0, 1, 0, 0),
-                       row(0, 0, 1, 0),
-                       row(0, 0, 1, 1),
-                       row(0, 1, 1, 0),
-                       row(0, 1, 1, 1));
+        assertRows(execute("SELECT * FROM %s WHERE a = ? and (b) IN ((?),(?)) and (c, d) > (?, ?)", 0, 0, 1, 0, 0),
+                   row(0, 0, 1, 0),
+                   row(0, 0, 1, 1),
+                   row(0, 1, 1, 0),
+                   row(0, 1, 1, 1));
 
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b) = (?) and (c, d) > (?, ?) and (c) <= (?) ", 0, 1, 0, 0, 1),
-                       row(0, 1, 1, 0),
-                       row(0, 1, 1, 1));
+        assertRows(execute("SELECT * FROM %s WHERE a = ? and (b) = (?) and (c, d) > (?, ?) and (c) <= (?) ", 0, 1, 0, 0, 1),
+                   row(0, 1, 1, 0),
+                   row(0, 1, 1, 1));
 
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b) = (?) and (c, d) > (?, ?) and c <= ? ", 0, 1, 0, 0, 1),
-                       row(0, 1, 1, 0),
-                       row(0, 1, 1, 1));
+        assertRows(execute("SELECT * FROM %s WHERE a = ? and (b) = (?) and (c, d) > (?, ?) and c <= ? ", 0, 1, 0, 0, 1),
+                   row(0, 1, 1, 0),
+                   row(0, 1, 1, 1));
 
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b) = (?) and (c, d) >= (?, ?) and (c, d) < (?, ?)", 0, 1, 0, 0, 1, 1),
-                       row(0, 1, 0, 0),
-                       row(0, 1, 1, 0));
+        assertRows(execute("SELECT * FROM %s WHERE a = ? and (b) = (?) and (c, d) >= (?, ?) and (c, d) < (?, ?)", 0, 1, 0, 0, 1, 1),
+                   row(0, 1, 0, 0),
+                   row(0, 1, 1, 0));
 
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b, c) = (?, ?) and d = ?", 0, 0, 1, 0),
-                       row(0, 0, 1, 0));
+        assertRows(execute("SELECT * FROM %s WHERE a = ? and (b, c) = (?, ?) and d = ?", 0, 0, 1, 0),
+                   row(0, 0, 1, 0));
 
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b, c) IN ((?, ?), (?, ?)) and d = ?", 0, 0, 1, 0, 0, 0),
-                       row(0, 0, 0, 0),
-                       row(0, 0, 1, 0));
+        assertRows(execute("SELECT * FROM %s WHERE a = ? and (b, c) IN ((?, ?), (?, ?)) and d = ?", 0, 0, 1, 0, 0, 0),
+                   row(0, 0, 0, 0),
+                   row(0, 0, 1, 0));
 
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and (d) < (1) and (b, c) = (?, ?) and (d) >= (?)", 0, 0, 1, 0),
-                       row(0, 0, 1, 0));
+        assertRows(execute("SELECT * FROM %s WHERE a = ? and (d) < (1) and (b, c) = (?, ?) and (d) >= (?)", 0, 0, 1, 0),
+                   row(0, 0, 1, 0));
 
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and (d) < (1) and (b, c) IN ((?, ?), (?, ?)) and (d) >= (?)", 0, 0, 1, 0, 0, 0),
-                       row(0, 0, 0, 0),
-                       row(0, 0, 1, 0));
-        }
+        assertRows(execute("SELECT * FROM %s WHERE a = ? and (d) < (1) and (b, c) IN ((?, ?), (?, ?)) and (d) >= (?)", 0, 0, 1, 0, 0, 0),
+                   row(0, 0, 0, 0),
+                   row(0, 0, 1, 0));
     }
 
     @Test
     public void testSinglePartitionInvalidQueries() throws Throwable
     {
-        for (String compactOption : new String[]{"", " WITH COMPACT STORAGE"})
-        {
-            createTable("CREATE TABLE %s (a int PRIMARY KEY, b int)" + compactOption);
-            assertInvalidMessage("Multi-column relations can only be applied to clustering columns but was applied to: a",
-                                 "SELECT * FROM %s WHERE (a) > (?)", 0);
-            assertInvalidMessage("Multi-column relations can only be applied to clustering columns but was applied to: a",
-                                 "SELECT * FROM %s WHERE (a) = (?)", 0);
-            assertInvalidMessage("Multi-column relations can only be applied to clustering columns but was applied to: b",
-                                 "SELECT * FROM %s WHERE (b) = (?)", 0);
-        }
+        createTable("CREATE TABLE %s (a int PRIMARY KEY, b int)");
+        assertInvalidMessage("Multi-column relations can only be applied to clustering columns but was applied to: a",
+                             "SELECT * FROM %s WHERE (a) > (?)", 0);
+        assertInvalidMessage("Multi-column relations can only be applied to clustering columns but was applied to: a",
+                             "SELECT * FROM %s WHERE (a) = (?)", 0);
+        assertInvalidMessage("Multi-column relations can only be applied to clustering columns but was applied to: b",
+                             "SELECT * FROM %s WHERE (b) = (?)", 0);
     }
 
     @Test
     public void testSingleClustering() throws Throwable
     {
-        for (String compactOption : new String[]{"", " WITH COMPACT STORAGE"})
-        {
-            createTable("CREATE TABLE %s (a int, b int, c int, PRIMARY KEY (a, b))" + compactOption);
+        createTable("CREATE TABLE %s (a int, b int, c int, PRIMARY KEY (a, b))");
 
-            execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, 0, 0);
-            execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, 1, 0);
-            execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, 2, 0);
+        execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, 0, 0);
+        execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, 1, 0);
+        execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, 2, 0);
 
-            // Equalities
+        // Equalities
 
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) = (?)", 0, 1),
-                    row(0, 1, 0)
-            );
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) = (?)", 0, 1),
+                   row(0, 1, 0)
+        );
 
-            // Same but check the whole tuple can be prepared
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) = ?", 0, tuple(1)),
-                    row(0, 1, 0)
-            );
+        // Same but check the whole tuple can be prepared
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) = ?", 0, tuple(1)),
+                   row(0, 1, 0)
+        );
 
-            assertEmpty(execute("SELECT * FROM %s WHERE a = ? AND (b) = (?)", 0, 3));
+        assertEmpty(execute("SELECT * FROM %s WHERE a = ? AND (b) = (?)", 0, 3));
 
-            // Inequalities
+        // Inequalities
 
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) > (?)", 0, 0),
-                    row(0, 1, 0),
-                    row(0, 2, 0)
-            );
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) > (?)", 0, 0),
+                   row(0, 1, 0),
+                   row(0, 2, 0)
+        );
 
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) >= (?)", 0, 1),
-                    row(0, 1, 0),
-                    row(0, 2, 0)
-            );
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) >= (?)", 0, 1),
+                   row(0, 1, 0),
+                   row(0, 2, 0)
+        );
 
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) < (?)", 0, 2),
-                    row(0, 0, 0),
-                    row(0, 1, 0)
-            );
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) < (?)", 0, 2),
+                   row(0, 0, 0),
+                   row(0, 1, 0)
+        );
 
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) <= (?)", 0, 1),
-                    row(0, 0, 0),
-                    row(0, 1, 0)
-            );
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) <= (?)", 0, 1),
+                   row(0, 0, 0),
+                   row(0, 1, 0)
+        );
 
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) > (?) AND (b) < (?)", 0, 0, 2),
-                    row(0, 1, 0)
-            );
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) > (?) AND (b) < (?)", 0, 0, 2),
+                   row(0, 1, 0)
+        );
 
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) > (?) AND b < ?", 0, 0, 2),
-                       row(0, 1, 0)
-            );
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) > (?) AND b < ?", 0, 0, 2),
+                   row(0, 1, 0)
+        );
 
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND b > ? AND (b) < (?)", 0, 0, 2),
-                       row(0, 1, 0)
-            );
-        }
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND b > ? AND (b) < (?)", 0, 0, 2),
+                   row(0, 1, 0)
+        );
     }
 
     @Test
     public void testNonEqualsRelation() throws Throwable
     {
-        for (String compactOption : new String[]{"", " WITH COMPACT STORAGE"})
-        {
-            createTable("CREATE TABLE %s (a int PRIMARY KEY, b int)" + compactOption);
-            assertInvalidMessage("Unsupported \"!=\" relation: (b) != (0)",
-                    "SELECT * FROM %s WHERE a = 0 AND (b) != (0)");
-        }
+        createTable("CREATE TABLE %s (a int PRIMARY KEY, b int)");
+        assertInvalidMessage("Unsupported \"!=\" relation: (b) != (0)",
+                             "SELECT * FROM %s WHERE a = 0 AND (b) != (0)");
     }
 
     @Test
     public void testMultipleClustering() throws Throwable
     {
-        for (String compactOption : new String[]{"", " WITH COMPACT STORAGE"})
-        {
-            createTable("CREATE TABLE %s (a int, b int, c int, d int, PRIMARY KEY (a, b, c, d))" + compactOption);
-
-            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 0, 0);
-            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 1, 0);
-            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 1, 1);
-
-            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 0, 0);
-            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 1, 0);
-            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 1, 1);
-
-            // Empty query
-            assertEmpty(execute("SELECT * FROM %s WHERE a = 0 AND (b, c, d) IN ()"));
-
-            // Equalities
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) = (?)", 0, 1),
-                    row(0, 1, 0, 0),
-                    row(0, 1, 1, 0),
-                    row(0, 1, 1, 1)
-            );
-
-            // Same with whole tuple prepared
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) = ?", 0, tuple(1)),
-                    row(0, 1, 0, 0),
-                    row(0, 1, 1, 0),
-                    row(0, 1, 1, 1)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c) = (?, ?)", 0, 1, 1),
-                    row(0, 1, 1, 0),
-                    row(0, 1, 1, 1)
-            );
-
-            // Same with whole tuple prepared
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c) = ?", 0, tuple(1, 1)),
-                    row(0, 1, 1, 0),
-                    row(0, 1, 1, 1)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) = (?, ?, ?)", 0, 1, 1, 1),
-                    row(0, 1, 1, 1)
-            );
-
-            // Same with whole tuple prepared
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) = ?", 0, tuple(1, 1, 1)),
-                    row(0, 1, 1, 1)
-            );
-
-            // Inequalities
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) > (?)", 0, 0),
-                    row(0, 1, 0, 0),
-                    row(0, 1, 1, 0),
-                    row(0, 1, 1, 1)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) >= (?)", 0, 0),
-                    row(0, 0, 0, 0),
-                    row(0, 0, 1, 0),
-                    row(0, 0, 1, 1),
-                    row(0, 1, 0, 0),
-                    row(0, 1, 1, 0),
-                    row(0, 1, 1, 1)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c) > (?, ?)", 0, 1, 0),
-                    row(0, 1, 1, 0),
-                    row(0, 1, 1, 1)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c) >= (?, ?)", 0, 1, 0),
-                    row(0, 1, 0, 0),
-                    row(0, 1, 1, 0),
-                    row(0, 1, 1, 1)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) > (?, ?, ?)", 0, 1, 1, 0),
-                    row(0, 1, 1, 1)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) >= (?, ?, ?)", 0, 1, 1, 0),
-                    row(0, 1, 1, 0),
-                    row(0, 1, 1, 1)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) < (?)", 0, 1),
-                    row(0, 0, 0, 0),
-                    row(0, 0, 1, 0),
-                    row(0, 0, 1, 1)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) <= (?)", 0, 1),
-                    row(0, 0, 0, 0),
-                    row(0, 0, 1, 0),
-                    row(0, 0, 1, 1),
-                    row(0, 1, 0, 0),
-                    row(0, 1, 1, 0),
-                    row(0, 1, 1, 1)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c) < (?, ?)", 0, 0, 1),
-                    row(0, 0, 0, 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c) <= (?, ?)", 0, 0, 1),
-                    row(0, 0, 0, 0),
-                    row(0, 0, 1, 0),
-                    row(0, 0, 1, 1)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) < (?, ?, ?)", 0, 0, 1, 1),
-                    row(0, 0, 0, 0),
-                    row(0, 0, 1, 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) <= (?, ?, ?)", 0, 0, 1, 1),
-                    row(0, 0, 0, 0),
-                    row(0, 0, 1, 0),
-                    row(0, 0, 1, 1)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) > (?, ?, ?) AND (b) < (?)", 0, 0, 1, 0, 1),
-                    row(0, 0, 1, 1)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) > (?, ?, ?) AND b < ?", 0, 0, 1, 0, 1),
-                       row(0, 0, 1, 1)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) > (?, ?, ?) AND (b, c) < (?, ?)", 0, 0, 1, 1, 1, 1),
-                    row(0, 1, 0, 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) > (?, ?, ?) AND (b, c, d) < (?, ?, ?)", 0, 0, 1, 1, 1, 1, 0),
-                    row(0, 1, 0, 0)
-            );
-
-            // Same with whole tuple prepared
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) > ? AND (b, c, d) < ?", 0, tuple(0, 1, 1), tuple(1, 1, 0)),
-                    row(0, 1, 0, 0)
-            );
-
-            // reversed
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) > (?) ORDER BY b DESC, c DESC, d DESC", 0, 0),
-                    row(0, 1, 1, 1),
-                    row(0, 1, 1, 0),
-                    row(0, 1, 0, 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) >= (?) ORDER BY b DESC, c DESC, d DESC", 0, 0),
-                    row(0, 1, 1, 1),
-                    row(0, 1, 1, 0),
-                    row(0, 1, 0, 0),
-                    row(0, 0, 1, 1),
-                    row(0, 0, 1, 0),
-                    row(0, 0, 0, 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c) > (?, ?) ORDER BY b DESC, c DESC, d DESC", 0, 1, 0),
-                    row(0, 1, 1, 1),
-                    row(0, 1, 1, 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c) >= (?, ?) ORDER BY b DESC, c DESC, d DESC", 0, 1, 0),
-                    row(0, 1, 1, 1),
-                    row(0, 1, 1, 0),
-                    row(0, 1, 0, 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) > (?, ?, ?) ORDER BY b DESC, c DESC, d DESC", 0, 1, 1, 0),
-                    row(0, 1, 1, 1)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) >= (?, ?, ?) ORDER BY b DESC, c DESC, d DESC", 0, 1, 1, 0),
-                    row(0, 1, 1, 1),
-                    row(0, 1, 1, 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) < (?) ORDER BY b DESC, c DESC, d DESC", 0, 1),
-                    row(0, 0, 1, 1),
-                    row(0, 0, 1, 0),
-                    row(0, 0, 0, 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) <= (?) ORDER BY b DESC, c DESC, d DESC", 0, 1),
-                    row(0, 1, 1, 1),
-                    row(0, 1, 1, 0),
-                    row(0, 1, 0, 0),
-                    row(0, 0, 1, 1),
-                    row(0, 0, 1, 0),
-                    row(0, 0, 0, 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c) < (?, ?) ORDER BY b DESC, c DESC, d DESC", 0, 0, 1),
-                    row(0, 0, 0, 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c) <= (?, ?) ORDER BY b DESC, c DESC, d DESC", 0, 0, 1),
-                    row(0, 0, 1, 1),
-                    row(0, 0, 1, 0),
-                    row(0, 0, 0, 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) < (?, ?, ?) ORDER BY b DESC, c DESC, d DESC", 0, 0, 1, 1),
-                    row(0, 0, 1, 0),
-                    row(0, 0, 0, 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) <= (?, ?, ?) ORDER BY b DESC, c DESC, d DESC", 0, 0, 1, 1),
-                    row(0, 0, 1, 1),
-                    row(0, 0, 1, 0),
-                    row(0, 0, 0, 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) > (?, ?, ?) AND (b) < (?) ORDER BY b DESC, c DESC, d DESC", 0, 0, 1, 0, 1),
-                    row(0, 0, 1, 1)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) > (?, ?, ?) AND b < ? ORDER BY b DESC, c DESC, d DESC", 0, 0, 1, 0, 1),
-                       row(0, 0, 1, 1)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) > (?, ?, ?) AND (b, c) < (?, ?) ORDER BY b DESC, c DESC, d DESC", 0, 0, 1, 1, 1, 1),
-                    row(0, 1, 0, 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) > (?, ?, ?) AND (b, c, d) < (?, ?, ?) ORDER BY b DESC, c DESC, d DESC", 0, 0, 1, 1, 1, 1, 0),
-                    row(0, 1, 0, 0)
-            );
-
-            // IN
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) IN ((?, ?, ?), (?, ?, ?))", 0, 0, 1, 0, 0, 1, 1),
-                    row(0, 0, 1, 0),
-                    row(0, 0, 1, 1)
-            );
-
-            // same query but with whole tuple prepared
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) IN (?, ?)", 0, tuple(0, 1, 0), tuple(0, 1, 1)),
-                    row(0, 0, 1, 0),
-                    row(0, 0, 1, 1)
-            );
-
-            // same query but with whole IN list prepared
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) IN ?", 0, list(tuple(0, 1, 0), tuple(0, 1, 1))),
-                    row(0, 0, 1, 0),
-                    row(0, 0, 1, 1)
-            );
-
-            // same query, but reversed order for the IN values
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) IN (?, ?)", 0, tuple(0, 1, 1), tuple(0, 1, 0)),
-                    row(0, 0, 1, 0),
-                    row(0, 0, 1, 1)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b, c) IN ((?, ?))", 0, 0, 1),
-                    row(0, 0, 1, 0),
-                    row(0, 0, 1, 1)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b) IN ((?))", 0, 0),
-                    row(0, 0, 0, 0),
-                    row(0, 0, 1, 0),
-                    row(0, 0, 1, 1)
-            );
-
-            assertEmpty(execute("SELECT * FROM %s WHERE a = ? and (b) IN ()", 0));
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c) IN ((?, ?)) ORDER BY b DESC, c DESC, d DESC", 0, 0, 1),
-                    row(0, 0, 1, 1),
-                    row(0, 0, 1, 0)
-            );
-
-            assertEmpty(execute("SELECT * FROM %s WHERE a = ? AND (b, c) IN () ORDER BY b DESC, c DESC, d DESC", 0));
-
-            // IN on both partition key and clustering key
-            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 1, 0, 0, 0);
-            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 1, 0, 1, 0);
-            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 1, 0, 1, 1);
-
-            assertRows(execute("SELECT * FROM %s WHERE a IN (?, ?) AND (b, c, d) IN (?, ?)", 0, 1, tuple(0, 1, 0), tuple(0, 1, 1)),
-                    row(0, 0, 1, 0),
-                    row(0, 0, 1, 1),
-                    row(1, 0, 1, 0),
-                    row(1, 0, 1, 1)
-            );
-
-            // same but with whole IN lists prepared
-            assertRows(execute("SELECT * FROM %s WHERE a IN ? AND (b, c, d) IN ?", list(0, 1), list(tuple(0, 1, 0), tuple(0, 1, 1))),
-                    row(0, 0, 1, 0),
-                    row(0, 0, 1, 1),
-                    row(1, 0, 1, 0),
-                    row(1, 0, 1, 1)
-            );
-
-            // same query, but reversed order for the IN values
-            assertRows(execute("SELECT * FROM %s WHERE a IN (?, ?) AND (b, c, d) IN (?, ?)", 1, 0, tuple(0, 1, 1), tuple(0, 1, 0)),
-                    row(0, 0, 1, 0),
-                    row(0, 0, 1, 1),
-                    row(1, 0, 1, 0),
-                    row(1, 0, 1, 1)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a IN (?, ?) and (b, c) IN ((?, ?))", 0, 1, 0, 1),
-                    row(0, 0, 1, 0),
-                    row(0, 0, 1, 1),
-                    row(1, 0, 1, 0),
-                    row(1, 0, 1, 1)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a IN (?, ?) and (b) IN ((?))", 0, 1, 0),
-                    row(0, 0, 0, 0),
-                    row(0, 0, 1, 0),
-                    row(0, 0, 1, 1),
-                    row(1, 0, 0, 0),
-                    row(1, 0, 1, 0),
-                    row(1, 0, 1, 1)
-            );
-        }
+        createTable("CREATE TABLE %s (a int, b int, c int, d int, PRIMARY KEY (a, b, c, d))");
+
+        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 0, 0);
+        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 1, 0);
+        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 1, 1);
+
+        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 0, 0);
+        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 1, 0);
+        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 1, 1);
+
+        // Empty query
+        assertEmpty(execute("SELECT * FROM %s WHERE a = 0 AND (b, c, d) IN ()"));
+
+        // Equalities
+
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) = (?)", 0, 1),
+                   row(0, 1, 0, 0),
+                   row(0, 1, 1, 0),
+                   row(0, 1, 1, 1)
+        );
+
+        // Same with whole tuple prepared
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) = ?", 0, tuple(1)),
+                   row(0, 1, 0, 0),
+                   row(0, 1, 1, 0),
+                   row(0, 1, 1, 1)
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c) = (?, ?)", 0, 1, 1),
+                   row(0, 1, 1, 0),
+                   row(0, 1, 1, 1)
+        );
+
+        // Same with whole tuple prepared
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c) = ?", 0, tuple(1, 1)),
+                   row(0, 1, 1, 0),
+                   row(0, 1, 1, 1)
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) = (?, ?, ?)", 0, 1, 1, 1),
+                   row(0, 1, 1, 1)
+        );
+
+        // Same with whole tuple prepared
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) = ?", 0, tuple(1, 1, 1)),
+                   row(0, 1, 1, 1)
+        );
+
+        // Inequalities
+
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) > (?)", 0, 0),
+                   row(0, 1, 0, 0),
+                   row(0, 1, 1, 0),
+                   row(0, 1, 1, 1)
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) >= (?)", 0, 0),
+                   row(0, 0, 0, 0),
+                   row(0, 0, 1, 0),
+                   row(0, 0, 1, 1),
+                   row(0, 1, 0, 0),
+                   row(0, 1, 1, 0),
+                   row(0, 1, 1, 1)
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c) > (?, ?)", 0, 1, 0),
+                   row(0, 1, 1, 0),
+                   row(0, 1, 1, 1)
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c) >= (?, ?)", 0, 1, 0),
+                   row(0, 1, 0, 0),
+                   row(0, 1, 1, 0),
+                   row(0, 1, 1, 1)
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) > (?, ?, ?)", 0, 1, 1, 0),
+                   row(0, 1, 1, 1)
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) >= (?, ?, ?)", 0, 1, 1, 0),
+                   row(0, 1, 1, 0),
+                   row(0, 1, 1, 1)
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) < (?)", 0, 1),
+                   row(0, 0, 0, 0),
+                   row(0, 0, 1, 0),
+                   row(0, 0, 1, 1)
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) <= (?)", 0, 1),
+                   row(0, 0, 0, 0),
+                   row(0, 0, 1, 0),
+                   row(0, 0, 1, 1),
+                   row(0, 1, 0, 0),
+                   row(0, 1, 1, 0),
+                   row(0, 1, 1, 1)
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c) < (?, ?)", 0, 0, 1),
+                   row(0, 0, 0, 0)
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c) <= (?, ?)", 0, 0, 1),
+                   row(0, 0, 0, 0),
+                   row(0, 0, 1, 0),
+                   row(0, 0, 1, 1)
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) < (?, ?, ?)", 0, 0, 1, 1),
+                   row(0, 0, 0, 0),
+                   row(0, 0, 1, 0)
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) <= (?, ?, ?)", 0, 0, 1, 1),
+                   row(0, 0, 0, 0),
+                   row(0, 0, 1, 0),
+                   row(0, 0, 1, 1)
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) > (?, ?, ?) AND (b) < (?)", 0, 0, 1, 0, 1),
+                   row(0, 0, 1, 1)
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) > (?, ?, ?) AND b < ?", 0, 0, 1, 0, 1),
+                   row(0, 0, 1, 1)
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) > (?, ?, ?) AND (b, c) < (?, ?)", 0, 0, 1, 1, 1, 1),
+                   row(0, 1, 0, 0)
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) > (?, ?, ?) AND (b, c, d) < (?, ?, ?)", 0, 0, 1, 1, 1, 1, 0),
+                   row(0, 1, 0, 0)
+        );
+
+        // Same with whole tuple prepared
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) > ? AND (b, c, d) < ?", 0, tuple(0, 1, 1), tuple(1, 1, 0)),
+                   row(0, 1, 0, 0)
+        );
+
+        // reversed
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) > (?) ORDER BY b DESC, c DESC, d DESC", 0, 0),
+                   row(0, 1, 1, 1),
+                   row(0, 1, 1, 0),
+                   row(0, 1, 0, 0)
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) >= (?) ORDER BY b DESC, c DESC, d DESC", 0, 0),
+                   row(0, 1, 1, 1),
+                   row(0, 1, 1, 0),
+                   row(0, 1, 0, 0),
+                   row(0, 0, 1, 1),
+                   row(0, 0, 1, 0),
+                   row(0, 0, 0, 0)
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c) > (?, ?) ORDER BY b DESC, c DESC, d DESC", 0, 1, 0),
+                   row(0, 1, 1, 1),
+                   row(0, 1, 1, 0)
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c) >= (?, ?) ORDER BY b DESC, c DESC, d DESC", 0, 1, 0),
+                   row(0, 1, 1, 1),
+                   row(0, 1, 1, 0),
+                   row(0, 1, 0, 0)
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) > (?, ?, ?) ORDER BY b DESC, c DESC, d DESC", 0, 1, 1, 0),
+                   row(0, 1, 1, 1)
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) >= (?, ?, ?) ORDER BY b DESC, c DESC, d DESC", 0, 1, 1, 0),
+                   row(0, 1, 1, 1),
+                   row(0, 1, 1, 0)
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) < (?) ORDER BY b DESC, c DESC, d DESC", 0, 1),
+                   row(0, 0, 1, 1),
+                   row(0, 0, 1, 0),
+                   row(0, 0, 0, 0)
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) <= (?) ORDER BY b DESC, c DESC, d DESC", 0, 1),
+                   row(0, 1, 1, 1),
+                   row(0, 1, 1, 0),
+                   row(0, 1, 0, 0),
+                   row(0, 0, 1, 1),
+                   row(0, 0, 1, 0),
+                   row(0, 0, 0, 0)
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c) < (?, ?) ORDER BY b DESC, c DESC, d DESC", 0, 0, 1),
+                   row(0, 0, 0, 0)
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c) <= (?, ?) ORDER BY b DESC, c DESC, d DESC", 0, 0, 1),
+                   row(0, 0, 1, 1),
+                   row(0, 0, 1, 0),
+                   row(0, 0, 0, 0)
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) < (?, ?, ?) ORDER BY b DESC, c DESC, d DESC", 0, 0, 1, 1),
+                   row(0, 0, 1, 0),
+                   row(0, 0, 0, 0)
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) <= (?, ?, ?) ORDER BY b DESC, c DESC, d DESC", 0, 0, 1, 1),
+                   row(0, 0, 1, 1),
+                   row(0, 0, 1, 0),
+                   row(0, 0, 0, 0)
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) > (?, ?, ?) AND (b) < (?) ORDER BY b DESC, c DESC, d DESC", 0, 0, 1, 0, 1),
+                   row(0, 0, 1, 1)
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) > (?, ?, ?) AND b < ? ORDER BY b DESC, c DESC, d DESC", 0, 0, 1, 0, 1),
+                   row(0, 0, 1, 1)
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) > (?, ?, ?) AND (b, c) < (?, ?) ORDER BY b DESC, c DESC, d DESC", 0, 0, 1, 1, 1, 1),
+                   row(0, 1, 0, 0)
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) > (?, ?, ?) AND (b, c, d) < (?, ?, ?) ORDER BY b DESC, c DESC, d DESC", 0, 0, 1, 1, 1, 1, 0),
+                   row(0, 1, 0, 0)
+        );
+
+        // IN
+
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) IN ((?, ?, ?), (?, ?, ?))", 0, 0, 1, 0, 0, 1, 1),
+                   row(0, 0, 1, 0),
+                   row(0, 0, 1, 1)
+        );
+
+        // same query but with whole tuple prepared
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) IN (?, ?)", 0, tuple(0, 1, 0), tuple(0, 1, 1)),
+                   row(0, 0, 1, 0),
+                   row(0, 0, 1, 1)
+        );
+
+        // same query but with whole IN list prepared
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) IN ?", 0, list(tuple(0, 1, 0), tuple(0, 1, 1))),
+                   row(0, 0, 1, 0),
+                   row(0, 0, 1, 1)
+        );
+
+        // same query, but reversed order for the IN values
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) IN (?, ?)", 0, tuple(0, 1, 1), tuple(0, 1, 0)),
+                   row(0, 0, 1, 0),
+                   row(0, 0, 1, 1)
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE a = ? and (b, c) IN ((?, ?))", 0, 0, 1),
+                   row(0, 0, 1, 0),
+                   row(0, 0, 1, 1)
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE a = ? and (b) IN ((?))", 0, 0),
+                   row(0, 0, 0, 0),
+                   row(0, 0, 1, 0),
+                   row(0, 0, 1, 1)
+        );
+
+        assertEmpty(execute("SELECT * FROM %s WHERE a = ? and (b) IN ()", 0));
+
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c) IN ((?, ?)) ORDER BY b DESC, c DESC, d DESC", 0, 0, 1),
+                   row(0, 0, 1, 1),
+                   row(0, 0, 1, 0)
+        );
+
+        assertEmpty(execute("SELECT * FROM %s WHERE a = ? AND (b, c) IN () ORDER BY b DESC, c DESC, d DESC", 0));
+
+        // IN on both partition key and clustering key
+        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 1, 0, 0, 0);
+        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 1, 0, 1, 0);
+        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 1, 0, 1, 1);
+
+        assertRows(execute("SELECT * FROM %s WHERE a IN (?, ?) AND (b, c, d) IN (?, ?)", 0, 1, tuple(0, 1, 0), tuple(0, 1, 1)),
+                   row(0, 0, 1, 0),
+                   row(0, 0, 1, 1),
+                   row(1, 0, 1, 0),
+                   row(1, 0, 1, 1)
+        );
+
+        // same but with whole IN lists prepared
+        assertRows(execute("SELECT * FROM %s WHERE a IN ? AND (b, c, d) IN ?", list(0, 1), list(tuple(0, 1, 0), tuple(0, 1, 1))),
+                   row(0, 0, 1, 0),
+                   row(0, 0, 1, 1),
+                   row(1, 0, 1, 0),
+                   row(1, 0, 1, 1)
+        );
+
+        // same query, but reversed order for the IN values
+        assertRows(execute("SELECT * FROM %s WHERE a IN (?, ?) AND (b, c, d) IN (?, ?)", 1, 0, tuple(0, 1, 1), tuple(0, 1, 0)),
+                   row(0, 0, 1, 0),
+                   row(0, 0, 1, 1),
+                   row(1, 0, 1, 0),
+                   row(1, 0, 1, 1)
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE a IN (?, ?) and (b, c) IN ((?, ?))", 0, 1, 0, 1),
+                   row(0, 0, 1, 0),
+                   row(0, 0, 1, 1),
+                   row(1, 0, 1, 0),
+                   row(1, 0, 1, 1)
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE a IN (?, ?) and (b) IN ((?))", 0, 1, 0),
+                   row(0, 0, 0, 0),
+                   row(0, 0, 1, 0),
+                   row(0, 0, 1, 1),
+                   row(1, 0, 0, 0),
+                   row(1, 0, 1, 0),
+                   row(1, 0, 1, 1)
+        );
     }
 
     @Test
     public void testMultipleClusteringReversedComponents() throws Throwable
     {
-        for (String compactOption : new String[]{"", " COMPACT STORAGE AND"})
-        {
-            createTable("CREATE TABLE %s (a int, b int, c int, d int, PRIMARY KEY (a, b, c, d)) WITH" + compactOption + " CLUSTERING ORDER BY (b DESC, c ASC, d DESC)");
-
-            // b and d are reversed in the clustering order
-            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 0, 0);
-            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 1, 1);
-            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 1, 0);
-
-            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 0, 0);
-            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 1, 1);
-            execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 1, 0);
-
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) > (?)", 0, 0),
-                    row(0, 1, 0, 0),
-                    row(0, 1, 1, 1),
-                    row(0, 1, 1, 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) >= (?)", 0, 0),
-                    row(0, 1, 0, 0),
-                    row(0, 1, 1, 1),
-                    row(0, 1, 1, 0),
-                    row(0, 0, 0, 0),
-                    row(0, 0, 1, 1),
-                    row(0, 0, 1, 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) < (?)", 0, 1),
-                    row(0, 0, 0, 0),
-                    row(0, 0, 1, 1),
-                    row(0, 0, 1, 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) <= (?)", 0, 1),
-                    row(0, 1, 0, 0),
-                    row(0, 1, 1, 1),
-                    row(0, 1, 1, 0),
-                    row(0, 0, 0, 0),
-                    row(0, 0, 1, 1),
-                    row(0, 0, 1, 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a=? AND (b, c, d) IN ((?, ?, ?), (?, ?, ?))", 0, 1, 1, 1, 0, 1, 1),
-                    row(0, 1, 1, 1),
-                    row(0, 0, 1, 1)
-            );
-
-            // same query, but reversed order for the IN values
-            assertRows(execute("SELECT * FROM %s WHERE a=? AND (b, c, d) IN ((?, ?, ?), (?, ?, ?))", 0, 0, 1, 1, 1, 1, 1),
-                    row(0, 1, 1, 1),
-                    row(0, 0, 1, 1)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) IN (?, ?, ?, ?, ?, ?)",
-                            0, tuple(1, 0, 0), tuple(1, 1, 1), tuple(1, 1, 0), tuple(0, 0, 0), tuple(0, 1, 1), tuple(0, 1, 0)),
-                    row(0, 1, 0, 0),
-                    row(0, 1, 1, 1),
-                    row(0, 1, 1, 0),
-                    row(0, 0, 0, 0),
-                    row(0, 0, 1, 1),
-                    row(0, 0, 1, 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c) IN (?)", 0, tuple(0, 1)),
-                    row(0, 0, 1, 1),
-                    row(0, 0, 1, 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c) IN (?)", 0, tuple(0, 0)),
-                    row(0, 0, 0, 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) IN ((?))", 0, 0),
-                    row(0, 0, 0, 0),
-                    row(0, 0, 1, 1),
-                    row(0, 0, 1, 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c) > (?, ?)", 0, 1, 0),
-                    row(0,1, 1, 1),
-                    row(0, 1, 1, 0)
-                    );
-        }
+        createTable("CREATE TABLE %s (a int, b int, c int, d int, PRIMARY KEY (a, b, c, d)) WITH CLUSTERING ORDER BY (b DESC, c ASC, d DESC)");
+
+        // b and d are reversed in the clustering order
+        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 0, 0);
+        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 1, 1);
+        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 1, 0);
+
+        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 0, 0);
+        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 1, 1);
+        execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 1, 0);
+
+
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) > (?)", 0, 0),
+                   row(0, 1, 0, 0),
+                   row(0, 1, 1, 1),
+                   row(0, 1, 1, 0)
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) >= (?)", 0, 0),
+                   row(0, 1, 0, 0),
+                   row(0, 1, 1, 1),
+                   row(0, 1, 1, 0),
+                   row(0, 0, 0, 0),
+                   row(0, 0, 1, 1),
+                   row(0, 0, 1, 0)
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) < (?)", 0, 1),
+                   row(0, 0, 0, 0),
+                   row(0, 0, 1, 1),
+                   row(0, 0, 1, 0)
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) <= (?)", 0, 1),
+                   row(0, 1, 0, 0),
+                   row(0, 1, 1, 1),
+                   row(0, 1, 1, 0),
+                   row(0, 0, 0, 0),
+                   row(0, 0, 1, 1),
+                   row(0, 0, 1, 0)
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE a=? AND (b, c, d) IN ((?, ?, ?), (?, ?, ?))", 0, 1, 1, 1, 0, 1, 1),
+                   row(0, 1, 1, 1),
+                   row(0, 0, 1, 1)
+        );
+
+        // same query, but reversed order for the IN values
+        assertRows(execute("SELECT * FROM %s WHERE a=? AND (b, c, d) IN ((?, ?, ?), (?, ?, ?))", 0, 0, 1, 1, 1, 1, 1),
+                   row(0, 1, 1, 1),
+                   row(0, 0, 1, 1)
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c, d) IN (?, ?, ?, ?, ?, ?)",
+                           0, tuple(1, 0, 0), tuple(1, 1, 1), tuple(1, 1, 0), tuple(0, 0, 0), tuple(0, 1, 1), tuple(0, 1, 0)),
+                   row(0, 1, 0, 0),
+                   row(0, 1, 1, 1),
+                   row(0, 1, 1, 0),
+                   row(0, 0, 0, 0),
+                   row(0, 0, 1, 1),
+                   row(0, 0, 1, 0)
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c) IN (?)", 0, tuple(0, 1)),
+                   row(0, 0, 1, 1),
+                   row(0, 0, 1, 0)
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c) IN (?)", 0, tuple(0, 0)),
+                   row(0, 0, 0, 0)
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) IN ((?))", 0, 0),
+                   row(0, 0, 0, 0),
+                   row(0, 0, 1, 1),
+                   row(0, 0, 1, 0)
+        );
+
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c) > (?, ?)", 0, 1, 0),
+                   row(0, 1, 1, 1),
+                   row(0, 1, 1, 0)
+        );
     }
 
     @Test
@@ -1017,16 +990,13 @@ public class SelectMultiColumnRelationTest extends CQLTester
     @Test
     public void testINWithDuplicateValue() throws Throwable
     {
-        for (String compactOption : new String[] { "", " WITH COMPACT STORAGE" })
-        {
-            createTable("CREATE TABLE %s (k1 int, k2 int, v int, PRIMARY KEY (k1, k2))" + compactOption);
-            execute("INSERT INTO %s (k1,  k2, v) VALUES (?, ?, ?)", 1, 1, 1);
+        createTable("CREATE TABLE %s (k1 int, k2 int, v int, PRIMARY KEY (k1, k2))");
+        execute("INSERT INTO %s (k1,  k2, v) VALUES (?, ?, ?)", 1, 1, 1);
 
-            assertRows(execute("SELECT * FROM %s WHERE k1 IN (?, ?) AND (k2) IN ((?), (?))", 1, 1, 1, 2),
-                       row(1, 1, 1));
-            assertRows(execute("SELECT * FROM %s WHERE k1 = ? AND (k2) IN ((?), (?))", 1, 1, 1),
-                       row(1, 1, 1));
-        }
+        assertRows(execute("SELECT * FROM %s WHERE k1 IN (?, ?) AND (k2) IN ((?), (?))", 1, 1, 1, 2),
+                   row(1, 1, 1));
+        assertRows(execute("SELECT * FROM %s WHERE k1 = ? AND (k2) IN ((?), (?))", 1, 1, 1),
+                   row(1, 1, 1));
     }
 
     @Test
@@ -1054,868 +1024,852 @@ public class SelectMultiColumnRelationTest extends CQLTester
     @Test
     public void testMixedOrderColumns1() throws Throwable
     {
-        for (String compactOption : new String[]{"", " COMPACT STORAGE AND "})
-        {
-            createTable("CREATE TABLE %s (a int, b int, c int, d int, e int, PRIMARY KEY (a, b, c, d, e)) WITH " +
-                        compactOption +
-                        " CLUSTERING ORDER BY (b DESC, c ASC, d DESC, e ASC)");
-
-            execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 2, 0, -1, 0);
-            execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 2, 0, -1, 1);
-            execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 2, 0, 1, 1);
-            execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 1, -1, 0, 0);
-            execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 1, -1, 1, 1);
-            execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 1, -1, 1, 0);
-            execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 1, 0, 1, -1);
-            execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 1, 0, 1, 1);
-            execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 1, 0, 0, -1);
-            execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 1, 0, 0, 0);
-            execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 1, 0, 0, 1);
-            execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 1, 0, -1, -1);
-            execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 1, 1, 0, -1);
-            execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 1, 1, 0, 0);
-            execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 1, 1, 0, -1);
-            execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 1, 1, 0, 0);
-            execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 1, 1, 0, 1);
-            execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 1, 1, -1, 0);
-            execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 0, 0, 0, 0);
-            execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, -1, 0, -1, 0);
-            execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, -1, 0, 0, 0);
-            assertRows(execute(
-            "SELECT * FROM %s" +
-            " WHERE a = ? " +
-            "AND (b,c,d,e)<=(?,?,?,?) " +
-            "AND (b)>(?)", 0, 2, 0, 1, 1, -1),
-
-                       row(0, 2, 0, 1, 1),
-                       row(0, 2, 0, -1, 0),
-                       row(0, 2, 0, -1, 1),
-                       row(0, 1, -1, 1, 0),
-                       row(0, 1, -1, 1, 1),
-                       row(0, 1, -1, 0, 0),
-                       row(0, 1, 0, 1, -1),
-                       row(0, 1, 0, 1, 1),
-                       row(0, 1, 0, 0, -1),
-                       row(0, 1, 0, 0, 0),
-                       row(0, 1, 0, 0, 1),
-                       row(0, 1, 0, -1, -1),
-                       row(0, 1, 1, 0, -1),
-                       row(0, 1, 1, 0, 0),
-                       row(0, 1, 1, 0, 1),
-                       row(0, 1, 1, -1, 0),
-                       row(0, 0, 0, 0, 0)
-            );
-
-
-            assertRows(execute(
-            "SELECT * FROM %s" +
-            " WHERE a = ? " +
-            "AND (b,c,d,e)<=(?,?,?,?) " +
-            "AND (b)>=(?)", 0, 2, 0, 1, 1, -1),
-
-                       row(0, 2, 0, 1, 1),
-                       row(0, 2, 0, -1, 0),
-                       row(0, 2, 0, -1, 1),
-                       row(0, 1, -1, 1, 0),
-                       row(0, 1, -1, 1, 1),
-                       row(0, 1, -1, 0, 0),
-                       row(0, 1, 0, 1, -1),
-                       row(0, 1, 0, 1, 1),
-                       row(0, 1, 0, 0, -1),
-                       row(0, 1, 0, 0, 0),
-                       row(0, 1, 0, 0, 1),
-                       row(0, 1, 0, -1, -1),
-                       row(0, 1, 1, 0, -1),
-                       row(0, 1, 1, 0, 0),
-                       row(0, 1, 1, 0, 1),
-                       row(0, 1, 1, -1, 0),
-                       row(0, 0, 0, 0, 0),
-                       row(0, -1, 0, 0, 0),
-                       row(0, -1, 0, -1, 0)
-            );
-
-            assertRows(execute(
-            "SELECT * FROM %s" +
-            " WHERE a = ? " +
-            "AND (b,c,d)>=(?,?,?)" +
-            "AND (b,c,d,e)<(?,?,?,?) ", 0, 1, 1, 0, 1, 1, 0, 1),
-                       row(0, 1, 1, 0, -1),
-                       row(0, 1, 1, 0, 0)
-
-            );
-
-            assertRows(execute(
-            "SELECT * FROM %s" +
-            " WHERE a = ? " +
-            "AND (b,c,d,e)>(?,?,?,?)" +
-            "AND (b,c,d)<=(?,?,?) ", 0, -1, 0, -1, -1, 2, 0, -1),
-
-                       row(0, 2, 0, -1, 0),
-                       row(0, 2, 0, -1, 1),
-                       row(0, 1, -1, 1, 0),
-                       row(0, 1, -1, 1, 1),
-                       row(0, 1, -1, 0, 0),
-                       row(0, 1, 0, 1, -1),
-                       row(0, 1, 0, 1, 1),
-                       row(0, 1, 0, 0, -1),
-                       row(0, 1, 0, 0, 0),
-                       row(0, 1, 0, 0, 1),
-                       row(0, 1, 0, -1, -1),
-                       row(0, 1, 1, 0, -1),
-                       row(0, 1, 1, 0, 0),
-                       row(0, 1, 1, 0, 1),
-                       row(0, 1, 1, -1, 0),
-                       row(0, 0, 0, 0, 0),
-                       row(0, -1, 0, 0, 0),
-                       row(0, -1, 0, -1, 0)
-            );
-
-            assertRows(execute(
-            "SELECT * FROM %s" +
-            " WHERE a = ? " +
-            "AND (b,c,d,e) < (?,?,?,?) " +
-            "AND (b,c,d,e)>(?,?,?,?)", 0, 1, 0, 0, 0, 1, 0, -1, -1),
-                       row(0, 1, 0, 0, -1)
-            );
-
-            assertRows(execute(
-            "SELECT * FROM %s" +
-            " WHERE a = ? " +
-            "AND (b,c,d,e) <= (?,?,?,?) " +
-            "AND (b,c,d,e)>(?,?,?,?)", 0, 1, 0, 0, 0, 1, 0, -1, -1),
-                       row(0, 1, 0, 0, -1),
-                       row(0, 1, 0, 0, 0)
-            );
-
-            assertRows(execute(
-            "SELECT * FROM %s" +
-            " WHERE a = ? " +
-            "AND (b)<(?) " +
-            "AND (b,c,d,e)>(?,?,?,?)", 0, 2, -1, 0, -1, -1),
-
-                       row(0, 1, -1, 1, 0),
-                       row(0, 1, -1, 1, 1),
-                       row(0, 1, -1, 0, 0),
-                       row(0, 1, 0, 1, -1),
-                       row(0, 1, 0, 1, 1),
-                       row(0, 1, 0, 0, -1),
-                       row(0, 1, 0, 0, 0),
-                       row(0, 1, 0, 0, 1),
-                       row(0, 1, 0, -1, -1),
-                       row(0, 1, 1, 0, -1),
-                       row(0, 1, 1, 0, 0),
-                       row(0, 1, 1, 0, 1),
-                       row(0, 1, 1, -1, 0),
-                       row(0, 0, 0, 0, 0),
-                       row(0, -1, 0, 0, 0),
-                       row(0, -1, 0, -1, 0)
-
-            );
-
-
-            assertRows(execute(
-            "SELECT * FROM %s" +
-            " WHERE a = ? " +
-            "AND (b)<(?) " +
-            "AND (b)>(?)", 0, 2, -1),
-
-                       row(0, 1, -1, 1, 0),
-                       row(0, 1, -1, 1, 1),
-                       row(0, 1, -1, 0, 0),
-                       row(0, 1, 0, 1, -1),
-                       row(0, 1, 0, 1, 1),
-                       row(0, 1, 0, 0, -1),
-                       row(0, 1, 0, 0, 0),
-                       row(0, 1, 0, 0, 1),
-                       row(0, 1, 0, -1, -1),
-                       row(0, 1, 1, 0, -1),
-                       row(0, 1, 1, 0, 0),
-                       row(0, 1, 1, 0, 1),
-                       row(0, 1, 1, -1, 0),
-                       row(0, 0, 0, 0, 0)
-
-            );
-
-            assertRows(execute(
-            "SELECT * FROM %s" +
-            " WHERE a = ? " +
-            "AND (b)<(?) " +
-            "AND (b)>=(?)", 0, 2, -1),
-
-                       row(0, 1, -1, 1, 0),
-                       row(0, 1, -1, 1, 1),
-                       row(0, 1, -1, 0, 0),
-                       row(0, 1, 0, 1, -1),
-                       row(0, 1, 0, 1, 1),
-                       row(0, 1, 0, 0, -1),
-                       row(0, 1, 0, 0, 0),
-                       row(0, 1, 0, 0, 1),
-                       row(0, 1, 0, -1, -1),
-                       row(0, 1, 1, 0, -1),
-                       row(0, 1, 1, 0, 0),
-                       row(0, 1, 1, 0, 1),
-                       row(0, 1, 1, -1, 0),
-                       row(0, 0, 0, 0, 0),
-                       row(0, -1, 0, 0, 0),
-                       row(0, -1, 0, -1, 0)
-            );
-
-            assertRows(execute(
-            "SELECT * FROM %s" +
-            " WHERE a = ? " +
-            "AND (b,c,d,e)<=(?,?,?,?) " +
-            "AND (b,c,d,e)>(?,?,?,?)", 0, 2, 0, 1, 1, -1, 0, -1, -1),
-
-                       row(0, 2, 0, 1, 1),
-                       row(0, 2, 0, -1, 0),
-                       row(0, 2, 0, -1, 1),
-                       row(0, 1, -1, 1, 0),
-                       row(0, 1, -1, 1, 1),
-                       row(0, 1, -1, 0, 0),
-                       row(0, 1, 0, 1, -1),
-                       row(0, 1, 0, 1, 1),
-                       row(0, 1, 0, 0, -1),
-                       row(0, 1, 0, 0, 0),
-                       row(0, 1, 0, 0, 1),
-                       row(0, 1, 0, -1, -1),
-                       row(0, 1, 1, 0, -1),
-                       row(0, 1, 1, 0, 0),
-                       row(0, 1, 1, 0, 1),
-                       row(0, 1, 1, -1, 0),
-                       row(0, 0, 0, 0, 0),
-                       row(0, -1, 0, 0, 0),
-                       row(0, -1, 0, -1, 0)
-            );
-
-            assertRows(execute(
-            "SELECT * FROM %s" +
-            " WHERE a = ? " +
-            "AND (b,c)<=(?,?) " +
-            "AND (b,c,d,e)>(?,?,?,?)", 0, 2, 0, -1, 0, -1, -1),
-
-                       row(0, 2, 0, 1, 1),
-                       row(0, 2, 0, -1, 0),
-                       row(0, 2, 0, -1, 1),
-                       row(0, 1, -1, 1, 0),
-                       row(0, 1, -1, 1, 1),
-                       row(0, 1, -1, 0, 0),
-                       row(0, 1, 0, 1, -1),
-                       row(0, 1, 0, 1, 1),
-                       row(0, 1, 0, 0, -1),
-                       row(0, 1, 0, 0, 0),
-                       row(0, 1, 0, 0, 1),
-                       row(0, 1, 0, -1, -1),
-                       row(0, 1, 1, 0, -1),
-                       row(0, 1, 1, 0, 0),
-                       row(0, 1, 1, 0, 1),
-                       row(0, 1, 1, -1, 0),
-                       row(0, 0, 0, 0, 0),
-                       row(0, -1, 0, 0, 0),
-                       row(0, -1, 0, -1, 0)
-            );
-
-            assertRows(execute(
-            "SELECT * FROM %s" +
-            " WHERE a = ? " +
-            "AND (b,c,d)<=(?,?,?) " +
-            "AND (b,c,d,e)>(?,?,?,?)", 0, 2, 0, -1, -1, 0, -1, -1),
-
-                       row(0, 2, 0, -1, 0),
-                       row(0, 2, 0, -1, 1),
-                       row(0, 1, -1, 1, 0),
-                       row(0, 1, -1, 1, 1),
-                       row(0, 1, -1, 0, 0),
-                       row(0, 1, 0, 1, -1),
-                       row(0, 1, 0, 1, 1),
-                       row(0, 1, 0, 0, -1),
-                       row(0, 1, 0, 0, 0),
-                       row(0, 1, 0, 0, 1),
-                       row(0, 1, 0, -1, -1),
-                       row(0, 1, 1, 0, -1),
-                       row(0, 1, 1, 0, 0),
-                       row(0, 1, 1, 0, 1),
-                       row(0, 1, 1, -1, 0),
-                       row(0, 0, 0, 0, 0),
-                       row(0, -1, 0, 0, 0),
-                       row(0, -1, 0, -1, 0)
-            );
-
-            assertRows(execute(
-            "SELECT * FROM %s" +
-            " WHERE a = ? " +
-            "AND (b,c,d,e)>(?,?,?,?)" +
-            "AND (b,c,d)<=(?,?,?) ", 0, -1, 0, -1, -1, 2, 0, -1),
-
-                       row(0, 2, 0, -1, 0),
-                       row(0, 2, 0, -1, 1),
-                       row(0, 1, -1, 1, 0),
-                       row(0, 1, -1, 1, 1),
-                       row(0, 1, -1, 0, 0),
-                       row(0, 1, 0, 1, -1),
-                       row(0, 1, 0, 1, 1),
-                       row(0, 1, 0, 0, -1),
-                       row(0, 1, 0, 0, 0),
-                       row(0, 1, 0, 0, 1),
-                       row(0, 1, 0, -1, -1),
-                       row(0, 1, 1, 0, -1),
-                       row(0, 1, 1, 0, 0),
-                       row(0, 1, 1, 0, 1),
-                       row(0, 1, 1, -1, 0),
-                       row(0, 0, 0, 0, 0),
-                       row(0, -1, 0, 0, 0),
-                       row(0, -1, 0, -1, 0)
-            );
-
-            assertRows(execute(
-            "SELECT * FROM %s" +
-            " WHERE a = ? " +
-            "AND (b,c,d)>=(?,?,?)" +
-            "AND (b,c,d,e)<(?,?,?,?) ", 0, 1, 1, 0, 1, 1, 0, 1),
-                       row(0, 1, 1, 0, -1),
-                       row(0, 1, 1, 0, 0)
-            );
-            assertRows(execute(
-            "SELECT * FROM %s" +
-            " WHERE a = ? " +
-            "AND (b,c,d,e)<(?,?,?,?) " +
-            "AND (b,c,d)>=(?,?,?)", 0, 1, 1, 0, 1, 1, 1, 0),
-                       row(0, 1, 1, 0, -1),
-                       row(0, 1, 1, 0, 0)
-
-            );
-
-            assertRows(execute(
-            "SELECT * FROM %s" +
-            " WHERE a = ? " +
-            "AND (b,c)<(?,?) " +
-            "AND (b,c,d,e)>(?,?,?,?)", 0, 2, 0, -1, 0, -1, -1),
-                       row(0, 1, -1, 1, 0),
-                       row(0, 1, -1, 1, 1),
-                       row(0, 1, -1, 0, 0),
-                       row(0, 1, 0, 1, -1),
-                       row(0, 1, 0, 1, 1),
-                       row(0, 1, 0, 0, -1),
-                       row(0, 1, 0, 0, 0),
-                       row(0, 1, 0, 0, 1),
-                       row(0, 1, 0, -1, -1),
-                       row(0, 1, 1, 0, -1),
-                       row(0, 1, 1, 0, 0),
-                       row(0, 1, 1, 0, 1),
-                       row(0, 1, 1, -1, 0),
-                       row(0, 0, 0, 0, 0),
-                       row(0, -1, 0, 0, 0),
-                       row(0, -1, 0, -1, 0)
-            );
-
-            assertRows(execute(
-            "SELECT * FROM %s" +
-            " WHERE a = ? " +
-            "AND (b,c)<(?,?) " +
-            "AND (b,c,d,e)>(?,?,?,?)", 0, 2, 0, -1, 0, -1, -1),
-                       row(0, 1, -1, 1, 0),
-                       row(0, 1, -1, 1, 1),
-                       row(0, 1, -1, 0, 0),
-                       row(0, 1, 0, 1, -1),
-                       row(0, 1, 0, 1, 1),
-                       row(0, 1, 0, 0, -1),
-                       row(0, 1, 0, 0, 0),
-                       row(0, 1, 0, 0, 1),
-                       row(0, 1, 0, -1, -1),
-                       row(0, 1, 1, 0, -1),
-                       row(0, 1, 1, 0, 0),
-                       row(0, 1, 1, 0, 1),
-                       row(0, 1, 1, -1, 0),
-                       row(0, 0, 0, 0, 0),
-                       row(0, -1, 0, 0, 0),
-                       row(0, -1, 0, -1, 0)
-            );
-
-            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b,c,d,e) <= (?,?,?,?)", 0, 1, 0, 0, 0),
-                       row(0, 1, -1, 1, 0),
-                       row(0, 1, -1, 1, 1),
-                       row(0, 1, -1, 0, 0),
-                       row(0, 1, 0, 0, -1),
-                       row(0, 1, 0, 0, 0),
-                       row(0, 1, 0, -1, -1),
-                       row(0, 0, 0, 0, 0),
-                       row(0, -1, 0, 0, 0),
-                       row(0, -1,

<TRUNCATED>

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


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

Posted by if...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/07fbd8ee/test/unit/org/apache/cassandra/cql3/validation/operations/SelectLimitTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/SelectLimitTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/SelectLimitTest.java
index 5d6ffb1..0198066 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/SelectLimitTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/SelectLimitTest.java
@@ -41,46 +41,6 @@ public class SelectLimitTest extends CQLTester
     }
 
     /**
-     * Test limit across a partition range, requires byte ordered partitioner,
-     * migrated from cql_tests.py:TestCQL.limit_range_test()
-     */
-    @Test
-    public void testPartitionRange() throws Throwable
-    {
-        createTable("CREATE TABLE %s (userid int, url text, time bigint, PRIMARY KEY (userid, url)) WITH COMPACT STORAGE");
-
-        for (int i = 0; i < 100; i++)
-            for (String tld : new String[] { "com", "org", "net" })
-                execute("INSERT INTO %s (userid, url, time) VALUES (?, ?, ?)", i, String.format("http://foo.%s", tld), 42L);
-
-        assertRows(execute("SELECT * FROM %s WHERE token(userid) >= token(2) LIMIT 1"),
-                   row(2, "http://foo.com", 42L));
-
-        assertRows(execute("SELECT * FROM %s WHERE token(userid) > token(2) LIMIT 1"),
-                   row(3, "http://foo.com", 42L));
-    }
-
-    /**
-     * Test limit across a column range,
-     * migrated from cql_tests.py:TestCQL.limit_multiget_test()
-     */
-    @Test
-    public void testColumnRange() throws Throwable
-    {
-        createTable("CREATE TABLE %s (userid int, url text, time bigint, PRIMARY KEY (userid, url)) WITH COMPACT STORAGE");
-
-        for (int i = 0; i < 100; i++)
-            for (String tld : new String[] { "com", "org", "net" })
-                execute("INSERT INTO %s (userid, url, time) VALUES (?, ?, ?)", i, String.format("http://foo.%s", tld), 42L);
-
-        // Check that we do limit the output to 1 *and* that we respect query
-        // order of keys (even though 48 is after 2)
-        assertRows(execute("SELECT * FROM %s WHERE userid IN (48, 2) LIMIT 1"),
-                   row(2, "http://foo.com", 42L));
-
-    }
-
-    /**
      * Test limit queries on a sparse table,
      * migrated from cql_tests.py:TestCQL.limit_sparse_test()
      */
@@ -98,103 +58,11 @@ public class SelectLimitTest extends CQLTester
     }
 
     @Test
-    public void testLimitInStaticTable() throws Throwable
-    {
-        createTable("CREATE TABLE %s (k int, v int, PRIMARY KEY (k) ) WITH COMPACT STORAGE ");
-
-        for (int i = 0; i < 10; i++)
-            execute("INSERT INTO %s(k, v) VALUES (?, ?)", i, i);
-
-        assertRows(execute("SELECT * FROM %s LIMIT 5"),
-                   row(0, 0),
-                   row(1, 1),
-                   row(2, 2),
-                   row(3, 3),
-                   row(4, 4));
-
-        assertRows(execute("SELECT v FROM %s LIMIT 5"),
-                   row(0),
-                   row(1),
-                   row(2),
-                   row(3),
-                   row(4));
-
-        assertRows(execute("SELECT k FROM %s LIMIT 5"),
-                   row(0),
-                   row(1),
-                   row(2),
-                   row(3),
-                   row(4));
-
-        assertRows(execute("SELECT DISTINCT k FROM %s LIMIT 5"),
-                   row(0),
-                   row(1),
-                   row(2),
-                   row(3),
-                   row(4));
-    }
-
-    /**
-     * Check for #7052 bug,
-     * migrated from cql_tests.py:TestCQL.limit_compact_table()
-     */
-    @Test
-    public void testLimitInCompactTable() throws Throwable
-    {
-        createTable("CREATE TABLE %s (k int, v int, PRIMARY KEY (k, v) ) WITH COMPACT STORAGE ");
-
-        for (int i = 0; i < 4; i++)
-            for (int j = 0; j < 4; j++)
-                execute("INSERT INTO %s(k, v) VALUES (?, ?)", i, j);
-
-        assertRows(execute("SELECT v FROM %s WHERE k=0 AND v > 0 AND v <= 4 LIMIT 2"),
-                   row(1),
-                   row(2));
-        assertRows(execute("SELECT v FROM %s WHERE k=0 AND v > -1 AND v <= 4 LIMIT 2"),
-                   row(0),
-                   row(1));
-        assertRows(execute("SELECT * FROM %s WHERE k IN (0, 1, 2) AND v > 0 AND v <= 4 LIMIT 2"),
-                   row(0, 1),
-                   row(0, 2));
-        assertRows(execute("SELECT * FROM %s WHERE k IN (0, 1, 2) AND v > -1 AND v <= 4 LIMIT 2"),
-                   row(0, 0),
-                   row(0, 1));
-        assertRows(execute("SELECT * FROM %s WHERE k IN (0, 1, 2) AND v > 0 AND v <= 4 LIMIT 6"),
-                   row(0, 1),
-                   row(0, 2),
-                   row(0, 3),
-                   row(1, 1),
-                   row(1, 2),
-                   row(1, 3));
-        assertRows(execute("SELECT * FROM %s WHERE v > 1 AND v <= 3 LIMIT 6 ALLOW FILTERING"),
-                   row(0, 2),
-                   row(0, 3),
-                   row(1, 2),
-                   row(1, 3),
-                   row(2, 2),
-                   row(2, 3));
-    }
-
-    @Test
     public void testPerPartitionLimit() throws Throwable
     {
-        perPartitionLimitTest(false);
-    }
-
-    @Test
-    public void testPerPartitionLimitWithCompactStorage() throws Throwable
-    {
-        perPartitionLimitTest(true);
-    }
-
-    private void perPartitionLimitTest(boolean withCompactStorage) throws Throwable
-    {
         String query = "CREATE TABLE %s (a int, b int, c int, PRIMARY KEY (a, b))";
 
-        if (withCompactStorage)
-            createTable(query + " WITH COMPACT STORAGE");
-        else
-            createTable(query);
+        createTable(query);
 
         for (int i = 0; i < 5; i++)
         {


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


[17/25] cassandra git commit: Merge branch 'cassandra-3.11' into trunk

Posted by if...@apache.org.
Merge branch 'cassandra-3.11' into trunk


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

Branch: refs/heads/trunk
Commit: 38bd20e0510048c3c93781afd05efdf31594c115
Parents: 5b09543 6d429cd
Author: Alex Petrov <ol...@gmail.com>
Authored: Mon Nov 6 16:38:30 2017 +0100
Committer: Alex Petrov <ol...@gmail.com>
Committed: Mon Nov 6 16:38:30 2017 +0100

----------------------------------------------------------------------
 NEWS.txt                      | 20 +++++++++++++++++++
 doc/native_protocol_v4.spec   |  4 ++++
 doc/source/cql/appendices.rst | 22 +++++++++++++++++++++
 src/antlr/Lexer.g             |  4 ++++
 src/antlr/Parser.g            | 40 +++++++++++++++++++++++++-------------
 5 files changed, 76 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/38bd20e0/NEWS.txt
----------------------------------------------------------------------
diff --cc NEWS.txt
index 09a9a7b,9812b22..30cad13
--- a/NEWS.txt
+++ b/NEWS.txt
@@@ -91,8 -40,28 +91,28 @@@ Upgradin
        storage format change, see CASSANDRA-13826. For append-only use case, you
        may still use this feature with a startup flag: "-Dcassandra.mv.allow_filtering_nonkey_columns_unsafe=true"
  
+ Compact Storage (only when upgrading from 3.X or any version lower than 3.0.15)
+ ---------------
+     - Starting version 4.0, Thrift and COMPACT STORAGE is no longer supported.
+       'ALTER ... DROP COMPACT STORAGE' statement makes Compact Tables CQL-compatible,
+       exposing internal structure of Thrift/Compact Tables. You can find more details
+       on exposed internal structure under: 
+       http://cassandra.apache.org/doc/latest/cql/appendices.html#appendix-c-dropping-compact-storage
+ 
+       For uninterrupted cluster upgrades, drivers now support 'NO_COMPACT' startup option.
+       Supplying this flag will have same effect as 'DROP COMPACT STORAGE', but only for the
+       current connection.
+ 
+       In order to upgrade, clients supporting a non-compact schema view can be rolled out
+       gradually. When all the clients are updated 'ALTER ... DROP COMPACT STORAGE' can be
+       executed. After dropping compact storage, ’NO_COMPACT' option will have no effect
+       after that.
+ 
+ 
+ Materialized Views
+ -------------------
  
 -Materialized Views (only when upgrading from 3.X or any version lower than 3.0.15)
 +Materialized Views (only when upgrading from any version lower than 3.0.15 (3.0 series) or 3.11.1 (3.X series))
  ---------------------------------------------------------------------------------------
      - Cassandra will no longer allow dropping columns on tables with Materialized Views.
      - A change was made in the way the Materialized View timestamp is computed, which

http://git-wip-us.apache.org/repos/asf/cassandra/blob/38bd20e0/src/antlr/Lexer.g
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/38bd20e0/src/antlr/Parser.g
----------------------------------------------------------------------
diff --cc src/antlr/Parser.g
index 185f968,26074b8..553d643
--- a/src/antlr/Parser.g
+++ b/src/antlr/Parser.g
@@@ -922,13 -805,14 +922,13 @@@ alterTableStatement returns [AlterTable
          Long deleteTimestamp = null;
      }
      : K_ALTER K_COLUMNFAMILY cf=columnFamilyName
-           ( K_ALTER id=cident  K_TYPE v=comparatorType  { type = AlterTableStatement.Type.ALTER; } { colNameList.add(new AlterTableStatementColumn(id,v)); }
-           | K_ADD  (        (id=cident   v=comparatorType   b1=cfisStatic { colNameList.add(new AlterTableStatementColumn(id,v,b1)); })
-                      | ('('  id1=cident  v1=comparatorType  b1=cfisStatic { colNameList.add(new AlterTableStatementColumn(id1,v1,b1)); }
-                        ( ',' idn=cident  vn=comparatorType  bn=cfisStatic { colNameList.add(new AlterTableStatementColumn(idn,vn,bn)); } )* ')' ) ) { type = AlterTableStatement.Type.ADD; }
-           | K_DROP ( (         id=cident  { colNameList.add(new AlterTableStatementColumn(id)); }
-                       | ('('  id1=cident { colNameList.add(new AlterTableStatementColumn(id1)); }
-                         ( ',' idn=cident { colNameList.add(new AlterTableStatementColumn(idn)); } )* ')') )
+           ( K_ALTER id=schema_cident  K_TYPE v=comparatorType  { type = AlterTableStatement.Type.ALTER; } { colNameList.add(new AlterTableStatementColumn(id,v)); }
+           | K_ADD  (        (aid=schema_cident  v=comparatorType   b1=cfisStatic { colNameList.add(new AlterTableStatementColumn(aid,v,b1)); })
+                      | ('('  id1=schema_cident  v1=comparatorType  b1=cfisStatic { colNameList.add(new AlterTableStatementColumn(id1,v1,b1)); }
+                        ( ',' idn=schema_cident  vn=comparatorType  bn=cfisStatic { colNameList.add(new AlterTableStatementColumn(idn,vn,bn)); } )* ')' ) ) { type = AlterTableStatement.Type.ADD; }
 -          | K_DROP K_COMPACT K_STORAGE          { type = AlterTableStatement.Type.DROP_COMPACT_STORAGE; }        
+           | K_DROP ( (        id=schema_cident  { colNameList.add(new AlterTableStatementColumn(id)); }
+                       | ('('  id1=schema_cident { colNameList.add(new AlterTableStatementColumn(id1)); }
+                         ( ',' idn=schema_cident { colNameList.add(new AlterTableStatementColumn(idn)); } )* ')') )
                       ( K_USING K_TIMESTAMP t=INTEGER { deleteTimestamp = Long.parseLong(Constants.Literal.integer($t.text).getText()); })? ) { type = AlterTableStatement.Type.DROP; }
            | K_WITH  properties[attrs]                 { type = AlterTableStatement.Type.OPTS; }
            | K_RENAME                                  { type = AlterTableStatement.Type.RENAME; }
@@@ -1290,10 -1174,20 +1290,20 @@@ userPassword[RoleOptions opts
  // Column Identifiers.  These need to be treated differently from other
  // identifiers because the underlying comparator is not necessarily text. See
  // CASSANDRA-8178 for details.
+ // Also, we need to support the internal of the super column map (for backward
+ // compatibility) which is empty (we only want to allow this is in data manipulation
+ // queries, not in schema defition etc).
 -cident returns [ColumnDefinition.Raw id]
 -    : EMPTY_QUOTED_NAME    { $id = ColumnDefinition.Raw.forQuoted(""); }
 -    | t=IDENT              { $id = ColumnDefinition.Raw.forUnquoted($t.text); }
 -    | t=QUOTED_NAME        { $id = ColumnDefinition.Raw.forQuoted($t.text); }
 -    | k=unreserved_keyword { $id = ColumnDefinition.Raw.forUnquoted(k); }
 +cident returns [ColumnMetadata.Raw id]
++    : EMPTY_QUOTED_NAME    { $id = ColumnMetadata.Raw.forQuoted(""); }
++    | t=IDENT              { $id = ColumnMetadata.Raw.forUnquoted($t.text); }
++    | t=QUOTED_NAME        { $id = ColumnMetadata.Raw.forQuoted($t.text); }
++    | k=unreserved_keyword { $id = ColumnMetadata.Raw.forUnquoted(k); }
+     ;
+ 
 -schema_cident returns [ColumnDefinition.Raw id]
 -    : t=IDENT              { $id = ColumnDefinition.Raw.forUnquoted($t.text); }
 -    | t=QUOTED_NAME        { $id = ColumnDefinition.Raw.forQuoted($t.text); }
 -    | k=unreserved_keyword { $id = ColumnDefinition.Raw.forUnquoted(k); }
++schema_cident returns [ColumnMetadata.Raw id]
 +    : t=IDENT              { $id = ColumnMetadata.Raw.forUnquoted($t.text); }
 +    | t=QUOTED_NAME        { $id = ColumnMetadata.Raw.forQuoted($t.text); }
 +    | k=unreserved_keyword { $id = ColumnMetadata.Raw.forUnquoted(k); }
      ;
  
  // Column identifiers where the comparator is known to be text


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


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

Posted by if...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/07fbd8ee/test/unit/org/apache/cassandra/cql3/validation/operations/UpdateTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/UpdateTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/UpdateTest.java
index af6c4f9..973531b 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/UpdateTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/UpdateTest.java
@@ -71,125 +71,113 @@ public class UpdateTest extends CQLTester
 
     private void testUpdate(boolean forceFlush) throws Throwable
     {
-        for (String compactOption : new String[] {"", " WITH COMPACT STORAGE" })
-        {
-            createTable("CREATE TABLE %s (partitionKey int," +
+        createTable("CREATE TABLE %s (partitionKey int," +
                     "clustering_1 int," +
                     "value int," +
-                    " PRIMARY KEY (partitionKey, clustering_1))" + compactOption);
-
-            execute("INSERT INTO %s (partitionKey, clustering_1, value) VALUES (0, 0, 0)");
-            execute("INSERT INTO %s (partitionKey, clustering_1, value) VALUES (0, 1, 1)");
-            execute("INSERT INTO %s (partitionKey, clustering_1, value) VALUES (0, 2, 2)");
-            execute("INSERT INTO %s (partitionKey, clustering_1, value) VALUES (0, 3, 3)");
-            execute("INSERT INTO %s (partitionKey, clustering_1, value) VALUES (1, 0, 4)");
-
-            flush(forceFlush);
-
-            execute("UPDATE %s SET value = ? WHERE partitionKey = ? AND clustering_1 = ?", 7, 0, 1);
-            flush(forceFlush);
-            assertRows(execute("SELECT value FROM %s WHERE partitionKey = ? AND clustering_1 = ?",
-                               0, 1),
-                       row(7));
-
-            execute("UPDATE %s SET value = ? WHERE partitionKey = ? AND (clustering_1) = (?)", 8, 0, 2);
-            flush(forceFlush);
-            assertRows(execute("SELECT value FROM %s WHERE partitionKey = ? AND clustering_1 = ?",
-                               0, 2),
-                       row(8));
-
-            execute("UPDATE %s SET value = ? WHERE partitionKey IN (?, ?) AND clustering_1 = ?", 9, 0, 1, 0);
-            flush(forceFlush);
-            assertRows(execute("SELECT * FROM %s WHERE partitionKey IN (?, ?) AND clustering_1 = ?",
-                               0, 1, 0),
-                       row(0, 0, 9),
-                       row(1, 0, 9));
-
-            execute("UPDATE %s SET value = ? WHERE partitionKey IN ? AND clustering_1 = ?", 19, Arrays.asList(0, 1), 0);
-            flush(forceFlush);
-            assertRows(execute("SELECT * FROM %s WHERE partitionKey IN ? AND clustering_1 = ?",
-                               Arrays.asList(0, 1), 0),
-                       row(0, 0, 19),
-                       row(1, 0, 19));
-
-            execute("UPDATE %s SET value = ? WHERE partitionKey = ? AND clustering_1 IN (?, ?)", 10, 0, 1, 0);
-            flush(forceFlush);
-            assertRows(execute("SELECT * FROM %s WHERE partitionKey = ? AND clustering_1 IN (?, ?)",
-                               0, 1, 0),
-                       row(0, 0, 10),
-                       row(0, 1, 10));
-
-            execute("UPDATE %s SET value = ? WHERE partitionKey = ? AND (clustering_1) IN ((?), (?))", 20, 0, 0, 1);
-            flush(forceFlush);
-            assertRows(execute("SELECT * FROM %s WHERE partitionKey = ? AND (clustering_1) IN ((?), (?))",
-                               0, 0, 1),
-                       row(0, 0, 20),
-                       row(0, 1, 20));
-
-            execute("UPDATE %s SET value = ? WHERE partitionKey = ? AND clustering_1 = ?", null, 0, 0);
-            flush(forceFlush);
-
-            if (isEmpty(compactOption))
-            {
-                assertRows(execute("SELECT * FROM %s WHERE partitionKey = ? AND (clustering_1) IN ((?), (?))",
-                                   0, 0, 1),
-                           row(0, 0, null),
-                           row(0, 1, 20));
-            }
-            else
-            {
-                assertRows(execute("SELECT * FROM %s WHERE partitionKey = ? AND (clustering_1) IN ((?), (?))",
-                                   0, 0, 1),
-                           row(0, 1, 20));
-            }
-
-            // test invalid queries
-
-            // missing primary key element
-            assertInvalidMessage("Some partition key parts are missing: partitionkey",
-                                 "UPDATE %s SET value = ? WHERE clustering_1 = ? ", 7, 1);
-
-            assertInvalidMessage("Some clustering keys are missing: clustering_1",
-                                 "UPDATE %s SET value = ? WHERE partitionKey = ?", 7, 0);
-
-            assertInvalidMessage("Some clustering keys are missing: clustering_1",
-                                 "UPDATE %s SET value = ? WHERE partitionKey = ?", 7, 0);
-
-            // token function
-            assertInvalidMessage("The token function cannot be used in WHERE clauses for UPDATE statements",
-                                 "UPDATE %s SET value = ? WHERE token(partitionKey) = token(?) AND clustering_1 = ?",
-                                 7, 0, 1);
-
-            // multiple time the same value
-            assertInvalidSyntax("UPDATE %s SET value = ?, value = ? WHERE partitionKey = ? AND clustering_1 = ?", 7, 0, 1);
-
-            // multiple time same primary key element in WHERE clause
-            assertInvalidMessage("clustering_1 cannot be restricted by more than one relation if it includes an Equal",
-                                 "UPDATE %s SET value = ? WHERE partitionKey = ? AND clustering_1 = ? AND clustering_1 = ?", 7, 0, 1, 1);
-
-            // unknown identifiers
-            assertInvalidMessage("Undefined column name value1",
-                                 "UPDATE %s SET value1 = ? WHERE partitionKey = ? AND clustering_1 = ?", 7, 0, 1);
-
-            assertInvalidMessage("Undefined column name partitionkey1",
-                                 "UPDATE %s SET value = ? WHERE partitionKey1 = ? AND clustering_1 = ?", 7, 0, 1);
-
-            assertInvalidMessage("Undefined column name clustering_3",
-                                 "UPDATE %s SET value = ? WHERE partitionKey = ? AND clustering_3 = ?", 7, 0, 1);
-
-            // Invalid operator in the where clause
-            assertInvalidMessage("Only EQ and IN relation are supported on the partition key (unless you use the token() function)",
-                                 "UPDATE %s SET value = ? WHERE partitionKey > ? AND clustering_1 = ?", 7, 0, 1);
-
-            assertInvalidMessage("Cannot use CONTAINS on non-collection column partitionkey",
-                                 "UPDATE %s SET value = ? WHERE partitionKey CONTAINS ? AND clustering_1 = ?", 7, 0, 1);
-
-            assertInvalidMessage("Non PRIMARY KEY columns found in where clause: value",
-                                 "UPDATE %s SET value = ? WHERE partitionKey = ? AND clustering_1 = ? AND value = ?", 7, 0, 1, 3);
-
-            assertInvalidMessage("Slice restrictions are not supported on the clustering columns in UPDATE statements",
-                                 "UPDATE %s SET value = ? WHERE partitionKey = ? AND clustering_1 > ?", 7, 0, 1);
-        }
+                    " PRIMARY KEY (partitionKey, clustering_1))");
+
+        execute("INSERT INTO %s (partitionKey, clustering_1, value) VALUES (0, 0, 0)");
+        execute("INSERT INTO %s (partitionKey, clustering_1, value) VALUES (0, 1, 1)");
+        execute("INSERT INTO %s (partitionKey, clustering_1, value) VALUES (0, 2, 2)");
+        execute("INSERT INTO %s (partitionKey, clustering_1, value) VALUES (0, 3, 3)");
+        execute("INSERT INTO %s (partitionKey, clustering_1, value) VALUES (1, 0, 4)");
+
+        flush(forceFlush);
+
+        execute("UPDATE %s SET value = ? WHERE partitionKey = ? AND clustering_1 = ?", 7, 0, 1);
+        flush(forceFlush);
+        assertRows(execute("SELECT value FROM %s WHERE partitionKey = ? AND clustering_1 = ?",
+                           0, 1),
+                   row(7));
+
+        execute("UPDATE %s SET value = ? WHERE partitionKey = ? AND (clustering_1) = (?)", 8, 0, 2);
+        flush(forceFlush);
+        assertRows(execute("SELECT value FROM %s WHERE partitionKey = ? AND clustering_1 = ?",
+                           0, 2),
+                   row(8));
+
+        execute("UPDATE %s SET value = ? WHERE partitionKey IN (?, ?) AND clustering_1 = ?", 9, 0, 1, 0);
+        flush(forceFlush);
+        assertRows(execute("SELECT * FROM %s WHERE partitionKey IN (?, ?) AND clustering_1 = ?",
+                           0, 1, 0),
+                   row(0, 0, 9),
+                   row(1, 0, 9));
+
+        execute("UPDATE %s SET value = ? WHERE partitionKey IN ? AND clustering_1 = ?", 19, Arrays.asList(0, 1), 0);
+        flush(forceFlush);
+        assertRows(execute("SELECT * FROM %s WHERE partitionKey IN ? AND clustering_1 = ?",
+                           Arrays.asList(0, 1), 0),
+                   row(0, 0, 19),
+                   row(1, 0, 19));
+
+        execute("UPDATE %s SET value = ? WHERE partitionKey = ? AND clustering_1 IN (?, ?)", 10, 0, 1, 0);
+        flush(forceFlush);
+        assertRows(execute("SELECT * FROM %s WHERE partitionKey = ? AND clustering_1 IN (?, ?)",
+                           0, 1, 0),
+                   row(0, 0, 10),
+                   row(0, 1, 10));
+
+        execute("UPDATE %s SET value = ? WHERE partitionKey = ? AND (clustering_1) IN ((?), (?))", 20, 0, 0, 1);
+        flush(forceFlush);
+        assertRows(execute("SELECT * FROM %s WHERE partitionKey = ? AND (clustering_1) IN ((?), (?))",
+                           0, 0, 1),
+                   row(0, 0, 20),
+                   row(0, 1, 20));
+
+        execute("UPDATE %s SET value = ? WHERE partitionKey = ? AND clustering_1 = ?", null, 0, 0);
+        flush(forceFlush);
+
+        assertRows(execute("SELECT * FROM %s WHERE partitionKey = ? AND (clustering_1) IN ((?), (?))",
+                           0, 0, 1),
+                   row(0, 0, null),
+                   row(0, 1, 20));
+
+        // test invalid queries
+
+        // missing primary key element
+        assertInvalidMessage("Some partition key parts are missing: partitionkey",
+                             "UPDATE %s SET value = ? WHERE clustering_1 = ? ", 7, 1);
+
+        assertInvalidMessage("Some clustering keys are missing: clustering_1",
+                             "UPDATE %s SET value = ? WHERE partitionKey = ?", 7, 0);
+
+        assertInvalidMessage("Some clustering keys are missing: clustering_1",
+                             "UPDATE %s SET value = ? WHERE partitionKey = ?", 7, 0);
+
+        // token function
+        assertInvalidMessage("The token function cannot be used in WHERE clauses for UPDATE statements",
+                             "UPDATE %s SET value = ? WHERE token(partitionKey) = token(?) AND clustering_1 = ?",
+                             7, 0, 1);
+
+        // multiple time the same value
+        assertInvalidSyntax("UPDATE %s SET value = ?, value = ? WHERE partitionKey = ? AND clustering_1 = ?", 7, 0, 1);
+
+        // multiple time same primary key element in WHERE clause
+        assertInvalidMessage("clustering_1 cannot be restricted by more than one relation if it includes an Equal",
+                             "UPDATE %s SET value = ? WHERE partitionKey = ? AND clustering_1 = ? AND clustering_1 = ?", 7, 0, 1, 1);
+
+        // unknown identifiers
+        assertInvalidMessage("Undefined column name value1",
+                             "UPDATE %s SET value1 = ? WHERE partitionKey = ? AND clustering_1 = ?", 7, 0, 1);
+
+        assertInvalidMessage("Undefined column name partitionkey1",
+                             "UPDATE %s SET value = ? WHERE partitionKey1 = ? AND clustering_1 = ?", 7, 0, 1);
+
+        assertInvalidMessage("Undefined column name clustering_3",
+                             "UPDATE %s SET value = ? WHERE partitionKey = ? AND clustering_3 = ?", 7, 0, 1);
+
+        // Invalid operator in the where clause
+        assertInvalidMessage("Only EQ and IN relation are supported on the partition key (unless you use the token() function)",
+                             "UPDATE %s SET value = ? WHERE partitionKey > ? AND clustering_1 = ?", 7, 0, 1);
+
+        assertInvalidMessage("Cannot use CONTAINS on non-collection column partitionkey",
+                             "UPDATE %s SET value = ? WHERE partitionKey CONTAINS ? AND clustering_1 = ?", 7, 0, 1);
+
+        assertInvalidMessage("Non PRIMARY KEY columns found in where clause: value",
+                             "UPDATE %s SET value = ? WHERE partitionKey = ? AND clustering_1 = ? AND value = ?", 7, 0, 1, 3);
+
+        assertInvalidMessage("Slice restrictions are not supported on the clustering columns in UPDATE statements",
+                             "UPDATE %s SET value = ? WHERE partitionKey = ? AND clustering_1 > ?", 7, 0, 1);
     }
 
     @Test
@@ -244,142 +232,127 @@ public class UpdateTest extends CQLTester
 
     private void testUpdateWithTwoClusteringColumns(boolean forceFlush) throws Throwable
     {
-        for (String compactOption : new String[] { "", " WITH COMPACT STORAGE" })
-        {
-            createTable("CREATE TABLE %s (partitionKey int," +
+        createTable("CREATE TABLE %s (partitionKey int," +
                     "clustering_1 int," +
                     "clustering_2 int," +
                     "value int," +
-                    " PRIMARY KEY (partitionKey, clustering_1, clustering_2))" + compactOption);
-
-            execute("INSERT INTO %s (partitionKey, clustering_1, clustering_2, value) VALUES (0, 0, 0, 0)");
-            execute("INSERT INTO %s (partitionKey, clustering_1, clustering_2, value) VALUES (0, 0, 1, 1)");
-            execute("INSERT INTO %s (partitionKey, clustering_1, clustering_2, value) VALUES (0, 0, 2, 2)");
-            execute("INSERT INTO %s (partitionKey, clustering_1, clustering_2, value) VALUES (0, 0, 3, 3)");
-            execute("INSERT INTO %s (partitionKey, clustering_1, clustering_2, value) VALUES (0, 1, 1, 4)");
-            execute("INSERT INTO %s (partitionKey, clustering_1, clustering_2, value) VALUES (0, 1, 2, 5)");
-            execute("INSERT INTO %s (partitionKey, clustering_1, clustering_2, value) VALUES (1, 0, 0, 6)");
-            flush(forceFlush);
-
-            execute("UPDATE %s SET value = ? WHERE partitionKey = ? AND clustering_1 = ? AND clustering_2 = ?", 7, 0, 1, 1);
-            flush(forceFlush);
-            assertRows(execute("SELECT value FROM %s WHERE partitionKey = ? AND clustering_1 = ? AND clustering_2 = ?",
-                               0, 1, 1),
-                       row(7));
-
-            execute("UPDATE %s SET value = ? WHERE partitionKey = ? AND (clustering_1, clustering_2) = (?, ?)", 8, 0, 1, 2);
-            flush(forceFlush);
-            assertRows(execute("SELECT value FROM %s WHERE partitionKey = ? AND clustering_1 = ? AND clustering_2 = ?",
-                               0, 1, 2),
-                       row(8));
-
-            execute("UPDATE %s SET value = ? WHERE partitionKey IN (?, ?) AND clustering_1 = ? AND clustering_2 = ?", 9, 0, 1, 0, 0);
-            flush(forceFlush);
-            assertRows(execute("SELECT * FROM %s WHERE partitionKey IN (?, ?) AND clustering_1 = ? AND clustering_2 = ?",
-                               0, 1, 0, 0),
-                       row(0, 0, 0, 9),
-                       row(1, 0, 0, 9));
-
-            execute("UPDATE %s SET value = ? WHERE partitionKey IN ? AND clustering_1 = ? AND clustering_2 = ?", 9, Arrays.asList(0, 1), 0, 0);
-            flush(forceFlush);
-            assertRows(execute("SELECT * FROM %s WHERE partitionKey IN ? AND clustering_1 = ? AND clustering_2 = ?",
-                               Arrays.asList(0, 1), 0, 0),
-                       row(0, 0, 0, 9),
-                       row(1, 0, 0, 9));
-
-            execute("UPDATE %s SET value = ? WHERE partitionKey = ? AND clustering_1 = ? AND clustering_2 IN (?, ?)", 12, 0, 1, 1, 2);
-            flush(forceFlush);
-            assertRows(execute("SELECT * FROM %s WHERE partitionKey = ? AND clustering_1 = ? AND clustering_2 IN (?, ?)",
-                               0, 1, 1, 2),
-                       row(0, 1, 1, 12),
-                       row(0, 1, 2, 12));
-
-            execute("UPDATE %s SET value = ? WHERE partitionKey = ? AND clustering_1 IN (?, ?) AND clustering_2 IN (?, ?)", 10, 0, 1, 0, 1, 2);
-            flush(forceFlush);
-            assertRows(execute("SELECT * FROM %s WHERE partitionKey = ? AND clustering_1 IN (?, ?) AND clustering_2 IN (?, ?)",
-                               0, 1, 0, 1, 2),
-                       row(0, 0, 1, 10),
-                       row(0, 0, 2, 10),
-                       row(0, 1, 1, 10),
-                       row(0, 1, 2, 10));
-
-            execute("UPDATE %s SET value = ? WHERE partitionKey = ? AND (clustering_1, clustering_2) IN ((?, ?), (?, ?))", 20, 0, 0, 2, 1, 2);
-            flush(forceFlush);
-            assertRows(execute("SELECT * FROM %s WHERE partitionKey = ? AND (clustering_1, clustering_2) IN ((?, ?), (?, ?))",
-                               0, 0, 2, 1, 2),
-                       row(0, 0, 2, 20),
-                       row(0, 1, 2, 20));
-
-            execute("UPDATE %s SET value = ? WHERE partitionKey = ? AND clustering_1 = ? AND clustering_2 = ?", null, 0, 0, 2);
-            flush(forceFlush);
-
-            if (isEmpty(compactOption))
-            {
-                assertRows(execute("SELECT * FROM %s WHERE partitionKey = ? AND (clustering_1, clustering_2) IN ((?, ?), (?, ?))",
-                                   0, 0, 2, 1, 2),
-                           row(0, 0, 2, null),
-                           row(0, 1, 2, 20));
-            }
-            else
-            {
-                assertRows(execute("SELECT * FROM %s WHERE partitionKey = ? AND (clustering_1, clustering_2) IN ((?, ?), (?, ?))",
-                                   0, 0, 2, 1, 2),
-                           row(0, 1, 2, 20));
-            }
-
-            // test invalid queries
-
-            // missing primary key element
-            assertInvalidMessage("Some partition key parts are missing: partitionkey",
-                                 "UPDATE %s SET value = ? WHERE clustering_1 = ? AND clustering_2 = ?", 7, 1, 1);
-
-            String errorMsg = isEmpty(compactOption) ? "Some clustering keys are missing: clustering_1"
-                                                     : "PRIMARY KEY column \"clustering_2\" cannot be restricted as preceding column \"clustering_1\" is not restricted";
-
-            assertInvalidMessage(errorMsg,
-                                 "UPDATE %s SET value = ? WHERE partitionKey = ? AND clustering_2 = ?", 7, 0, 1);
-
-            assertInvalidMessage("Some clustering keys are missing: clustering_1, clustering_2",
-                                 "UPDATE %s SET value = ? WHERE partitionKey = ?", 7, 0);
-
-            // token function
-            assertInvalidMessage("The token function cannot be used in WHERE clauses for UPDATE statements",
-                                 "UPDATE %s SET value = ? WHERE token(partitionKey) = token(?) AND clustering_1 = ? AND clustering_2 = ?",
-                                 7, 0, 1, 1);
-
-            // multiple time the same value
-            assertInvalidSyntax("UPDATE %s SET value = ?, value = ? WHERE partitionKey = ? AND clustering_1 = ? AND clustering_2 = ?", 7, 0, 1, 1);
-
-            // multiple time same primary key element in WHERE clause
-            assertInvalidMessage("clustering_1 cannot be restricted by more than one relation if it includes an Equal",
-                                 "UPDATE %s SET value = ? WHERE partitionKey = ? AND clustering_1 = ? AND clustering_2 = ? AND clustering_1 = ?", 7, 0, 1, 1, 1);
-
-            // unknown identifiers
-            assertInvalidMessage("Undefined column name value1",
-                                 "UPDATE %s SET value1 = ? WHERE partitionKey = ? AND clustering_1 = ? AND clustering_2 = ?", 7, 0, 1, 1);
-
-            assertInvalidMessage("Undefined column name partitionkey1",
-                                 "UPDATE %s SET value = ? WHERE partitionKey1 = ? AND clustering_1 = ? AND clustering_2 = ?", 7, 0, 1, 1);
-
-            assertInvalidMessage("Undefined column name clustering_3",
-                                 "UPDATE %s SET value = ? WHERE partitionKey = ? AND clustering_1 = ? AND clustering_3 = ?", 7, 0, 1, 1);
-
-            // Invalid operator in the where clause
-            assertInvalidMessage("Only EQ and IN relation are supported on the partition key (unless you use the token() function)",
-                                 "UPDATE %s SET value = ? WHERE partitionKey > ? AND clustering_1 = ? AND clustering_2 = ?", 7, 0, 1, 1);
-
-            assertInvalidMessage("Cannot use CONTAINS on non-collection column partitionkey",
-                                 "UPDATE %s SET value = ? WHERE partitionKey CONTAINS ? AND clustering_1 = ? AND clustering_2 = ?", 7, 0, 1, 1);
-
-            assertInvalidMessage("Non PRIMARY KEY columns found in where clause: value",
-                                 "UPDATE %s SET value = ? WHERE partitionKey = ? AND clustering_1 = ? AND clustering_2 = ? AND value = ?", 7, 0, 1, 1, 3);
-
-            assertInvalidMessage("Slice restrictions are not supported on the clustering columns in UPDATE statements",
-                                 "UPDATE %s SET value = ? WHERE partitionKey = ? AND clustering_1 > ?", 7, 0, 1);
-
-            assertInvalidMessage("Slice restrictions are not supported on the clustering columns in UPDATE statements",
-                                 "UPDATE %s SET value = ? WHERE partitionKey = ? AND (clustering_1, clustering_2) > (?, ?)", 7, 0, 1, 1);
-        }
+                    " PRIMARY KEY (partitionKey, clustering_1, clustering_2))");
+
+        execute("INSERT INTO %s (partitionKey, clustering_1, clustering_2, value) VALUES (0, 0, 0, 0)");
+        execute("INSERT INTO %s (partitionKey, clustering_1, clustering_2, value) VALUES (0, 0, 1, 1)");
+        execute("INSERT INTO %s (partitionKey, clustering_1, clustering_2, value) VALUES (0, 0, 2, 2)");
+        execute("INSERT INTO %s (partitionKey, clustering_1, clustering_2, value) VALUES (0, 0, 3, 3)");
+        execute("INSERT INTO %s (partitionKey, clustering_1, clustering_2, value) VALUES (0, 1, 1, 4)");
+        execute("INSERT INTO %s (partitionKey, clustering_1, clustering_2, value) VALUES (0, 1, 2, 5)");
+        execute("INSERT INTO %s (partitionKey, clustering_1, clustering_2, value) VALUES (1, 0, 0, 6)");
+        flush(forceFlush);
+
+        execute("UPDATE %s SET value = ? WHERE partitionKey = ? AND clustering_1 = ? AND clustering_2 = ?", 7, 0, 1, 1);
+        flush(forceFlush);
+        assertRows(execute("SELECT value FROM %s WHERE partitionKey = ? AND clustering_1 = ? AND clustering_2 = ?",
+                           0, 1, 1),
+                   row(7));
+
+        execute("UPDATE %s SET value = ? WHERE partitionKey = ? AND (clustering_1, clustering_2) = (?, ?)", 8, 0, 1, 2);
+        flush(forceFlush);
+        assertRows(execute("SELECT value FROM %s WHERE partitionKey = ? AND clustering_1 = ? AND clustering_2 = ?",
+                           0, 1, 2),
+                   row(8));
+
+        execute("UPDATE %s SET value = ? WHERE partitionKey IN (?, ?) AND clustering_1 = ? AND clustering_2 = ?", 9, 0, 1, 0, 0);
+        flush(forceFlush);
+        assertRows(execute("SELECT * FROM %s WHERE partitionKey IN (?, ?) AND clustering_1 = ? AND clustering_2 = ?",
+                           0, 1, 0, 0),
+                   row(0, 0, 0, 9),
+                   row(1, 0, 0, 9));
+
+        execute("UPDATE %s SET value = ? WHERE partitionKey IN ? AND clustering_1 = ? AND clustering_2 = ?", 9, Arrays.asList(0, 1), 0, 0);
+        flush(forceFlush);
+        assertRows(execute("SELECT * FROM %s WHERE partitionKey IN ? AND clustering_1 = ? AND clustering_2 = ?",
+                           Arrays.asList(0, 1), 0, 0),
+                   row(0, 0, 0, 9),
+                   row(1, 0, 0, 9));
+
+        execute("UPDATE %s SET value = ? WHERE partitionKey = ? AND clustering_1 = ? AND clustering_2 IN (?, ?)", 12, 0, 1, 1, 2);
+        flush(forceFlush);
+        assertRows(execute("SELECT * FROM %s WHERE partitionKey = ? AND clustering_1 = ? AND clustering_2 IN (?, ?)",
+                           0, 1, 1, 2),
+                   row(0, 1, 1, 12),
+                   row(0, 1, 2, 12));
+
+        execute("UPDATE %s SET value = ? WHERE partitionKey = ? AND clustering_1 IN (?, ?) AND clustering_2 IN (?, ?)", 10, 0, 1, 0, 1, 2);
+        flush(forceFlush);
+        assertRows(execute("SELECT * FROM %s WHERE partitionKey = ? AND clustering_1 IN (?, ?) AND clustering_2 IN (?, ?)",
+                           0, 1, 0, 1, 2),
+                   row(0, 0, 1, 10),
+                   row(0, 0, 2, 10),
+                   row(0, 1, 1, 10),
+                   row(0, 1, 2, 10));
+
+        execute("UPDATE %s SET value = ? WHERE partitionKey = ? AND (clustering_1, clustering_2) IN ((?, ?), (?, ?))", 20, 0, 0, 2, 1, 2);
+        flush(forceFlush);
+        assertRows(execute("SELECT * FROM %s WHERE partitionKey = ? AND (clustering_1, clustering_2) IN ((?, ?), (?, ?))",
+                           0, 0, 2, 1, 2),
+                   row(0, 0, 2, 20),
+                   row(0, 1, 2, 20));
+
+        execute("UPDATE %s SET value = ? WHERE partitionKey = ? AND clustering_1 = ? AND clustering_2 = ?", null, 0, 0, 2);
+        flush(forceFlush);
+
+        assertRows(execute("SELECT * FROM %s WHERE partitionKey = ? AND (clustering_1, clustering_2) IN ((?, ?), (?, ?))",
+                           0, 0, 2, 1, 2),
+                   row(0, 0, 2, null),
+                   row(0, 1, 2, 20));
+
+        // test invalid queries
+
+        // missing primary key element
+        assertInvalidMessage("Some partition key parts are missing: partitionkey",
+                             "UPDATE %s SET value = ? WHERE clustering_1 = ? AND clustering_2 = ?", 7, 1, 1);
+
+        assertInvalidMessage("Some clustering keys are missing: clustering_1",
+                             "UPDATE %s SET value = ? WHERE partitionKey = ? AND clustering_2 = ?", 7, 0, 1);
+
+        assertInvalidMessage("Some clustering keys are missing: clustering_1, clustering_2",
+                             "UPDATE %s SET value = ? WHERE partitionKey = ?", 7, 0);
+
+        // token function
+        assertInvalidMessage("The token function cannot be used in WHERE clauses for UPDATE statements",
+                             "UPDATE %s SET value = ? WHERE token(partitionKey) = token(?) AND clustering_1 = ? AND clustering_2 = ?",
+                             7, 0, 1, 1);
+
+        // multiple time the same value
+        assertInvalidSyntax("UPDATE %s SET value = ?, value = ? WHERE partitionKey = ? AND clustering_1 = ? AND clustering_2 = ?", 7, 0, 1, 1);
+
+        // multiple time same primary key element in WHERE clause
+        assertInvalidMessage("clustering_1 cannot be restricted by more than one relation if it includes an Equal",
+                             "UPDATE %s SET value = ? WHERE partitionKey = ? AND clustering_1 = ? AND clustering_2 = ? AND clustering_1 = ?", 7, 0, 1, 1, 1);
+
+        // unknown identifiers
+        assertInvalidMessage("Undefined column name value1",
+                             "UPDATE %s SET value1 = ? WHERE partitionKey = ? AND clustering_1 = ? AND clustering_2 = ?", 7, 0, 1, 1);
+
+        assertInvalidMessage("Undefined column name partitionkey1",
+                             "UPDATE %s SET value = ? WHERE partitionKey1 = ? AND clustering_1 = ? AND clustering_2 = ?", 7, 0, 1, 1);
+
+        assertInvalidMessage("Undefined column name clustering_3",
+                             "UPDATE %s SET value = ? WHERE partitionKey = ? AND clustering_1 = ? AND clustering_3 = ?", 7, 0, 1, 1);
+
+        // Invalid operator in the where clause
+        assertInvalidMessage("Only EQ and IN relation are supported on the partition key (unless you use the token() function)",
+                             "UPDATE %s SET value = ? WHERE partitionKey > ? AND clustering_1 = ? AND clustering_2 = ?", 7, 0, 1, 1);
+
+        assertInvalidMessage("Cannot use CONTAINS on non-collection column partitionkey",
+                             "UPDATE %s SET value = ? WHERE partitionKey CONTAINS ? AND clustering_1 = ? AND clustering_2 = ?", 7, 0, 1, 1);
+
+        assertInvalidMessage("Non PRIMARY KEY columns found in where clause: value",
+                             "UPDATE %s SET value = ? WHERE partitionKey = ? AND clustering_1 = ? AND clustering_2 = ? AND value = ?", 7, 0, 1, 1, 3);
+
+        assertInvalidMessage("Slice restrictions are not supported on the clustering columns in UPDATE statements",
+                             "UPDATE %s SET value = ? WHERE partitionKey = ? AND clustering_1 > ?", 7, 0, 1);
+
+        assertInvalidMessage("Slice restrictions are not supported on the clustering columns in UPDATE statements",
+                             "UPDATE %s SET value = ? WHERE partitionKey = ? AND (clustering_1, clustering_2) > (?, ?)", 7, 0, 1, 1);
     }
 
     @Test
@@ -391,49 +364,46 @@ public class UpdateTest extends CQLTester
 
     public void testUpdateWithMultiplePartitionKeyComponents(boolean forceFlush) throws Throwable
     {
-        for (String compactOption : new String[] { "", " WITH COMPACT STORAGE" })
-        {
-            createTable("CREATE TABLE %s (partitionKey_1 int," +
+        createTable("CREATE TABLE %s (partitionKey_1 int," +
                     "partitionKey_2 int," +
                     "clustering_1 int," +
                     "clustering_2 int," +
                     "value int," +
-                    " PRIMARY KEY ((partitionKey_1, partitionKey_2), clustering_1, clustering_2))" + compactOption);
-
-            execute("INSERT INTO %s (partitionKey_1, partitionKey_2, clustering_1, clustering_2, value) VALUES (0, 0, 0, 0, 0)");
-            execute("INSERT INTO %s (partitionKey_1, partitionKey_2, clustering_1, clustering_2, value) VALUES (0, 1, 0, 1, 1)");
-            execute("INSERT INTO %s (partitionKey_1, partitionKey_2, clustering_1, clustering_2, value) VALUES (0, 1, 1, 1, 2)");
-            execute("INSERT INTO %s (partitionKey_1, partitionKey_2, clustering_1, clustering_2, value) VALUES (1, 0, 0, 1, 3)");
-            execute("INSERT INTO %s (partitionKey_1, partitionKey_2, clustering_1, clustering_2, value) VALUES (1, 1, 0, 1, 3)");
-            flush(forceFlush);
-
-            execute("UPDATE %s SET value = ? WHERE partitionKey_1 = ? AND partitionKey_2 = ? AND clustering_1 = ? AND clustering_2 = ?", 7, 0, 0, 0, 0);
-            flush(forceFlush);
-            assertRows(execute("SELECT value FROM %s WHERE partitionKey_1 = ? AND partitionKey_2 = ? AND clustering_1 = ? AND clustering_2 = ?",
-                               0, 0, 0, 0),
-                       row(7));
-
-            execute("UPDATE %s SET value = ? WHERE partitionKey_1 IN (?, ?) AND partitionKey_2 = ? AND clustering_1 = ? AND clustering_2 = ?", 9, 0, 1, 1, 0, 1);
-            flush(forceFlush);
-            assertRows(execute("SELECT * FROM %s WHERE partitionKey_1 IN (?, ?) AND partitionKey_2 = ? AND clustering_1 = ? AND clustering_2 = ?",
-                               0, 1, 1, 0, 1),
-                       row(0, 1, 0, 1, 9),
-                       row(1, 1, 0, 1, 9));
-
-            execute("UPDATE %s SET value = ? WHERE partitionKey_1 IN (?, ?) AND partitionKey_2 IN (?, ?) AND clustering_1 = ? AND clustering_2 = ?", 10, 0, 1, 0, 1, 0, 1);
-            flush(forceFlush);
-            assertRows(execute("SELECT * FROM %s"),
-                       row(0, 0, 0, 0, 7),
-                       row(0, 0, 0, 1, 10),
-                       row(0, 1, 0, 1, 10),
-                       row(0, 1, 1, 1, 2),
-                       row(1, 0, 0, 1, 10),
-                       row(1, 1, 0, 1, 10));
-
-            // missing primary key element
-            assertInvalidMessage("Some partition key parts are missing: partitionkey_2",
-                                 "UPDATE %s SET value = ? WHERE partitionKey_1 = ? AND clustering_1 = ? AND clustering_2 = ?", 7, 1, 1);
-        }
+                    " PRIMARY KEY ((partitionKey_1, partitionKey_2), clustering_1, clustering_2))");
+
+        execute("INSERT INTO %s (partitionKey_1, partitionKey_2, clustering_1, clustering_2, value) VALUES (0, 0, 0, 0, 0)");
+        execute("INSERT INTO %s (partitionKey_1, partitionKey_2, clustering_1, clustering_2, value) VALUES (0, 1, 0, 1, 1)");
+        execute("INSERT INTO %s (partitionKey_1, partitionKey_2, clustering_1, clustering_2, value) VALUES (0, 1, 1, 1, 2)");
+        execute("INSERT INTO %s (partitionKey_1, partitionKey_2, clustering_1, clustering_2, value) VALUES (1, 0, 0, 1, 3)");
+        execute("INSERT INTO %s (partitionKey_1, partitionKey_2, clustering_1, clustering_2, value) VALUES (1, 1, 0, 1, 3)");
+        flush(forceFlush);
+
+        execute("UPDATE %s SET value = ? WHERE partitionKey_1 = ? AND partitionKey_2 = ? AND clustering_1 = ? AND clustering_2 = ?", 7, 0, 0, 0, 0);
+        flush(forceFlush);
+        assertRows(execute("SELECT value FROM %s WHERE partitionKey_1 = ? AND partitionKey_2 = ? AND clustering_1 = ? AND clustering_2 = ?",
+                           0, 0, 0, 0),
+                   row(7));
+
+        execute("UPDATE %s SET value = ? WHERE partitionKey_1 IN (?, ?) AND partitionKey_2 = ? AND clustering_1 = ? AND clustering_2 = ?", 9, 0, 1, 1, 0, 1);
+        flush(forceFlush);
+        assertRows(execute("SELECT * FROM %s WHERE partitionKey_1 IN (?, ?) AND partitionKey_2 = ? AND clustering_1 = ? AND clustering_2 = ?",
+                           0, 1, 1, 0, 1),
+                   row(0, 1, 0, 1, 9),
+                   row(1, 1, 0, 1, 9));
+
+        execute("UPDATE %s SET value = ? WHERE partitionKey_1 IN (?, ?) AND partitionKey_2 IN (?, ?) AND clustering_1 = ? AND clustering_2 = ?", 10, 0, 1, 0, 1, 0, 1);
+        flush(forceFlush);
+        assertRows(execute("SELECT * FROM %s"),
+                   row(0, 0, 0, 0, 7),
+                   row(0, 0, 0, 1, 10),
+                   row(0, 1, 0, 1, 10),
+                   row(0, 1, 1, 1, 2),
+                   row(1, 0, 0, 1, 10),
+                   row(1, 1, 0, 1, 10));
+
+        // missing primary key element
+        assertInvalidMessage("Some partition key parts are missing: partitionkey_2",
+                             "UPDATE %s SET value = ? WHERE partitionKey_1 = ? AND clustering_1 = ? AND clustering_2 = ?", 7, 1, 1);
     }
 
     @Test

http://git-wip-us.apache.org/repos/asf/cassandra/blob/07fbd8ee/test/unit/org/apache/cassandra/db/ColumnFamilyStoreCQLHelperTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/ColumnFamilyStoreCQLHelperTest.java b/test/unit/org/apache/cassandra/db/ColumnFamilyStoreCQLHelperTest.java
index 6d680ce..453ab06 100644
--- a/test/unit/org/apache/cassandra/db/ColumnFamilyStoreCQLHelperTest.java
+++ b/test/unit/org/apache/cassandra/db/ColumnFamilyStoreCQLHelperTest.java
@@ -234,71 +234,6 @@ public class ColumnFamilyStoreCQLHelperTest extends CQLTester
     }
 
     @Test
-    public void testCfmCompactStorageCQL()
-    {
-        String keyspace = "cql_test_keyspace_compact";
-        String table = "test_table_compact";
-
-        TableMetadata.Builder metadata =
-            TableMetadata.builder(keyspace, table)
-                         .isDense(true)
-                         .addPartitionKeyColumn("pk1", IntegerType.instance)
-                         .addPartitionKeyColumn("pk2", AsciiType.instance)
-                         .addClusteringColumn("ck1", ReversedType.getInstance(IntegerType.instance))
-                         .addClusteringColumn("ck2", IntegerType.instance)
-                         .addRegularColumn("reg", IntegerType.instance);
-
-        SchemaLoader.createKeyspace(keyspace, KeyspaceParams.simple(1), metadata);
-
-        ColumnFamilyStore cfs = Keyspace.open(keyspace).getColumnFamilyStore(table);
-
-        assertTrue(ColumnFamilyStoreCQLHelper.getTableMetadataAsCQL(cfs.metadata(), true).startsWith(
-        "CREATE TABLE IF NOT EXISTS cql_test_keyspace_compact.test_table_compact (\n" +
-        "\tpk1 varint,\n" +
-        "\tpk2 ascii,\n" +
-        "\tck1 varint,\n" +
-        "\tck2 varint,\n" +
-        "\treg varint,\n" +
-        "\tPRIMARY KEY ((pk1, pk2), ck1, ck2))\n" +
-        "\tWITH ID = " + cfs.metadata.id + "\n" +
-        "\tAND COMPACT STORAGE\n" +
-        "\tAND CLUSTERING ORDER BY (ck1 DESC, ck2 ASC)"));
-    }
-
-    @Test
-    public void testCfmCounterCQL()
-    {
-        String keyspace = "cql_test_keyspace_counter";
-        String table = "test_table_counter";
-
-        TableMetadata.Builder metadata =
-            TableMetadata.builder(keyspace, table)
-                         .isDense(true)
-                         .isCounter(true)
-                         .addPartitionKeyColumn("pk1", IntegerType.instance)
-                         .addPartitionKeyColumn("pk2", AsciiType.instance)
-                         .addClusteringColumn("ck1", ReversedType.getInstance(IntegerType.instance))
-                         .addClusteringColumn("ck2", IntegerType.instance)
-                         .addRegularColumn("cnt", CounterColumnType.instance);
-
-        SchemaLoader.createKeyspace(keyspace, KeyspaceParams.simple(1), metadata);
-
-        ColumnFamilyStore cfs = Keyspace.open(keyspace).getColumnFamilyStore(table);
-
-        assertTrue(ColumnFamilyStoreCQLHelper.getTableMetadataAsCQL(cfs.metadata(), true).startsWith(
-        "CREATE TABLE IF NOT EXISTS cql_test_keyspace_counter.test_table_counter (\n" +
-        "\tpk1 varint,\n" +
-        "\tpk2 ascii,\n" +
-        "\tck1 varint,\n" +
-        "\tck2 varint,\n" +
-        "\tcnt counter,\n" +
-        "\tPRIMARY KEY ((pk1, pk2), ck1, ck2))\n" +
-        "\tWITH ID = " + cfs.metadata.id + "\n" +
-        "\tAND COMPACT STORAGE\n" +
-        "\tAND CLUSTERING ORDER BY (ck1 DESC, ck2 ASC)"));
-    }
-
-    @Test
     public void testCfmOptionsCQL()
     {
         String keyspace = "cql_test_keyspace_options";
@@ -508,165 +443,4 @@ public class ColumnFamilyStoreCQLHelperTest extends CQLTester
         schemaChange("DROP TYPE " + keyspace() + "." + typeA);
         validate.run();
     }
-
-    @Test
-    public void testDenseTable() throws Throwable
-    {
-        String tableName = createTable("CREATE TABLE IF NOT EXISTS %s (" +
-                                       "pk1 varint PRIMARY KEY," +
-                                       "reg1 int)" +
-                                       " WITH COMPACT STORAGE");
-
-        ColumnFamilyStore cfs = Keyspace.open(keyspace()).getColumnFamilyStore(tableName);
-
-        assertTrue(ColumnFamilyStoreCQLHelper.getTableMetadataAsCQL(cfs.metadata(), true).startsWith(
-        "CREATE TABLE IF NOT EXISTS " + keyspace() + "." + tableName + " (\n" +
-        "\tpk1 varint PRIMARY KEY,\n" +
-        "\treg1 int)\n" +
-        "\tWITH ID = " + cfs.metadata.id + "\n" +
-        "\tAND COMPACT STORAGE"));
-    }
-
-    @Test
-    public void testStaticCompactTable() throws Throwable
-    {
-        String tableName = createTable("CREATE TABLE IF NOT EXISTS %s (" +
-                                       "pk1 varint PRIMARY KEY," +
-                                       "reg1 int," +
-                                       "reg2 int)" +
-                                       " WITH COMPACT STORAGE");
-
-        ColumnFamilyStore cfs = Keyspace.open(keyspace()).getColumnFamilyStore(tableName);
-
-        assertTrue(ColumnFamilyStoreCQLHelper.getTableMetadataAsCQL(cfs.metadata(), true).startsWith(
-        "CREATE TABLE IF NOT EXISTS " + keyspace() + "." + tableName + " (\n" +
-        "\tpk1 varint PRIMARY KEY,\n" +
-        "\treg1 int,\n" +
-        "\treg2 int)\n" +
-        "\tWITH ID = " + cfs.metadata.id + "\n" +
-        "\tAND COMPACT STORAGE"));
-    }
-
-    @Test
-    public void testStaticCompactWithCounters() throws Throwable
-    {
-        String tableName = createTable("CREATE TABLE IF NOT EXISTS %s (" +
-                                       "pk1 varint PRIMARY KEY," +
-                                       "reg1 counter," +
-                                       "reg2 counter)" +
-                                       " WITH COMPACT STORAGE");
-
-
-        ColumnFamilyStore cfs = Keyspace.open(keyspace()).getColumnFamilyStore(tableName);
-
-        assertTrue(ColumnFamilyStoreCQLHelper.getTableMetadataAsCQL(cfs.metadata(), true).startsWith(
-        "CREATE TABLE IF NOT EXISTS " + keyspace() + "." + tableName + " (\n" +
-        "\tpk1 varint PRIMARY KEY,\n" +
-        "\treg1 counter,\n" +
-        "\treg2 counter)\n" +
-        "\tWITH ID = " + cfs.metadata.id + "\n" +
-        "\tAND COMPACT STORAGE"));
-    }
-
-    @Test
-    public void testDenseCompactTableWithoutRegulars() throws Throwable
-    {
-        String tableName = createTable("CREATE TABLE IF NOT EXISTS %s (" +
-                                       "pk1 varint," +
-                                       "ck1 int," +
-                                       "PRIMARY KEY (pk1, ck1))" +
-                                       " WITH COMPACT STORAGE");
-
-        ColumnFamilyStore cfs = Keyspace.open(keyspace()).getColumnFamilyStore(tableName);
-
-        assertTrue(ColumnFamilyStoreCQLHelper.getTableMetadataAsCQL(cfs.metadata(), true).startsWith(
-        "CREATE TABLE IF NOT EXISTS " + keyspace() + "." + tableName + " (\n" +
-        "\tpk1 varint,\n" +
-        "\tck1 int,\n" +
-        "\tPRIMARY KEY (pk1, ck1))\n" +
-        "\tWITH ID = " + cfs.metadata.id + "\n" +
-        "\tAND COMPACT STORAGE"));
-    }
-
-    @Test
-    public void testCompactDynamic() throws Throwable
-    {
-        String tableName = createTable("CREATE TABLE IF NOT EXISTS %s (" +
-                                       "pk1 varint," +
-                                       "ck1 int," +
-                                       "reg int," +
-                                       "PRIMARY KEY (pk1, ck1))" +
-                                       " WITH COMPACT STORAGE");
-
-        ColumnFamilyStore cfs = Keyspace.open(keyspace()).getColumnFamilyStore(tableName);
-
-        assertTrue(ColumnFamilyStoreCQLHelper.getTableMetadataAsCQL(cfs.metadata(), true).startsWith(
-        "CREATE TABLE IF NOT EXISTS " + keyspace() + "." + tableName + " (\n" +
-        "\tpk1 varint,\n" +
-        "\tck1 int,\n" +
-        "\treg int,\n" +
-        "\tPRIMARY KEY (pk1, ck1))\n" +
-        "\tWITH ID = " + cfs.metadata.id + "\n" +
-        "\tAND COMPACT STORAGE"));
-    }
-
-    @Test
-    public void testDynamicComposite() throws Throwable
-    {
-        Map<Byte, AbstractType<?>> aliases = new HashMap<>();
-        aliases.put((byte)'a', BytesType.instance);
-        aliases.put((byte)'b', BytesType.instance);
-        aliases.put((byte)'c', BytesType.instance);
-
-        String DYNAMIC_COMPOSITE = "dynamic_composite";
-        AbstractType<?> dynamicComposite = DynamicCompositeType.getInstance(aliases);
-
-        SchemaLoader.createKeyspace(DYNAMIC_COMPOSITE,
-                                    KeyspaceParams.simple(1),
-                                    SchemaLoader.denseCFMD(DYNAMIC_COMPOSITE, DYNAMIC_COMPOSITE, dynamicComposite));
-
-        ColumnFamilyStore cfs = Keyspace.open(DYNAMIC_COMPOSITE).getColumnFamilyStore(DYNAMIC_COMPOSITE);
-
-        assertTrue(ColumnFamilyStoreCQLHelper.getTableMetadataAsCQL(cfs.metadata(), true).startsWith(
-        "CREATE TABLE IF NOT EXISTS " + DYNAMIC_COMPOSITE + "." + DYNAMIC_COMPOSITE + " (\n" +
-        "\tkey ascii,\n" +
-        "\tcols 'org.apache.cassandra.db.marshal.DynamicCompositeType(a=>org.apache.cassandra.db.marshal.BytesType,b=>org.apache.cassandra.db.marshal.BytesType,c=>org.apache.cassandra.db.marshal.BytesType)',\n" +
-        "\tval ascii,\n" +
-        "\tPRIMARY KEY (key, cols))\n" +
-        "\tWITH ID = " + cfs.metadata.id + "\n" +
-        "\tAND COMPACT STORAGE"));
-    }
-
-    @Test
-    public void superColumnFamilyTest() throws Throwable
-    {
-        final String KEYSPACE = "thrift_compact_table_with_supercolumns_test";
-        final String TABLE = "test_table_1";
-
-        TableMetadata.Builder table =
-            TableMetadata.builder(KEYSPACE, TABLE)
-                         .isSuper(true)
-                         .addPartitionKeyColumn("pk", BytesType.instance)
-                         .addClusteringColumn("c1", AsciiType.instance)
-                         .addClusteringColumn("c2", AsciiType.instance)
-                         .addRegularColumn("", MapType.getInstance(Int32Type.instance, AsciiType.instance, true));
-
-        SchemaLoader.createKeyspace(KEYSPACE, KeyspaceParams.simple(1), table);
-
-        ColumnFamilyStore cfs = Keyspace.open(KEYSPACE).getColumnFamilyStore(TABLE);
-
-        assertTrue(ColumnFamilyStoreCQLHelper.getTableMetadataAsCQL(cfs.metadata(), true).startsWith(
-        "/*\n" +
-        "Warning: Table " + KEYSPACE + "." + TABLE + " omitted because it has constructs not compatible with CQL (was created via legacy API).\n\n" +
-        "Approximate structure, for reference:\n" +
-        "(this should not be used to reproduce this schema)\n\n" +
-        "CREATE TABLE IF NOT EXISTS " + KEYSPACE + "." + TABLE + " (\n" +
-        "\tpk blob,\n" +
-        "\tc1 ascii,\n" +
-        "\tc2 ascii,\n" +
-        "\t\"\" map<int, ascii>,\n" +
-        "\tPRIMARY KEY (pk, c1, c2))\n" +
-        "\tWITH ID = " + cfs.metadata.id + "\n" +
-        "\tAND COMPACT STORAGE"));
-    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/07fbd8ee/test/unit/org/apache/cassandra/db/ScrubTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/ScrubTest.java b/test/unit/org/apache/cassandra/db/ScrubTest.java
index 490431a..24eb0d0 100644
--- a/test/unit/org/apache/cassandra/db/ScrubTest.java
+++ b/test/unit/org/apache/cassandra/db/ScrubTest.java
@@ -520,33 +520,6 @@ public class ScrubTest
         CompactionManager.instance.performScrub(cfs2, false, false, 2);
     }
 
-    /**
-     * For CASSANDRA-6892 too, check that for a compact table with one cluster column, we can insert whatever
-     * we want as value for the clustering column, including something that would conflict with a CQL column definition.
-     */
-    @Test
-    public void testValidationCompactStorage() throws Exception
-    {
-        QueryProcessor.process(String.format("CREATE TABLE \"%s\".test_compact_dynamic_columns (a int, b text, c text, PRIMARY KEY (a, b)) WITH COMPACT STORAGE", KEYSPACE), ConsistencyLevel.ONE);
-
-        Keyspace keyspace = Keyspace.open(KEYSPACE);
-        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("test_compact_dynamic_columns");
-
-        QueryProcessor.executeInternal(String.format("INSERT INTO \"%s\".test_compact_dynamic_columns (a, b, c) VALUES (0, 'a', 'foo')", KEYSPACE));
-        QueryProcessor.executeInternal(String.format("INSERT INTO \"%s\".test_compact_dynamic_columns (a, b, c) VALUES (0, 'b', 'bar')", KEYSPACE));
-        QueryProcessor.executeInternal(String.format("INSERT INTO \"%s\".test_compact_dynamic_columns (a, b, c) VALUES (0, 'c', 'boo')", KEYSPACE));
-        cfs.forceBlockingFlush();
-        CompactionManager.instance.performScrub(cfs, true, true, 2);
-
-        // Scrub is silent, but it will remove broken records. So reading everything back to make sure nothing to "scrubbed away"
-        UntypedResultSet rs = QueryProcessor.executeInternal(String.format("SELECT * FROM \"%s\".test_compact_dynamic_columns", KEYSPACE));
-        assertEquals(3, rs.size());
-
-        Iterator<UntypedResultSet.Row> iter = rs.iterator();
-        assertEquals("foo", iter.next().getString("c"));
-        assertEquals("bar", iter.next().getString("c"));
-        assertEquals("boo", iter.next().getString("c"));
-    }
 
     @Test /* CASSANDRA-5174 */
     public void testScrubKeysIndex_preserveOrder() throws IOException, ExecutionException, InterruptedException

http://git-wip-us.apache.org/repos/asf/cassandra/blob/07fbd8ee/test/unit/org/apache/cassandra/db/partition/PartitionUpdateTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/partition/PartitionUpdateTest.java b/test/unit/org/apache/cassandra/db/partition/PartitionUpdateTest.java
index fdd34f1..0f83dac 100644
--- a/test/unit/org/apache/cassandra/db/partition/PartitionUpdateTest.java
+++ b/test/unit/org/apache/cassandra/db/partition/PartitionUpdateTest.java
@@ -47,18 +47,4 @@ public class PartitionUpdateTest extends CQLTester
         builder.newRow(1).add("a", 1);
         Assert.assertEquals(2, builder.build().operationCount());
     }
-
-    @Test
-    public void testOperationCountWithCompactTable()
-    {
-        createTable("CREATE TABLE %s (key text PRIMARY KEY, a int) WITH COMPACT STORAGE");
-        TableMetadata cfm = currentTableMetadata();
-
-        PartitionUpdate update = new RowUpdateBuilder(cfm, FBUtilities.timestampMicros(), "key0").add("a", 1)
-                                                                                                 .buildUpdate();
-        Assert.assertEquals(1, update.operationCount());
-
-        update = new RowUpdateBuilder(cfm, FBUtilities.timestampMicros(), "key0").buildUpdate();
-        Assert.assertEquals(0, update.operationCount());
-    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/07fbd8ee/test/unit/org/apache/cassandra/index/SecondaryIndexManagerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/index/SecondaryIndexManagerTest.java b/test/unit/org/apache/cassandra/index/SecondaryIndexManagerTest.java
index 9af7072..ac3e5b2 100644
--- a/test/unit/org/apache/cassandra/index/SecondaryIndexManagerTest.java
+++ b/test/unit/org/apache/cassandra/index/SecondaryIndexManagerTest.java
@@ -553,7 +553,7 @@ public class SecondaryIndexManagerTest extends CQLTester
 
     private void assertMarkedAsBuilt(String indexName) throws Throwable
     {
-        assertRows(execute(builtIndexesQuery), row(KEYSPACE, indexName));
+        assertRows(execute(builtIndexesQuery), row(KEYSPACE, indexName, null));
     }
 
     private void assertNotMarkedAsBuilt() throws Throwable

http://git-wip-us.apache.org/repos/asf/cassandra/blob/07fbd8ee/test/unit/org/apache/cassandra/index/internal/CassandraIndexTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/index/internal/CassandraIndexTest.java b/test/unit/org/apache/cassandra/index/internal/CassandraIndexTest.java
index c0ca2ba..24480f5 100644
--- a/test/unit/org/apache/cassandra/index/internal/CassandraIndexTest.java
+++ b/test/unit/org/apache/cassandra/index/internal/CassandraIndexTest.java
@@ -325,21 +325,6 @@ public class CassandraIndexTest extends CQLTester
     }
 
     @Test
-    public void indexOnRegularColumnWithCompactStorage() throws Throwable
-    {
-        new TestScript().tableDefinition("CREATE TABLE %s (k int, v int, PRIMARY KEY (k)) WITH COMPACT STORAGE;")
-                        .target("v")
-                        .withFirstRow(row(0, 0))
-                        .withSecondRow(row(1,1))
-                        .missingIndexMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE)
-                        .firstQueryExpression("v=0")
-                        .secondQueryExpression("v=1")
-                        .updateExpression("SET v=2")
-                        .postUpdateQueryExpression("v=2")
-                        .run();
-    }
-
-    @Test
     public void indexOnStaticColumn() throws Throwable
     {
         Object[] row1 = row("k0", "c0", "s0");
@@ -512,7 +497,7 @@ public class CassandraIndexTest extends CQLTester
         waitForIndex(KEYSPACE, tableName, indexName);
         // check that there are no other rows in the built indexes table
         assertRows(execute(String.format("SELECT * FROM %s.\"%s\"", SchemaConstants.SYSTEM_KEYSPACE_NAME, SystemKeyspace.BUILT_INDEXES)),
-                   row(KEYSPACE, indexName));
+                   row(KEYSPACE, indexName, null));
 
         // rebuild the index and verify the built status table
         getCurrentColumnFamilyStore().rebuildSecondaryIndex(indexName);
@@ -520,7 +505,7 @@ public class CassandraIndexTest extends CQLTester
 
         // check that there are no other rows in the built indexes table
         assertRows(execute(String.format("SELECT * FROM %s.\"%s\"", SchemaConstants.SYSTEM_KEYSPACE_NAME, SystemKeyspace.BUILT_INDEXES)),
-                   row(KEYSPACE, indexName));
+                   row(KEYSPACE, indexName, null ));
     }
 
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/07fbd8ee/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java b/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java
index 8cbf006..4892184 100644
--- a/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java
@@ -196,19 +196,16 @@ public class LegacySSTableTest
 
     private void streamLegacyTables(String legacyVersion) throws Exception
     {
-        for (int compact = 0; compact <= 1; compact++)
-        {
-            logger.info("Streaming legacy version {}{}", legacyVersion, getCompactNameSuffix(compact));
-            streamLegacyTable("legacy_%s_simple%s", legacyVersion, getCompactNameSuffix(compact));
-            streamLegacyTable("legacy_%s_simple_counter%s", legacyVersion, getCompactNameSuffix(compact));
-            streamLegacyTable("legacy_%s_clust%s", legacyVersion, getCompactNameSuffix(compact));
-            streamLegacyTable("legacy_%s_clust_counter%s", legacyVersion, getCompactNameSuffix(compact));
-        }
+            logger.info("Streaming legacy version {}", legacyVersion);
+            streamLegacyTable("legacy_%s_simple", legacyVersion);
+            streamLegacyTable("legacy_%s_simple_counter", legacyVersion);
+            streamLegacyTable("legacy_%s_clust", legacyVersion);
+            streamLegacyTable("legacy_%s_clust_counter", legacyVersion);
     }
 
-    private void streamLegacyTable(String tablePattern, String legacyVersion, String compactNameSuffix) throws Exception
+    private void streamLegacyTable(String tablePattern, String legacyVersion) throws Exception
     {
-        String table = String.format(tablePattern, legacyVersion, compactNameSuffix);
+        String table = String.format(tablePattern, legacyVersion);
         SSTableReader sstable = SSTableReader.open(getDescriptor(legacyVersion, table));
         IPartitioner p = sstable.getPartitioner();
         List<Range<Token>> ranges = new ArrayList<>();
@@ -224,38 +221,29 @@ public class LegacySSTableTest
 
     private static void truncateLegacyTables(String legacyVersion) throws Exception
     {
-        for (int compact = 0; compact <= 1; compact++)
-        {
-            logger.info("Truncating legacy version {}{}", legacyVersion, getCompactNameSuffix(compact));
-            Keyspace.open("legacy_tables").getColumnFamilyStore(String.format("legacy_%s_simple%s", legacyVersion, getCompactNameSuffix(compact))).truncateBlocking();
-            Keyspace.open("legacy_tables").getColumnFamilyStore(String.format("legacy_%s_simple_counter%s", legacyVersion, getCompactNameSuffix(compact))).truncateBlocking();
-            Keyspace.open("legacy_tables").getColumnFamilyStore(String.format("legacy_%s_clust%s", legacyVersion, getCompactNameSuffix(compact))).truncateBlocking();
-            Keyspace.open("legacy_tables").getColumnFamilyStore(String.format("legacy_%s_clust_counter%s", legacyVersion, getCompactNameSuffix(compact))).truncateBlocking();
-        }
+        logger.info("Truncating legacy version {}", legacyVersion);
+        Keyspace.open("legacy_tables").getColumnFamilyStore(String.format("legacy_%s_simple", legacyVersion)).truncateBlocking();
+        Keyspace.open("legacy_tables").getColumnFamilyStore(String.format("legacy_%s_simple_counter", legacyVersion)).truncateBlocking();
+        Keyspace.open("legacy_tables").getColumnFamilyStore(String.format("legacy_%s_clust", legacyVersion)).truncateBlocking();
+        Keyspace.open("legacy_tables").getColumnFamilyStore(String.format("legacy_%s_clust_counter", legacyVersion)).truncateBlocking();
     }
 
     private static void compactLegacyTables(String legacyVersion) throws Exception
     {
-        for (int compact = 0; compact <= 1; compact++)
-        {
-            logger.info("Compacting legacy version {}{}", legacyVersion, getCompactNameSuffix(compact));
-            Keyspace.open("legacy_tables").getColumnFamilyStore(String.format("legacy_%s_simple%s", legacyVersion, getCompactNameSuffix(compact))).forceMajorCompaction();
-            Keyspace.open("legacy_tables").getColumnFamilyStore(String.format("legacy_%s_simple_counter%s", legacyVersion, getCompactNameSuffix(compact))).forceMajorCompaction();
-            Keyspace.open("legacy_tables").getColumnFamilyStore(String.format("legacy_%s_clust%s", legacyVersion, getCompactNameSuffix(compact))).forceMajorCompaction();
-            Keyspace.open("legacy_tables").getColumnFamilyStore(String.format("legacy_%s_clust_counter%s", legacyVersion, getCompactNameSuffix(compact))).forceMajorCompaction();
-        }
+        logger.info("Compacting legacy version {}", legacyVersion);
+        Keyspace.open("legacy_tables").getColumnFamilyStore(String.format("legacy_%s_simple", legacyVersion)).forceMajorCompaction();
+        Keyspace.open("legacy_tables").getColumnFamilyStore(String.format("legacy_%s_simple_counter", legacyVersion)).forceMajorCompaction();
+        Keyspace.open("legacy_tables").getColumnFamilyStore(String.format("legacy_%s_clust", legacyVersion)).forceMajorCompaction();
+        Keyspace.open("legacy_tables").getColumnFamilyStore(String.format("legacy_%s_clust_counter", legacyVersion)).forceMajorCompaction();
     }
 
     private static void loadLegacyTables(String legacyVersion) throws Exception
     {
-        for (int compact = 0; compact <= 1; compact++)
-        {
-            logger.info("Preparing legacy version {}{}", legacyVersion, getCompactNameSuffix(compact));
-            loadLegacyTable("legacy_%s_simple%s", legacyVersion, getCompactNameSuffix(compact));
-            loadLegacyTable("legacy_%s_simple_counter%s", legacyVersion, getCompactNameSuffix(compact));
-            loadLegacyTable("legacy_%s_clust%s", legacyVersion, getCompactNameSuffix(compact));
-            loadLegacyTable("legacy_%s_clust_counter%s", legacyVersion, getCompactNameSuffix(compact));
-        }
+            logger.info("Preparing legacy version {}", legacyVersion);
+            loadLegacyTable("legacy_%s_simple", legacyVersion);
+            loadLegacyTable("legacy_%s_simple_counter", legacyVersion);
+            loadLegacyTable("legacy_%s_clust", legacyVersion);
+            loadLegacyTable("legacy_%s_clust_counter", legacyVersion);
     }
 
     private static void verifyCache(String legacyVersion, long startCount) throws InterruptedException, java.util.concurrent.ExecutionException
@@ -274,68 +262,65 @@ public class LegacySSTableTest
 
     private static void verifyReads(String legacyVersion)
     {
-        for (int compact = 0; compact <= 1; compact++)
+        for (int ck = 0; ck < 50; ck++)
         {
-            for (int ck = 0; ck < 50; ck++)
+            String ckValue = Integer.toString(ck) + longString;
+            for (int pk = 0; pk < 5; pk++)
             {
-                String ckValue = Integer.toString(ck) + longString;
-                for (int pk = 0; pk < 5; pk++)
+                logger.debug("for pk={} ck={}", pk, ck);
+
+                String pkValue = Integer.toString(pk);
+                UntypedResultSet rs;
+                if (ck == 0)
                 {
-                    logger.debug("for pk={} ck={}", pk, ck);
-
-                    String pkValue = Integer.toString(pk);
-                    UntypedResultSet rs;
-                    if (ck == 0)
-                    {
-                        readSimpleTable(legacyVersion, getCompactNameSuffix(compact),  pkValue);
-                        readSimpleCounterTable(legacyVersion, getCompactNameSuffix(compact), pkValue);
-                    }
-
-                    readClusteringTable(legacyVersion, getCompactNameSuffix(compact), ck, ckValue, pkValue);
-                    readClusteringCounterTable(legacyVersion, getCompactNameSuffix(compact), ckValue, pkValue);
+                    readSimpleTable(legacyVersion, pkValue);
+                    readSimpleCounterTable(legacyVersion, pkValue);
                 }
+
+                readClusteringTable(legacyVersion, ck, ckValue, pkValue);
+                readClusteringCounterTable(legacyVersion, ckValue, pkValue);
             }
         }
     }
 
-    private static void readClusteringCounterTable(String legacyVersion, String compactSuffix, String ckValue, String pkValue)
+    private static void readClusteringCounterTable(String legacyVersion, String ckValue, String pkValue)
     {
-        logger.debug("Read legacy_{}_clust_counter{}", legacyVersion, compactSuffix);
+        logger.debug("Read legacy_{}_clust_counter", legacyVersion);
         UntypedResultSet rs;
-        rs = QueryProcessor.executeInternal(String.format("SELECT val FROM legacy_tables.legacy_%s_clust_counter%s WHERE pk=? AND ck=?", legacyVersion, compactSuffix), pkValue, ckValue);
+        rs = QueryProcessor.executeInternal(String.format("SELECT val FROM legacy_tables.legacy_%s_clust_counter WHERE pk=? AND ck=?", legacyVersion), pkValue, ckValue);
         Assert.assertNotNull(rs);
         Assert.assertEquals(1, rs.size());
         Assert.assertEquals(1L, rs.one().getLong("val"));
     }
 
-    private static void readClusteringTable(String legacyVersion, String compactSuffix, int ck, String ckValue, String pkValue)
+    private static void readClusteringTable(String legacyVersion, int ck, String ckValue, String pkValue)
     {
-        logger.debug("Read legacy_{}_clust{}", legacyVersion, compactSuffix);
+        logger.debug("Read legacy_{}_clust", legacyVersion);
         UntypedResultSet rs;
-        rs = QueryProcessor.executeInternal(String.format("SELECT val FROM legacy_tables.legacy_%s_clust%s WHERE pk=? AND ck=?", legacyVersion, compactSuffix), pkValue, ckValue);
+        rs = QueryProcessor.executeInternal(String.format("SELECT val FROM legacy_tables.legacy_%s_clust WHERE pk=? AND ck=?", legacyVersion), pkValue, ckValue);
         assertLegacyClustRows(1, rs);
 
         String ckValue2 = Integer.toString(ck < 10 ? 40 : ck - 1) + longString;
         String ckValue3 = Integer.toString(ck > 39 ? 10 : ck + 1) + longString;
-        rs = QueryProcessor.executeInternal(String.format("SELECT val FROM legacy_tables.legacy_%s_clust%s WHERE pk=? AND ck IN (?, ?, ?)", legacyVersion, compactSuffix), pkValue, ckValue, ckValue2, ckValue3);
+        rs = QueryProcessor.executeInternal(String.format("SELECT val FROM legacy_tables.legacy_%s_clust WHERE pk=? AND ck IN (?, ?, ?)", legacyVersion), pkValue, ckValue, ckValue2, ckValue3);
         assertLegacyClustRows(3, rs);
     }
 
-    private static void readSimpleCounterTable(String legacyVersion, String compactSuffix, String pkValue)
+    private static void readSimpleCounterTable(String legacyVersion, String pkValue)
     {
-        logger.debug("Read legacy_{}_simple_counter{}", legacyVersion, compactSuffix);
+        logger.debug("Read legacy_{}_simple_counter", legacyVersion);
         UntypedResultSet rs;
-        rs = QueryProcessor.executeInternal(String.format("SELECT val FROM legacy_tables.legacy_%s_simple_counter%s WHERE pk=?", legacyVersion, compactSuffix), pkValue);
+        rs = QueryProcessor.executeInternal(String.format("SELECT val FROM legacy_tables.legacy_%s_simple_counter WHERE pk=?", legacyVersion), pkValue);
         Assert.assertNotNull(rs);
         Assert.assertEquals(1, rs.size());
         Assert.assertEquals(1L, rs.one().getLong("val"));
     }
 
-    private static void readSimpleTable(String legacyVersion, String compactSuffix, String pkValue)
+    private static void readSimpleTable(String legacyVersion, String pkValue)
     {
-        logger.debug("Read simple: legacy_{}_simple{}", legacyVersion, compactSuffix);
+        logger.debug("Read simple: legacy_{}_simple", legacyVersion);
         UntypedResultSet rs;
-        rs = QueryProcessor.executeInternal(String.format("SELECT val FROM legacy_tables.legacy_%s_simple%s WHERE pk=?", legacyVersion, compactSuffix), pkValue);
+        rs = QueryProcessor.executeInternal(String.format("SELECT val FROM legacy_tables.legacy_%s_simple WHERE pk=?", legacyVersion), pkValue);
         Assert.assertNotNull(rs);
         Assert.assertEquals(1, rs.size());
         Assert.assertEquals("foo bar baz", rs.one().getString("val"));
@@ -348,31 +333,18 @@ public class LegacySSTableTest
 
     private static void createTables(String legacyVersion)
     {
-        for (int i=0; i<=1; i++)
-        {
-            String compactSuffix = getCompactNameSuffix(i);
-            String tableSuffix = i == 0? "" : " WITH COMPACT STORAGE";
-            QueryProcessor.executeInternal(String.format("CREATE TABLE legacy_tables.legacy_%s_simple%s (pk text PRIMARY KEY, val text)%s", legacyVersion, compactSuffix, tableSuffix));
-            QueryProcessor.executeInternal(String.format("CREATE TABLE legacy_tables.legacy_%s_simple_counter%s (pk text PRIMARY KEY, val counter)%s", legacyVersion, compactSuffix, tableSuffix));
-            QueryProcessor.executeInternal(String.format("CREATE TABLE legacy_tables.legacy_%s_clust%s (pk text, ck text, val text, PRIMARY KEY (pk, ck))%s", legacyVersion, compactSuffix, tableSuffix));
-            QueryProcessor.executeInternal(String.format("CREATE TABLE legacy_tables.legacy_%s_clust_counter%s (pk text, ck text, val counter, PRIMARY KEY (pk, ck))%s", legacyVersion, compactSuffix, tableSuffix));
-        }
-    }
-
-    private static String getCompactNameSuffix(int i)
-    {
-        return i == 0? "" : "_compact";
+        QueryProcessor.executeInternal(String.format("CREATE TABLE legacy_tables.legacy_%s_simple (pk text PRIMARY KEY, val text)", legacyVersion));
+        QueryProcessor.executeInternal(String.format("CREATE TABLE legacy_tables.legacy_%s_simple_counter (pk text PRIMARY KEY, val counter)", legacyVersion));
+        QueryProcessor.executeInternal(String.format("CREATE TABLE legacy_tables.legacy_%s_clust (pk text, ck text, val text, PRIMARY KEY (pk, ck))", legacyVersion));
+        QueryProcessor.executeInternal(String.format("CREATE TABLE legacy_tables.legacy_%s_clust_counter (pk text, ck text, val counter, PRIMARY KEY (pk, ck))", legacyVersion));
     }
 
     private static void truncateTables(String legacyVersion)
     {
-        for (int compact = 0; compact <= 1; compact++)
-        {
-            QueryProcessor.executeInternal(String.format("TRUNCATE legacy_tables.legacy_%s_simple%s", legacyVersion, getCompactNameSuffix(compact)));
-            QueryProcessor.executeInternal(String.format("TRUNCATE legacy_tables.legacy_%s_simple_counter%s", legacyVersion, getCompactNameSuffix(compact)));
-            QueryProcessor.executeInternal(String.format("TRUNCATE legacy_tables.legacy_%s_clust%s", legacyVersion, getCompactNameSuffix(compact)));
-            QueryProcessor.executeInternal(String.format("TRUNCATE legacy_tables.legacy_%s_clust_counter%s", legacyVersion, getCompactNameSuffix(compact)));
-        }
+        QueryProcessor.executeInternal(String.format("TRUNCATE legacy_tables.legacy_%s_simple", legacyVersion));
+        QueryProcessor.executeInternal(String.format("TRUNCATE legacy_tables.legacy_%s_simple_counter", legacyVersion));
+        QueryProcessor.executeInternal(String.format("TRUNCATE legacy_tables.legacy_%s_clust", legacyVersion));
+        QueryProcessor.executeInternal(String.format("TRUNCATE legacy_tables.legacy_%s_clust_counter", legacyVersion));
         CacheService.instance.invalidateCounterCache();
         CacheService.instance.invalidateKeyCache();
     }
@@ -390,9 +362,9 @@ public class LegacySSTableTest
         }
     }
 
-    private static void loadLegacyTable(String tablePattern, String legacyVersion, String compactSuffix) throws IOException
+    private static void loadLegacyTable(String tablePattern, String legacyVersion) throws IOException
     {
-        String table = String.format(tablePattern, legacyVersion, compactSuffix);
+        String table = String.format(tablePattern, legacyVersion);
 
         logger.info("Loading legacy table {}", table);
 
@@ -427,28 +399,24 @@ public class LegacySSTableTest
         }
         String randomString = sb.toString();
 
-        for (int compact = 0; compact <= 1; compact++)
+        for (int pk = 0; pk < 5; pk++)
         {
-            for (int pk = 0; pk < 5; pk++)
-            {
-                String valPk = Integer.toString(pk);
-                QueryProcessor.executeInternal(String.format("INSERT INTO legacy_tables.legacy_%s_simple%s (pk, val) VALUES ('%s', '%s')",
-                                                             BigFormat.latestVersion, getCompactNameSuffix(compact), valPk, "foo bar baz"));
+            String valPk = Integer.toString(pk);
+            QueryProcessor.executeInternal(String.format("INSERT INTO legacy_tables.legacy_%s_simple (pk, val) VALUES ('%s', '%s')",
+                                                         BigFormat.latestVersion, valPk, "foo bar baz"));
 
-                QueryProcessor.executeInternal(String.format("UPDATE legacy_tables.legacy_%s_simple_counter%s SET val = val + 1 WHERE pk = '%s'",
-                                                             BigFormat.latestVersion, getCompactNameSuffix(compact), valPk));
+            QueryProcessor.executeInternal(String.format("UPDATE legacy_tables.legacy_%s_simple_counter SET val = val + 1 WHERE pk = '%s'",
+                                                         BigFormat.latestVersion, valPk));
 
-                for (int ck = 0; ck < 50; ck++)
-                {
-                    String valCk = Integer.toString(ck);
-
-                    QueryProcessor.executeInternal(String.format("INSERT INTO legacy_tables.legacy_%s_clust%s (pk, ck, val) VALUES ('%s', '%s', '%s')",
-                                                                 BigFormat.latestVersion, getCompactNameSuffix(compact), valPk, valCk + longString, randomString));
+            for (int ck = 0; ck < 50; ck++)
+            {
+                String valCk = Integer.toString(ck);
 
-                    QueryProcessor.executeInternal(String.format("UPDATE legacy_tables.legacy_%s_clust_counter%s SET val = val + 1 WHERE pk = '%s' AND ck='%s'",
-                                                                 BigFormat.latestVersion, getCompactNameSuffix(compact), valPk, valCk + longString));
+                QueryProcessor.executeInternal(String.format("INSERT INTO legacy_tables.legacy_%s_clust (pk, ck, val) VALUES ('%s', '%s', '%s')",
+                                                             BigFormat.latestVersion, valPk, valCk + longString, randomString));
 
-                }
+                QueryProcessor.executeInternal(String.format("UPDATE legacy_tables.legacy_%s_clust_counter SET val = val + 1 WHERE pk = '%s' AND ck='%s'",
+                                                             BigFormat.latestVersion, valPk, valCk + longString));
             }
         }
 
@@ -456,13 +424,10 @@ public class LegacySSTableTest
 
         File ksDir = new File(LEGACY_SSTABLE_ROOT, String.format("%s/legacy_tables", BigFormat.latestVersion));
         ksDir.mkdirs();
-        for (int compact = 0; compact <= 1; compact++)
-        {
-            copySstablesFromTestData(String.format("legacy_%s_simple%s", BigFormat.latestVersion, getCompactNameSuffix(compact)), ksDir);
-            copySstablesFromTestData(String.format("legacy_%s_simple_counter%s", BigFormat.latestVersion, getCompactNameSuffix(compact)), ksDir);
-            copySstablesFromTestData(String.format("legacy_%s_clust%s", BigFormat.latestVersion, getCompactNameSuffix(compact)), ksDir);
-            copySstablesFromTestData(String.format("legacy_%s_clust_counter%s", BigFormat.latestVersion, getCompactNameSuffix(compact)), ksDir);
-        }
+        copySstablesFromTestData(String.format("legacy_%s_simple", BigFormat.latestVersion), ksDir);
+        copySstablesFromTestData(String.format("legacy_%s_simple_counter", BigFormat.latestVersion), ksDir);
+        copySstablesFromTestData(String.format("legacy_%s_clust", BigFormat.latestVersion), ksDir);
+        copySstablesFromTestData(String.format("legacy_%s_clust_counter", BigFormat.latestVersion), ksDir);
     }
 
     private void copySstablesFromTestData(String table, File ksDir) throws IOException

http://git-wip-us.apache.org/repos/asf/cassandra/blob/07fbd8ee/tools/stress/src/org/apache/cassandra/stress/settings/SettingsSchema.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/settings/SettingsSchema.java b/tools/stress/src/org/apache/cassandra/stress/settings/SettingsSchema.java
index 26d159e..4c67c4f 100644
--- a/tools/stress/src/org/apache/cassandra/stress/settings/SettingsSchema.java
+++ b/tools/stress/src/org/apache/cassandra/stress/settings/SettingsSchema.java
@@ -138,7 +138,7 @@ public class SettingsSchema implements Serializable
         }
 
         //Compression
-        b.append(") WITH COMPACT STORAGE AND compression = {");
+        b.append(") WITH compression = {");
         if (compression != null)
             b.append("'sstable_compression' : '").append(compression).append("'");
 
@@ -179,7 +179,7 @@ public class SettingsSchema implements Serializable
         }
 
         //Compression
-        b.append(") WITH COMPACT STORAGE AND compression = {");
+        b.append(") WITH compression = {");
         if (compression != null)
             b.append("'sstable_compression' : '").append(compression).append("'");
 


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


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

Posted by if...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/07fbd8ee/test/unit/org/apache/cassandra/cql3/validation/operations/SelectOrderByTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/SelectOrderByTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/SelectOrderByTest.java
index a1d5743..fd44c50 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/SelectOrderByTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/SelectOrderByTest.java
@@ -29,72 +29,66 @@ public class SelectOrderByTest extends CQLTester
     @Test
     public void testNormalSelectionOrderSingleClustering() throws Throwable
     {
-        for (String descOption : new String[]{"", " WITH CLUSTERING ORDER BY (b DESC)"})
-        {
-            createTable("CREATE TABLE %s (a int, b int, c int, PRIMARY KEY (a, b))" + descOption);
-            execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, 0, 0);
-            execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, 1, 1);
-            execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, 2, 2);
+        createTable("CREATE TABLE %s (a int, b int, c int, PRIMARY KEY (a, b))");
+        execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, 0, 0);
+        execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, 1, 1);
+        execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, 2, 2);
 
-            beforeAndAfterFlush(() -> {
-                assertRows(execute("SELECT * FROM %s WHERE a=? ORDER BY b ASC", 0),
-                           row(0, 0, 0),
-                           row(0, 1, 1),
-                           row(0, 2, 2)
-                        );
-
-                assertRows(execute("SELECT * FROM %s WHERE a=? ORDER BY b DESC", 0),
-                           row(0, 2, 2),
-                           row(0, 1, 1),
-                           row(0, 0, 0)
-                        );
-
-                // order by the only column in the selection
-                assertRows(execute("SELECT b FROM %s WHERE a=? ORDER BY b ASC", 0),
-                           row(0), row(1), row(2));
-
-                assertRows(execute("SELECT b FROM %s WHERE a=? ORDER BY b DESC", 0),
-                           row(2), row(1), row(0));
-
-                // order by a column not in the selection
-                assertRows(execute("SELECT c FROM %s WHERE a=? ORDER BY b ASC", 0),
-                           row(0), row(1), row(2));
-
-                assertRows(execute("SELECT c FROM %s WHERE a=? ORDER BY b DESC", 0),
-                           row(2), row(1), row(0));
-            });
-        }
+        beforeAndAfterFlush(() -> {
+            assertRows(execute("SELECT * FROM %s WHERE a=? ORDER BY b ASC", 0),
+                       row(0, 0, 0),
+                       row(0, 1, 1),
+                       row(0, 2, 2)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a=? ORDER BY b DESC", 0),
+                       row(0, 2, 2),
+                       row(0, 1, 1),
+                       row(0, 0, 0)
+            );
+
+            // order by the only column in the selection
+            assertRows(execute("SELECT b FROM %s WHERE a=? ORDER BY b ASC", 0),
+                       row(0), row(1), row(2));
+
+            assertRows(execute("SELECT b FROM %s WHERE a=? ORDER BY b DESC", 0),
+                       row(2), row(1), row(0));
+
+            // order by a column not in the selection
+            assertRows(execute("SELECT c FROM %s WHERE a=? ORDER BY b ASC", 0),
+                       row(0), row(1), row(2));
+
+            assertRows(execute("SELECT c FROM %s WHERE a=? ORDER BY b DESC", 0),
+                       row(2), row(1), row(0));
+        });
     }
 
     @Test
     public void testFunctionSelectionOrderSingleClustering() throws Throwable
     {
-        for (String descOption : new String[]{"", " WITH CLUSTERING ORDER BY (b DESC)"})
-        {
-            createTable("CREATE TABLE %s (a int, b int, c int, PRIMARY KEY (a, b))" + descOption);
-            execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, 0, 0);
-            execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, 1, 1);
-            execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, 2, 2);
+        createTable("CREATE TABLE %s (a int, b int, c int, PRIMARY KEY (a, b))");
+        execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, 0, 0);
+        execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, 1, 1);
+        execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, 2, 2);
 
-            beforeAndAfterFlush(() -> {
-                // order by the only column in the selection
-                assertRows(execute("SELECT blobAsInt(intAsBlob(b)) FROM %s WHERE a=? ORDER BY b ASC", 0),
-                           row(0), row(1), row(2));
+        beforeAndAfterFlush(() -> {
+            // order by the only column in the selection
+            assertRows(execute("SELECT blobAsInt(intAsBlob(b)) FROM %s WHERE a=? ORDER BY b ASC", 0),
+                       row(0), row(1), row(2));
 
-                assertRows(execute("SELECT blobAsInt(intAsBlob(b)) FROM %s WHERE a=? ORDER BY b DESC", 0),
-                           row(2), row(1), row(0));
+            assertRows(execute("SELECT blobAsInt(intAsBlob(b)) FROM %s WHERE a=? ORDER BY b DESC", 0),
+                       row(2), row(1), row(0));
 
-                // order by a column not in the selection
-                assertRows(execute("SELECT blobAsInt(intAsBlob(c)) FROM %s WHERE a=? ORDER BY b ASC", 0),
-                           row(0), row(1), row(2));
+            // order by a column not in the selection
+            assertRows(execute("SELECT blobAsInt(intAsBlob(c)) FROM %s WHERE a=? ORDER BY b ASC", 0),
+                       row(0), row(1), row(2));
 
-                assertRows(execute("SELECT blobAsInt(intAsBlob(c)) FROM %s WHERE a=? ORDER BY b DESC", 0),
-                           row(2), row(1), row(0));
+            assertRows(execute("SELECT blobAsInt(intAsBlob(c)) FROM %s WHERE a=? ORDER BY b DESC", 0),
+                       row(2), row(1), row(0));
 
-                assertInvalid("SELECT * FROM %s WHERE a=? ORDER BY c ASC", 0);
-                assertInvalid("SELECT * FROM %s WHERE a=? ORDER BY c DESC", 0);
-            });
-        }
+            assertInvalid("SELECT * FROM %s WHERE a=? ORDER BY c ASC", 0);
+            assertInvalid("SELECT * FROM %s WHERE a=? ORDER BY c DESC", 0);
+        });
     }
 
     @Test
@@ -102,26 +96,23 @@ public class SelectOrderByTest extends CQLTester
     {
         String type = createType("CREATE TYPE %s (a int)");
 
-        for (String descOption : new String[]{"", " WITH CLUSTERING ORDER BY (b DESC)"})
-        {
-            createTable("CREATE TABLE %s (a int, b int, c frozen<" + type + "   >, PRIMARY KEY (a, b))" + descOption);
-            execute("INSERT INTO %s (a, b, c) VALUES (?, ?, {a: ?})", 0, 0, 0);
-            execute("INSERT INTO %s (a, b, c) VALUES (?, ?, {a: ?})", 0, 1, 1);
-            execute("INSERT INTO %s (a, b, c) VALUES (?, ?, {a: ?})", 0, 2, 2);
+        createTable("CREATE TABLE %s (a int, b int, c frozen<" + type + "   >, PRIMARY KEY (a, b))");
+        execute("INSERT INTO %s (a, b, c) VALUES (?, ?, {a: ?})", 0, 0, 0);
+        execute("INSERT INTO %s (a, b, c) VALUES (?, ?, {a: ?})", 0, 1, 1);
+        execute("INSERT INTO %s (a, b, c) VALUES (?, ?, {a: ?})", 0, 2, 2);
 
-            beforeAndAfterFlush(() -> {
-                // order by a column not in the selection
-                assertRows(execute("SELECT c.a FROM %s WHERE a=? ORDER BY b ASC", 0),
-                           row(0), row(1), row(2));
+        beforeAndAfterFlush(() -> {
+            // order by a column not in the selection
+            assertRows(execute("SELECT c.a FROM %s WHERE a=? ORDER BY b ASC", 0),
+                       row(0), row(1), row(2));
 
-                assertRows(execute("SELECT c.a FROM %s WHERE a=? ORDER BY b DESC", 0),
-                           row(2), row(1), row(0));
+            assertRows(execute("SELECT c.a FROM %s WHERE a=? ORDER BY b DESC", 0),
+                       row(2), row(1), row(0));
 
-                assertRows(execute("SELECT blobAsInt(intAsBlob(c.a)) FROM %s WHERE a=? ORDER BY b DESC", 0),
-                           row(2), row(1), row(0));
-            });
-            dropTable("DROP TABLE %s");
-        }
+            assertRows(execute("SELECT blobAsInt(intAsBlob(c.a)) FROM %s WHERE a=? ORDER BY b DESC", 0),
+                       row(2), row(1), row(0));
+        });
+        dropTable("DROP TABLE %s");
     }
 
     @Test
@@ -245,77 +236,6 @@ public class SelectOrderByTest extends CQLTester
     }
 
     /**
-     * Check ORDER BY support in SELECT statement
-     * migrated from cql_tests.py:TestCQL.order_by_test()
-     */
-    @Test
-    public void testSimpleOrderBy() throws Throwable
-    {
-        createTable("CREATE TABLE %s (k int, c int, v int, PRIMARY KEY (k, c)) WITH COMPACT STORAGE");
-
-        for (int i = 0; i < 10; i++)
-            execute("INSERT INTO %s (k, c, v) VALUES (0, ?, ?)", i, i);
-
-        beforeAndAfterFlush(() -> {
-            assertRows(execute("SELECT v FROM %s WHERE k = 0 ORDER BY c DESC"),
-                       row(9), row(8), row(7), row(6), row(5), row(4), row(3), row(2), row(1), row(0));
-        });
-
-        createTable("CREATE TABLE %s (k int, c1 int, c2 int, v int, PRIMARY KEY (k, c1, c2)) WITH COMPACT STORAGE");
-
-        for (int i = 0; i < 4; i++)
-            for (int j = 0; j < 2; j++)
-                execute("INSERT INTO %s (k, c1, c2, v) VALUES (0, ?, ?, ?)", i, j, i * 2 + j);
-
-        beforeAndAfterFlush(() -> {
-            assertInvalid("SELECT v FROM %s WHERE k = 0 ORDER BY c DESC");
-            assertInvalid("SELECT v FROM %s WHERE k = 0 ORDER BY c2 DESC");
-            assertInvalid("SELECT v FROM %s WHERE k = 0 ORDER BY k DESC");
-
-            assertRows(execute("SELECT v FROM %s WHERE k = 0 ORDER BY c1 DESC"),
-                       row(7), row(6), row(5), row(4), row(3), row(2), row(1), row(0));
-
-            assertRows(execute("SELECT v FROM %s WHERE k = 0 ORDER BY c1"),
-                       row(0), row(1), row(2), row(3), row(4), row(5), row(6), row(7));
-        });
-    }
-
-    /**
-     * More ORDER BY checks (#4160)
-     * migrated from cql_tests.py:TestCQL.more_order_by_test()
-     */
-    @Test
-    public void testMoreOrderBy() throws Throwable
-    {
-        createTable("CREATE TABLE %s (row text, number int, string text, PRIMARY KEY(row, number)) WITH COMPACT STORAGE ");
-
-        execute("INSERT INTO %s (row, number, string) VALUES ('row', 1, 'one')");
-        execute("INSERT INTO %s (row, number, string) VALUES ('row', 2, 'two')");
-        execute("INSERT INTO %s (row, number, string) VALUES ('row', 3, 'three')");
-        execute("INSERT INTO %s (row, number, string) VALUES ('row', 4, 'four')");
-
-        beforeAndAfterFlush(() -> {
-            assertRows(execute("SELECT number FROM %s WHERE row='row' AND number < 3 ORDER BY number ASC"),
-                       row(1), row(2));
-
-            assertRows(execute("SELECT number FROM %s WHERE row='row' AND number >= 3 ORDER BY number ASC"),
-                       row(3), row(4));
-
-            assertRows(execute("SELECT number FROM %s WHERE row='row' AND number < 3 ORDER BY number DESC"),
-                       row(2), row(1));
-
-            assertRows(execute("SELECT number FROM %s WHERE row='row' AND number >= 3 ORDER BY number DESC"),
-                       row(4), row(3));
-
-            assertRows(execute("SELECT number FROM %s WHERE row='row' AND number > 3 ORDER BY number DESC"),
-                       row(4));
-
-            assertRows(execute("SELECT number FROM %s WHERE row='row' AND number <= 3 ORDER BY number DESC"),
-                       row(3), row(2), row(1));
-        });
-    }
-
-    /**
      * Check we don't allow order by on row key (#4246)
      * migrated from cql_tests.py:TestCQL.order_by_validation_test()
      */

http://git-wip-us.apache.org/repos/asf/cassandra/blob/07fbd8ee/test/unit/org/apache/cassandra/cql3/validation/operations/SelectOrderedPartitionerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/SelectOrderedPartitionerTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/SelectOrderedPartitionerTest.java
index 53d2a9b..a14a2a4 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/SelectOrderedPartitionerTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/SelectOrderedPartitionerTest.java
@@ -394,19 +394,6 @@ public class SelectOrderedPartitionerTest extends CQLTester
                    row(0, 1),
                    row(1, 0),
                    row(1, 1));
-
-        // Check for dense tables too
-        createTable(" CREATE TABLE %s (k int, c int, PRIMARY KEY (k, c)) WITH COMPACT STORAGE");
-
-        for (int k = 0; k < 2; k++)
-            for (int c = 0; c < 2; c++)
-                execute("INSERT INTO %s (k, c) VALUES (?, ?)", k, c);
-
-        assertRows(execute("SELECT * FROM %s"),
-                   row(0, 0),
-                   row(0, 1),
-                   row(1, 0),
-                   row(1, 1));
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/cassandra/blob/07fbd8ee/test/unit/org/apache/cassandra/cql3/validation/operations/SelectSingleColumnRelationTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/SelectSingleColumnRelationTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/SelectSingleColumnRelationTest.java
index 7e5afda..09fd464 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/SelectSingleColumnRelationTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/SelectSingleColumnRelationTest.java
@@ -77,23 +77,6 @@ public class SelectSingleColumnRelationTest extends CQLTester
         execute("insert into %s (a, b, c, d) values (?, ?, ?, ?)", "first", 3, 7, 3);
         execute("insert into %s (a, b, c, d) values (?, ?, ?, ?)", "second", 4, 8, 4);
 
-        testSelectQueriesWithClusteringColumnRelations();
-    }
-
-    @Test
-    public void testClusteringColumnRelationsWithCompactStorage() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a text, b int, c int, d int, primary key(a, b, c)) WITH COMPACT STORAGE;");
-        execute("insert into %s (a, b, c, d) values (?, ?, ?, ?)", "first", 1, 5, 1);
-        execute("insert into %s (a, b, c, d) values (?, ?, ?, ?)", "first", 2, 6, 2);
-        execute("insert into %s (a, b, c, d) values (?, ?, ?, ?)", "first", 3, 7, 3);
-        execute("insert into %s (a, b, c, d) values (?, ?, ?, ?)", "second", 4, 8, 4);
-
-        testSelectQueriesWithClusteringColumnRelations();
-    }
-
-    private void testSelectQueriesWithClusteringColumnRelations() throws Throwable
-    {
         assertRows(execute("select * from %s where a in (?, ?)", "first", "second"),
                    row("first", 1, 5, 1),
                    row("first", 2, 6, 2),
@@ -424,36 +407,30 @@ public class SelectSingleColumnRelationTest extends CQLTester
     @Test
     public void testEmptyIN() throws Throwable
     {
-        for (String compactOption : new String[] { "", " WITH COMPACT STORAGE" })
-        {
-            createTable("CREATE TABLE %s (k1 int, k2 int, v int, PRIMARY KEY (k1, k2))" + compactOption);
+        createTable("CREATE TABLE %s (k1 int, k2 int, v int, PRIMARY KEY (k1, k2))");
 
-            for (int i = 0; i <= 2; i++)
-                for (int j = 0; j <= 2; j++)
-                    execute("INSERT INTO %s (k1, k2, v) VALUES (?, ?, ?)", i, j, i + j);
+        for (int i = 0; i <= 2; i++)
+            for (int j = 0; j <= 2; j++)
+                execute("INSERT INTO %s (k1, k2, v) VALUES (?, ?, ?)", i, j, i + j);
 
-            assertEmpty(execute("SELECT v FROM %s WHERE k1 IN ()"));
-            assertEmpty(execute("SELECT v FROM %s WHERE k1 = 0 AND k2 IN ()"));
-        }
+        assertEmpty(execute("SELECT v FROM %s WHERE k1 IN ()"));
+        assertEmpty(execute("SELECT v FROM %s WHERE k1 = 0 AND k2 IN ()"));
     }
 
     @Test
     public void testINWithDuplicateValue() throws Throwable
     {
-        for (String compactOption : new String[] { "", " WITH COMPACT STORAGE" })
-        {
-            createTable("CREATE TABLE %s (k1 int, k2 int, v int, PRIMARY KEY (k1, k2))" + compactOption);
-            execute("INSERT INTO %s (k1,  k2, v) VALUES (?, ?, ?)", 1, 1, 1);
+        createTable("CREATE TABLE %s (k1 int, k2 int, v int, PRIMARY KEY (k1, k2))");
+        execute("INSERT INTO %s (k1,  k2, v) VALUES (?, ?, ?)", 1, 1, 1);
 
-            assertRows(execute("SELECT * FROM %s WHERE k1 IN (?, ?)", 1, 1),
-                       row(1, 1, 1));
+        assertRows(execute("SELECT * FROM %s WHERE k1 IN (?, ?)", 1, 1),
+                   row(1, 1, 1));
 
-            assertRows(execute("SELECT * FROM %s WHERE k1 IN (?, ?) AND k2 IN (?, ?)", 1, 1, 1, 1),
-                       row(1, 1, 1));
+        assertRows(execute("SELECT * FROM %s WHERE k1 IN (?, ?) AND k2 IN (?, ?)", 1, 1, 1, 1),
+                   row(1, 1, 1));
 
-            assertRows(execute("SELECT * FROM %s WHERE k1 = ? AND k2 IN (?, ?)", 1, 1, 1),
-                       row(1, 1, 1));
-        }
+        assertRows(execute("SELECT * FROM %s WHERE k1 = ? AND k2 IN (?, ?)", 1, 1, 1),
+                   row(1, 1, 1));
     }
 
     @Test


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