You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by al...@apache.org on 2018/07/17 15:40:23 UTC

[01/13] cassandra git commit: Make all DDL statements idempotent and not dependent on global state

Repository: cassandra
Updated Branches:
  refs/heads/trunk d3a994b10 -> 207c80c1f


http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/test/unit/org/apache/cassandra/index/CustomIndexTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/index/CustomIndexTest.java b/test/unit/org/apache/cassandra/index/CustomIndexTest.java
index 3036b1a..2bad5f3 100644
--- a/test/unit/org/apache/cassandra/index/CustomIndexTest.java
+++ b/test/unit/org/apache/cassandra/index/CustomIndexTest.java
@@ -37,7 +37,7 @@ import org.apache.cassandra.cql3.CQLTester;
 import org.apache.cassandra.cql3.ColumnIdentifier;
 import org.apache.cassandra.cql3.restrictions.IndexRestrictions;
 import org.apache.cassandra.cql3.restrictions.StatementRestrictions;
-import org.apache.cassandra.cql3.statements.IndexTarget;
+import org.apache.cassandra.cql3.statements.schema.IndexTarget;
 import org.apache.cassandra.cql3.statements.ModificationStatement;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.marshal.AbstractType;
@@ -54,7 +54,7 @@ import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.concurrent.OpOrder;
 
-import static org.apache.cassandra.cql3.statements.IndexTarget.CUSTOM_INDEX_OPTION_NAME;
+import static org.apache.cassandra.cql3.statements.schema.IndexTarget.CUSTOM_INDEX_OPTION_NAME;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
@@ -163,11 +163,11 @@ public class CustomIndexTest extends CQLTester
     {
         createTable("CREATE TABLE %s(k int, c int, v1 int, v2 int, PRIMARY KEY (k,c))");
 
-        assertInvalidMessage("Duplicate column v1 in index target list",
+        assertInvalidMessage("Duplicate column 'v1' in index target list",
                              String.format("CREATE CUSTOM INDEX ON %%s(v1, v1) USING '%s'",
                                            StubIndex.class.getName()));
 
-        assertInvalidMessage("Duplicate column v1 in index target list",
+        assertInvalidMessage("Duplicate column 'v1' in index target list",
                              String.format("CREATE CUSTOM INDEX ON %%s(v1, v1, c, c) USING '%s'",
                                            StubIndex.class.getName()));
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/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 24480f5..65c2417 100644
--- a/test/unit/org/apache/cassandra/index/internal/CassandraIndexTest.java
+++ b/test/unit/org/apache/cassandra/index/internal/CassandraIndexTest.java
@@ -510,10 +510,10 @@ public class CassandraIndexTest extends CQLTester
 
 
     // this is slightly annoying, but we cannot read rows from the methods in Util as
-    // ReadCommand#executeInternal uses metadata retrieved via the tableId, which the index
+    // ReadCommand#executeLocally uses metadata retrieved via the tableId, which the index
     // CFS inherits from the base CFS. This has the 'wrong' partitioner (the index table
     // uses LocalPartition, the base table a real one, so we cannot read from the index
-    // table with executeInternal
+    // table with executeLocally
     private void assertIndexRowTtl(ColumnFamilyStore indexCfs, int indexedValue, int ttl) throws Throwable
     {
         DecoratedKey indexKey = indexCfs.decorateKey(ByteBufferUtil.bytes(indexedValue));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/test/unit/org/apache/cassandra/index/internal/CustomCassandraIndex.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/index/internal/CustomCassandraIndex.java b/test/unit/org/apache/cassandra/index/internal/CustomCassandraIndex.java
index f007c09..9a806c2 100644
--- a/test/unit/org/apache/cassandra/index/internal/CustomCassandraIndex.java
+++ b/test/unit/org/apache/cassandra/index/internal/CustomCassandraIndex.java
@@ -36,7 +36,7 @@ import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.schema.TableMetadataRef;
 import org.apache.cassandra.cql3.Operator;
-import org.apache.cassandra.cql3.statements.IndexTarget;
+import org.apache.cassandra.cql3.statements.schema.IndexTarget;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.compaction.CompactionManager;
 import org.apache.cassandra.db.filter.RowFilter;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/test/unit/org/apache/cassandra/index/sasi/SASIIndexTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/index/sasi/SASIIndexTest.java b/test/unit/org/apache/cassandra/index/sasi/SASIIndexTest.java
index c617764..90a59dd 100644
--- a/test/unit/org/apache/cassandra/index/sasi/SASIIndexTest.java
+++ b/test/unit/org/apache/cassandra/index/sasi/SASIIndexTest.java
@@ -44,7 +44,7 @@ import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.cql3.Term;
-import org.apache.cassandra.cql3.statements.IndexTarget;
+import org.apache.cassandra.cql3.statements.schema.IndexTarget;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.filter.ColumnFilter;
 import org.apache.cassandra.db.filter.DataLimits;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/test/unit/org/apache/cassandra/repair/StreamingRepairTaskTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/repair/StreamingRepairTaskTest.java b/test/unit/org/apache/cassandra/repair/StreamingRepairTaskTest.java
index b845e93..ea5ebbf 100644
--- a/test/unit/org/apache/cassandra/repair/StreamingRepairTaskTest.java
+++ b/test/unit/org/apache/cassandra/repair/StreamingRepairTaskTest.java
@@ -26,7 +26,7 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
-import org.apache.cassandra.cql3.statements.CreateTableStatement;
+import org.apache.cassandra.cql3.statements.schema.CreateTableStatement;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.repair.messages.SyncRequest;
 import org.apache.cassandra.schema.KeyspaceParams;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/test/unit/org/apache/cassandra/repair/consistent/CoordinatorSessionsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/repair/consistent/CoordinatorSessionsTest.java b/test/unit/org/apache/cassandra/repair/consistent/CoordinatorSessionsTest.java
index bc90e9b..b9b1fbf 100644
--- a/test/unit/org/apache/cassandra/repair/consistent/CoordinatorSessionsTest.java
+++ b/test/unit/org/apache/cassandra/repair/consistent/CoordinatorSessionsTest.java
@@ -27,12 +27,12 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
-import org.apache.cassandra.cql3.statements.CreateTableStatement;
-import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.cql3.statements.schema.CreateTableStatement;
 import org.apache.cassandra.repair.AbstractRepairTest;
 import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.repair.messages.FailSession;
 import org.apache.cassandra.repair.messages.FinalizePromise;
 import org.apache.cassandra.repair.messages.PrepareConsistentResponse;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/test/unit/org/apache/cassandra/repair/consistent/LocalSessionTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/repair/consistent/LocalSessionTest.java b/test/unit/org/apache/cassandra/repair/consistent/LocalSessionTest.java
index df51444..d368510 100644
--- a/test/unit/org/apache/cassandra/repair/consistent/LocalSessionTest.java
+++ b/test/unit/org/apache/cassandra/repair/consistent/LocalSessionTest.java
@@ -40,16 +40,16 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
-import org.apache.cassandra.cql3.statements.CreateTableStatement;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.cql3.QueryProcessor;
+import org.apache.cassandra.cql3.statements.schema.CreateTableStatement;
 import org.apache.cassandra.repair.AbstractRepairTest;
 import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.repair.KeyspaceRepairManager;
 import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.schema.SchemaConstants;
-import org.apache.cassandra.cql3.QueryProcessor;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.SystemKeyspace;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/test/unit/org/apache/cassandra/schema/IndexMetadataTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/schema/IndexMetadataTest.java b/test/unit/org/apache/cassandra/schema/IndexMetadataTest.java
index 785ed73..c9e0d52 100644
--- a/test/unit/org/apache/cassandra/schema/IndexMetadataTest.java
+++ b/test/unit/org/apache/cassandra/schema/IndexMetadataTest.java
@@ -23,6 +23,8 @@ package org.apache.cassandra.schema;
 import org.junit.Assert;
 import org.junit.Test;
 
+import org.apache.cassandra.cql3.ColumnIdentifier;
+
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
@@ -49,8 +51,7 @@ public class IndexMetadataTest {
     @Test
     public void testGetDefaultIndexName()
     {
-        Assert.assertEquals("aB4__idx", IndexMetadata.getDefaultIndexName("a B-4@!_+", null));
-        Assert.assertEquals("34_Ddd_F6_idx", IndexMetadata.getDefaultIndexName("34_()Ddd", "#F%6*"));
-        
+        Assert.assertEquals("aB4__idx", IndexMetadata.generateDefaultIndexName("a B-4@!_+"));
+        Assert.assertEquals("34_Ddd_F6_idx", IndexMetadata.generateDefaultIndexName("34_()Ddd", new ColumnIdentifier("#F%6*", true)));
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/test/unit/org/apache/cassandra/schema/MigrationManagerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/schema/MigrationManagerTest.java b/test/unit/org/apache/cassandra/schema/MigrationManagerTest.java
index 3fbc3d7..5c70903 100644
--- a/test/unit/org/apache/cassandra/schema/MigrationManagerTest.java
+++ b/test/unit/org/apache/cassandra/schema/MigrationManagerTest.java
@@ -205,7 +205,7 @@ public class MigrationManagerTest
         store.forceBlockingFlush();
         assertTrue(store.getDirectories().sstableLister(Directories.OnTxnErr.THROW).list().size() > 0);
 
-        MigrationManager.announceTableDrop(ks.name, cfm.name);
+        MigrationManager.announceTableDrop(ks.name, cfm.name, false);
 
         assertFalse(Schema.instance.getKeyspaceMetadata(ks.name).tables.get(cfm.name).isPresent());
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/test/unit/org/apache/cassandra/schema/SchemaKeyspaceTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/schema/SchemaKeyspaceTest.java b/test/unit/org/apache/cassandra/schema/SchemaKeyspaceTest.java
index b3a7047..501c429 100644
--- a/test/unit/org/apache/cassandra/schema/SchemaKeyspaceTest.java
+++ b/test/unit/org/apache/cassandra/schema/SchemaKeyspaceTest.java
@@ -32,7 +32,7 @@ import org.junit.Test;
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.cql3.QueryProcessor;
 import org.apache.cassandra.cql3.UntypedResultSet;
-import org.apache.cassandra.cql3.statements.CreateTableStatement;
+import org.apache.cassandra.cql3.statements.schema.CreateTableStatement;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.Mutation;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/test/unit/org/apache/cassandra/service/QueryPagerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/service/QueryPagerTest.java b/test/unit/org/apache/cassandra/service/QueryPagerTest.java
index 56bf59c..407efc6 100644
--- a/test/unit/org/apache/cassandra/service/QueryPagerTest.java
+++ b/test/unit/org/apache/cassandra/service/QueryPagerTest.java
@@ -27,7 +27,7 @@ import org.junit.Test;
 import org.junit.runner.RunWith;
 
 import org.apache.cassandra.*;
-import org.apache.cassandra.cql3.statements.CreateTableStatement;
+import org.apache.cassandra.cql3.statements.schema.CreateTableStatement;
 import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.cql3.ColumnIdentifier;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/test/unit/org/apache/cassandra/transport/MessagePayloadTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/transport/MessagePayloadTest.java b/test/unit/org/apache/cassandra/transport/MessagePayloadTest.java
index 5b8067e..0c15bca 100644
--- a/test/unit/org/apache/cassandra/transport/MessagePayloadTest.java
+++ b/test/unit/org/apache/cassandra/transport/MessagePayloadTest.java
@@ -36,7 +36,6 @@ import org.apache.cassandra.cql3.QueryHandler;
 import org.apache.cassandra.cql3.QueryOptions;
 import org.apache.cassandra.cql3.QueryProcessor;
 import org.apache.cassandra.cql3.statements.BatchStatement;
-import org.apache.cassandra.cql3.statements.ParsedStatement;
 import org.apache.cassandra.cql3.CQLTester;
 import org.apache.cassandra.exceptions.RequestExecutionException;
 import org.apache.cassandra.exceptions.RequestValidationException;
@@ -368,7 +367,7 @@ public class MessagePayloadTest extends CQLTester
 
     public static class TestQueryHandler implements QueryHandler
     {
-        public ParsedStatement.Prepared getPrepared(MD5Digest id)
+        public QueryProcessor.Prepared getPrepared(MD5Digest id)
         {
             return QueryProcessor.instance.getPrepared(id);
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/test/unit/org/apache/cassandra/triggers/TriggersSchemaTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/triggers/TriggersSchemaTest.java b/test/unit/org/apache/cassandra/triggers/TriggersSchemaTest.java
index 88f74a2..31111bd 100644
--- a/test/unit/org/apache/cassandra/triggers/TriggersSchemaTest.java
+++ b/test/unit/org/apache/cassandra/triggers/TriggersSchemaTest.java
@@ -21,7 +21,7 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
-import org.apache.cassandra.cql3.statements.CreateTableStatement;
+import org.apache.cassandra.cql3.statements.schema.CreateTableStatement;
 import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.exceptions.ConfigurationException;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/tools/stress/src/org/apache/cassandra/io/sstable/StressCQLSSTableWriter.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/io/sstable/StressCQLSSTableWriter.java b/tools/stress/src/org/apache/cassandra/io/sstable/StressCQLSSTableWriter.java
index ae72f21..7dabe84 100644
--- a/tools/stress/src/org/apache/cassandra/io/sstable/StressCQLSSTableWriter.java
+++ b/tools/stress/src/org/apache/cassandra/io/sstable/StressCQLSSTableWriter.java
@@ -29,6 +29,9 @@ import org.apache.commons.lang3.ArrayUtils;
 import com.datastax.driver.core.ProtocolVersion;
 import com.datastax.driver.core.TypeCodec;
 import org.antlr.runtime.RecognitionException;
+import org.apache.cassandra.cql3.CQLStatement;
+import org.apache.cassandra.cql3.statements.schema.CreateTableStatement;
+import org.apache.cassandra.cql3.statements.schema.CreateTypeStatement;
 import org.apache.cassandra.schema.TableId;
 import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.config.DatabaseDescriptor;
@@ -39,9 +42,6 @@ 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;
@@ -58,7 +58,6 @@ import org.apache.cassandra.schema.TableMetadataRef;
 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.
@@ -359,8 +358,8 @@ public class StressCQLSSTableWriter implements Closeable
 
         private Boolean makeRangeAware = false;
 
-        private CreateTableStatement.RawStatement schemaStatement;
-        private final List<CreateTypeStatement> typeStatements;
+        private CreateTableStatement.Raw schemaStatement;
+        private final List<CreateTypeStatement.Raw> typeStatements;
         private UpdateStatement.ParsedInsert insertStatement;
         private IPartitioner partitioner;
 
@@ -430,7 +429,7 @@ public class StressCQLSSTableWriter implements Closeable
 
         public Builder withType(String typeDefinition) throws SyntaxException
         {
-            typeStatements.add(parseStatement(typeDefinition, CreateTypeStatement.class, "CREATE TYPE"));
+            typeStatements.add(parseStatement(typeDefinition, CreateTypeStatement.Raw.class, "CREATE TYPE"));
             return this;
         }
 
@@ -450,7 +449,7 @@ public class StressCQLSSTableWriter implements Closeable
          */
         public Builder forTable(String schema)
         {
-            this.schemaStatement = parseStatement(schema, CreateTableStatement.RawStatement.class, "CREATE TABLE");
+            this.schemaStatement = parseStatement(schema, CreateTableStatement.Raw.class, "CREATE TABLE");
             return this;
         }
 
@@ -567,25 +566,25 @@ public class StressCQLSSTableWriter implements Closeable
                 if (partitioner == null)
                     partitioner = cfs.getPartitioner();
 
-                Pair<UpdateStatement, List<ColumnSpecification>> preparedInsert = prepareInsert();
+                UpdateStatement 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);
+                                                     ? new SSTableSimpleWriter(cfs.getDirectories().getDirectoryForNewSSTables(), cfs.metadata, preparedInsert.updatedColumns())
+                                                     : new SSTableSimpleUnsortedWriter(cfs.getDirectories().getDirectoryForNewSSTables(), cfs.metadata, preparedInsert.updatedColumns(), bufferSizeInMB);
 
                 if (formatType != null)
                     writer.setSSTableFormatType(formatType);
 
                 writer.setRangeAwareWriting(makeRangeAware);
 
-                return new StressCQLSSTableWriter(cfs, writer, preparedInsert.left, preparedInsert.right);
+                return new StressCQLSSTableWriter(cfs, writer, preparedInsert, preparedInsert.getBindVariables());
             }
         }
 
-        private static void createTypes(String keyspace, List<CreateTypeStatement> typeStatements)
+        private static void createTypes(String keyspace, List<CreateTypeStatement.Raw> typeStatements)
         {
             KeyspaceMetadata ksm = Schema.instance.getKeyspaceMetadata(keyspace);
             Types.RawBuilder builder = Types.rawBuilder(keyspace);
-            for (CreateTypeStatement st : typeStatements)
+            for (CreateTypeStatement.Raw st : typeStatements)
                 st.addToRawBuilder(builder);
 
             ksm = ksm.withSwapped(builder.build());
@@ -594,14 +593,14 @@ public class StressCQLSSTableWriter implements Closeable
 
         public static ColumnFamilyStore createOfflineTable(String schema, List<File> directoryList)
         {
-            return createOfflineTable(parseStatement(schema, CreateTableStatement.RawStatement.class, "CREATE TABLE"), Collections.EMPTY_LIST, directoryList);
+            return createOfflineTable(parseStatement(schema, CreateTableStatement.Raw.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)
+        public static ColumnFamilyStore createOfflineTable(CreateTableStatement.Raw schemaStatement, List<CreateTypeStatement.Raw> typeStatements, List<File> directoryList)
         {
             String keyspace = schemaStatement.keyspace();
 
@@ -612,16 +611,17 @@ public class StressCQLSSTableWriter implements Closeable
 
             KeyspaceMetadata ksm = Schema.instance.getKeyspaceMetadata(keyspace);
 
-            TableMetadata tableMetadata = ksm.tables.getNullable(schemaStatement.columnFamily());
+            TableMetadata tableMetadata = ksm.tables.getNullable(schemaStatement.table());
             if (tableMetadata != null)
                 return Schema.instance.getColumnFamilyStoreInstance(tableMetadata.id);
 
-            CreateTableStatement statement = (CreateTableStatement) schemaStatement.prepare(ksm.types).statement;
-            statement.validate(ClientState.forInternalCalls());
+            ClientState state = ClientState.forInternalCalls();
+            CreateTableStatement statement = schemaStatement.prepare(state);
+            statement.validate(state);
 
             //Build metadata with a portable tableId
-            tableMetadata = statement.builder()
-                                     .id(deterministicId(statement.keyspace(), statement.columnFamily()))
+            tableMetadata = statement.builder(ksm.types)
+                                     .id(deterministicId(schemaStatement.keyspace(), schemaStatement.table()))
                                      .build();
 
             Keyspace.setInitialized();
@@ -646,28 +646,29 @@ public class StressCQLSSTableWriter implements Closeable
          *
          * @return prepared Insert statement and it's bound names
          */
-        private Pair<UpdateStatement, List<ColumnSpecification>> prepareInsert()
+        private UpdateStatement prepareInsert()
         {
-            ParsedStatement.Prepared cqlStatement = insertStatement.prepare();
-            UpdateStatement insert = (UpdateStatement) cqlStatement.statement;
-            insert.validate(ClientState.forInternalCalls());
+            ClientState state = ClientState.forInternalCalls();
+            CQLStatement cqlStatement = insertStatement.prepare(state);
+            UpdateStatement insert = (UpdateStatement) cqlStatement;
+            insert.validate(state);
 
             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())
+            if (insert.getBindVariables().isEmpty())
                 throw new IllegalArgumentException("Provided insert statement has no bind variables");
 
-            return Pair.create(insert, cqlStatement.boundNames);
+            return insert;
         }
     }
 
-    public static <T extends ParsedStatement> T parseStatement(String query, Class<T> klass, String type)
+    public static <T extends CQLStatement.Raw> T parseStatement(String query, Class<T> klass, String type)
     {
         try
         {
-            ParsedStatement stmt = CQLFragmentParser.parseAnyUnhandled(CqlParser::query, query);
+            CQLStatement.Raw 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());

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/tools/stress/src/org/apache/cassandra/stress/CompactionStress.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/CompactionStress.java b/tools/stress/src/org/apache/cassandra/stress/CompactionStress.java
index ffc1ace..b4be63d 100644
--- a/tools/stress/src/org/apache/cassandra/stress/CompactionStress.java
+++ b/tools/stress/src/org/apache/cassandra/stress/CompactionStress.java
@@ -30,7 +30,7 @@ import com.google.common.util.concurrent.Uninterruptibles;
 
 import io.airlift.airline.*;
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.cql3.statements.CreateTableStatement;
+import org.apache.cassandra.cql3.statements.schema.CreateTableStatement;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.Directories;
 import org.apache.cassandra.db.SystemKeyspace;
@@ -112,7 +112,7 @@ public abstract class CompactionStress implements Runnable
     {
         generateTokens(stressProfile.seedStr, StorageService.instance.getTokenMetadata(), numTokens);
 
-        CreateTableStatement.RawStatement createStatement = stressProfile.getCreateStatement();
+        CreateTableStatement.Raw createStatement = stressProfile.getCreateStatement();
         List<File> dataDirectories = getDataDirectories();
 
         ColumnFamilyStore cfs = StressCQLSSTableWriter.Builder.createOfflineTable(createStatement, Collections.EMPTY_LIST, dataDirectories);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/tools/stress/src/org/apache/cassandra/stress/StressProfile.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/StressProfile.java b/tools/stress/src/org/apache/cassandra/stress/StressProfile.java
index cda9c58..8de3ddf 100644
--- a/tools/stress/src/org/apache/cassandra/stress/StressProfile.java
+++ b/tools/stress/src/org/apache/cassandra/stress/StressProfile.java
@@ -40,9 +40,8 @@ import com.datastax.driver.core.exceptions.AlreadyExistsException;
 import org.antlr.runtime.RecognitionException;
 import org.apache.cassandra.cql3.CQLFragmentParser;
 import org.apache.cassandra.cql3.CqlParser;
-import org.apache.cassandra.cql3.QueryProcessor;
-import org.apache.cassandra.cql3.statements.CreateTableStatement;
 import org.apache.cassandra.cql3.statements.ModificationStatement;
+import org.apache.cassandra.cql3.statements.schema.CreateTableStatement;
 import org.apache.cassandra.exceptions.RequestValidationException;
 import org.apache.cassandra.exceptions.SyntaxException;
 import org.apache.cassandra.schema.ColumnMetadata;
@@ -165,7 +164,7 @@ public class StressProfile implements Serializable
         {
             try
             {
-                String name = CQLFragmentParser.parseAnyUnhandled(CqlParser::createKeyspaceStatement, keyspaceCql).keyspace();
+                String name = CQLFragmentParser.parseAnyUnhandled(CqlParser::createKeyspaceStatement, keyspaceCql).keyspaceName;
                 assert name.equalsIgnoreCase(keyspaceName) : "Name in keyspace_definition doesn't match keyspace property: '" + name + "' != '" + keyspaceName + "'";
             }
             catch (RecognitionException | SyntaxException e)
@@ -182,7 +181,7 @@ public class StressProfile implements Serializable
         {
             try
             {
-                String name = CQLFragmentParser.parseAnyUnhandled(CqlParser::createTableStatement, tableCql).columnFamily();
+                String name = CQLFragmentParser.parseAnyUnhandled(CqlParser::createTableStatement, tableCql).table();
                 assert name.equalsIgnoreCase(tableName) : "Name in table_definition doesn't match table property: '" + name + "' != '" + tableName + "'";
             }
             catch (RecognitionException | RuntimeException e)
@@ -461,11 +460,10 @@ public class StressProfile implements Serializable
         return new PartitionGenerator(partitionColumns, clusteringColumns, regularColumns, PartitionGenerator.Order.ARBITRARY);
     }
 
-    public CreateTableStatement.RawStatement getCreateStatement()
+    public CreateTableStatement.Raw getCreateStatement()
     {
-        CreateTableStatement.RawStatement createStatement = QueryProcessor.parseStatement(tableCql, CreateTableStatement.RawStatement.class, "CREATE TABLE");
-        createStatement.prepareKeyspace(keyspaceName);
-
+        CreateTableStatement.Raw createStatement = CQLFragmentParser.parseAny(CqlParser::createTableStatement, tableCql, "CREATE TABLE");
+        createStatement.keyspace(keyspaceName);
         return createStatement;
     }
 


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


[12/13] cassandra git commit: Make all DDL statements idempotent and not dependent on global state

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/cql3/QualifiedName.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/QualifiedName.java b/src/java/org/apache/cassandra/cql3/QualifiedName.java
new file mode 100644
index 0000000..fb2e110
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/QualifiedName.java
@@ -0,0 +1,116 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.cql3;
+
+import java.util.Locale;
+import java.util.Objects;
+
+/**
+ * Class for the names of the keyspace-prefixed elements (e.g. table, index, view names)
+ */
+public class QualifiedName
+{
+    /**
+     * The keyspace name as stored internally.
+     */
+    private String keyspace;
+    private String name;
+
+    public QualifiedName()
+    {
+    }
+
+    public QualifiedName(String keyspace, String name)
+    {
+        this.keyspace = keyspace;
+        this.name = name;
+    }
+
+    /**
+     * Sets the keyspace.
+     *
+     * @param ks the keyspace name
+     * @param keepCase <code>true</code> if the case must be kept, <code>false</code> otherwise.
+     */
+    public final void setKeyspace(String ks, boolean keepCase)
+    {
+        keyspace = toInternalName(ks, keepCase);
+    }
+
+    /**
+     * Checks if the keyspace is specified.
+     * @return <code>true</code> if the keyspace is specified, <code>false</code> otherwise.
+     */
+    public final boolean hasKeyspace()
+    {
+        return keyspace != null;
+    }
+
+    public final String getKeyspace()
+    {
+        return keyspace;
+    }
+
+    public void setName(String cf, boolean keepCase)
+    {
+        name = toInternalName(cf, keepCase);
+    }
+
+    public String getName()
+    {
+        return name;
+    }
+
+    @Override
+    public String toString()
+    {
+        return hasKeyspace()
+             ? String.format("%s.%s", keyspace, name)
+             : name;
+    }
+
+    @Override
+    public int hashCode()
+    {
+        return Objects.hash(keyspace, name);
+    }
+
+    public boolean equals(Object o)
+    {
+        if (this == o)
+            return true;
+
+        if (!(o instanceof QualifiedName))
+            return false;
+
+        QualifiedName qn = (QualifiedName) o;
+        return Objects.equals(keyspace, qn.keyspace) && name.equals(qn.name);
+    }
+
+    /**
+     * Converts the specified name into the name used internally.
+     *
+     * @param name the name
+     * @param keepCase <code>true</code> if the case must be kept, <code>false</code> otherwise.
+     * @return the name used internally.
+     */
+    private static String toInternalName(String name, boolean keepCase)
+    {
+        return keepCase ? name : name.toLowerCase(Locale.US);
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/cql3/QueryHandler.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/QueryHandler.java b/src/java/org/apache/cassandra/cql3/QueryHandler.java
index d3b41f0..b21f9e3 100644
--- a/src/java/org/apache/cassandra/cql3/QueryHandler.java
+++ b/src/java/org/apache/cassandra/cql3/QueryHandler.java
@@ -21,7 +21,6 @@ import java.nio.ByteBuffer;
 import java.util.Map;
 
 import org.apache.cassandra.cql3.statements.BatchStatement;
-import org.apache.cassandra.cql3.statements.ParsedStatement;
 import org.apache.cassandra.exceptions.RequestExecutionException;
 import org.apache.cassandra.exceptions.RequestValidationException;
 import org.apache.cassandra.service.ClientState;
@@ -41,7 +40,7 @@ public interface QueryHandler
                                    ClientState clientState,
                                    Map<String, ByteBuffer> customPayload) throws RequestValidationException;
 
-    ParsedStatement.Prepared getPrepared(MD5Digest id);
+    QueryHandler.Prepared getPrepared(MD5Digest id);
 
     ResultMessage processPrepared(CQLStatement statement,
                                   QueryState state,
@@ -54,4 +53,30 @@ public interface QueryHandler
                                BatchQueryOptions options,
                                Map<String, ByteBuffer> customPayload,
                                long queryStartNanoTime) throws RequestExecutionException, RequestValidationException;
+
+    public static class Prepared
+    {
+        public final CQLStatement statement;
+
+        public final MD5Digest resultMetadataId;
+
+        /**
+         * Contains the CQL statement source if the statement has been "regularly" perpared via
+         * {@link QueryHandler#prepare(String, ClientState, Map)}.
+         * Other usages of this class may or may not contain the CQL statement source.
+         */
+        public final String rawCQLStatement;
+
+        public Prepared(CQLStatement statement)
+        {
+            this(statement, "");
+        }
+
+        public Prepared(CQLStatement statement, String rawCQLStatement)
+        {
+            this.statement = statement;
+            this.rawCQLStatement = rawCQLStatement;
+            this.resultMetadataId = ResultSet.ResultMetadata.fromPrepared(statement).getResultMetadataId();
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/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 3f0b196..77b4cdc 100644
--- a/src/java/org/apache/cassandra/cql3/QueryProcessor.java
+++ b/src/java/org/apache/cassandra/cql3/QueryProcessor.java
@@ -67,11 +67,11 @@ public class QueryProcessor implements QueryHandler
 
     private static final Logger logger = LoggerFactory.getLogger(QueryProcessor.class);
 
-    private static final Cache<MD5Digest, ParsedStatement.Prepared> preparedStatements;
+    private static final Cache<MD5Digest, Prepared> preparedStatements;
 
     // A map for prepared statements used internally (which we don't want to mix with user statement, in particular we don't
     // bother with expiration on those.
-    private static final ConcurrentMap<String, ParsedStatement.Prepared> internalStatements = new ConcurrentHashMap<>();
+    private static final ConcurrentMap<String, Prepared> internalStatements = new ConcurrentHashMap<>();
 
     // Direct calls to processStatement do not increment the preparedStatementsExecuted/regularStatementsExecuted
     // counters. Callers of processStatement are responsible for correctly notifying metrics
@@ -118,7 +118,7 @@ public class QueryProcessor implements QueryHandler
     }
 
     // Work around initialization dependency
-    private static enum InternalStateInstance
+    private enum InternalStateInstance
     {
         INSTANCE;
 
@@ -126,9 +126,7 @@ public class QueryProcessor implements QueryHandler
 
         InternalStateInstance()
         {
-            ClientState state = ClientState.forInternalCalls();
-            state.setKeyspace(SchemaConstants.SYSTEM_KEYSPACE_NAME);
-            this.queryState = new QueryState(state);
+            queryState = new QueryState(ClientState.forInternalCalls(SchemaConstants.SYSTEM_KEYSPACE_NAME));
         }
     }
 
@@ -174,7 +172,7 @@ public class QueryProcessor implements QueryHandler
         Schema.instance.registerListener(new StatementInvalidatingListener());
     }
 
-    public ParsedStatement.Prepared getPrepared(MD5Digest id)
+    public Prepared getPrepared(MD5Digest id)
     {
         return preparedStatements.getIfPresent(id);
     }
@@ -201,7 +199,7 @@ public class QueryProcessor implements QueryHandler
     {
         logger.trace("Process {} @CL.{}", statement, options.getConsistency());
         ClientState clientState = queryState.getClientState();
-        statement.checkAccess(clientState);
+        statement.authorize(clientState);
         statement.validate(clientState);
 
         ResultMessage result = statement.execute(queryState, options, queryStartNanoTime);
@@ -226,10 +224,9 @@ public class QueryProcessor implements QueryHandler
     public ResultMessage process(String queryString, QueryState queryState, QueryOptions options, long queryStartNanoTime)
     throws RequestExecutionException, RequestValidationException
     {
-        ParsedStatement.Prepared p = getStatement(queryString, queryState.getClientState().cloneWithKeyspaceIfSet(options.getKeyspace()));
-        options.prepare(p.boundNames);
-        CQLStatement prepared = p.statement;
-        if (prepared.getBoundTerms() != options.getValues().size())
+        CQLStatement prepared = getStatement(queryString, queryState.getClientState().cloneWithKeyspaceIfSet(options.getKeyspace()));
+        options.prepare(prepared.getBindVariables());
+        if (prepared.getBindVariables().size() != options.getValues().size())
             throw new InvalidRequestException("Invalid amount of bind variables");
 
         if (!queryState.getClientState().isInternal)
@@ -238,7 +235,7 @@ public class QueryProcessor implements QueryHandler
         return processStatement(prepared, queryState, options, queryStartNanoTime);
     }
 
-    public static ParsedStatement.Prepared parseStatement(String queryStr, ClientState clientState) throws RequestValidationException
+    public static CQLStatement parseStatement(String queryStr, ClientState clientState) throws RequestValidationException
     {
         return getStatement(queryStr, clientState);
     }
@@ -257,43 +254,45 @@ public class QueryProcessor implements QueryHandler
             return null;
     }
 
-    private static QueryOptions makeInternalOptions(ParsedStatement.Prepared prepared, Object[] values)
+    private static QueryOptions makeInternalOptions(CQLStatement prepared, Object[] values)
     {
         return makeInternalOptions(prepared, values, ConsistencyLevel.ONE);
     }
 
-    private static QueryOptions makeInternalOptions(ParsedStatement.Prepared prepared, Object[] values, ConsistencyLevel cl)
+    private static QueryOptions makeInternalOptions(CQLStatement prepared, Object[] values, ConsistencyLevel cl)
     {
-        if (prepared.boundNames.size() != values.length)
-            throw new IllegalArgumentException(String.format("Invalid number of values. Expecting %d but got %d", prepared.boundNames.size(), values.length));
+        if (prepared.getBindVariables().size() != values.length)
+            throw new IllegalArgumentException(String.format("Invalid number of values. Expecting %d but got %d", prepared.getBindVariables().size(), values.length));
 
-        List<ByteBuffer> boundValues = new ArrayList<ByteBuffer>(values.length);
+        List<ByteBuffer> boundValues = new ArrayList<>(values.length);
         for (int i = 0; i < values.length; i++)
         {
             Object value = values[i];
-            AbstractType type = prepared.boundNames.get(i).type;
+            AbstractType type = prepared.getBindVariables().get(i).type;
             boundValues.add(value instanceof ByteBuffer || value == null ? (ByteBuffer)value : type.decompose(value));
         }
         return QueryOptions.forInternalCalls(cl, boundValues);
     }
 
-    public static ParsedStatement.Prepared prepareInternal(String query) throws RequestValidationException
+    public static Prepared prepareInternal(String query) throws RequestValidationException
     {
-        ParsedStatement.Prepared prepared = internalStatements.get(query);
+        Prepared prepared = internalStatements.get(query);
         if (prepared != null)
             return prepared;
 
         // Note: if 2 threads prepare the same query, we'll live so don't bother synchronizing
-        prepared = parseStatement(query, internalQueryState().getClientState());
-        prepared.statement.validate(internalQueryState().getClientState());
-        internalStatements.putIfAbsent(query, prepared);
+        CQLStatement statement = parseStatement(query, internalQueryState().getClientState());
+        statement.validate(internalQueryState().getClientState());
+
+        prepared = new Prepared(statement);
+        internalStatements.put(query, prepared);
         return prepared;
     }
 
     public static UntypedResultSet executeInternal(String query, Object... values)
     {
-        ParsedStatement.Prepared prepared = prepareInternal(query);
-        ResultMessage result = prepared.statement.executeInternal(internalQueryState(), makeInternalOptions(prepared, values));
+        Prepared prepared = prepareInternal(query);
+        ResultMessage result = prepared.statement.executeLocally(internalQueryState(), makeInternalOptions(prepared.statement, values));
         if (result instanceof ResultMessage.Rows)
             return UntypedResultSet.create(((ResultMessage.Rows)result).result);
         else
@@ -311,8 +310,8 @@ public class QueryProcessor implements QueryHandler
     {
         try
         {
-            ParsedStatement.Prepared prepared = prepareInternal(query);
-            ResultMessage result = prepared.statement.execute(state, makeInternalOptions(prepared, values, cl), System.nanoTime());
+            Prepared prepared = prepareInternal(query);
+            ResultMessage result = prepared.statement.execute(state, makeInternalOptions(prepared.statement, values, cl), System.nanoTime());
             if (result instanceof ResultMessage.Rows)
                 return UntypedResultSet.create(((ResultMessage.Rows)result).result);
             else
@@ -326,24 +325,24 @@ public class QueryProcessor implements QueryHandler
 
     public static UntypedResultSet executeInternalWithPaging(String query, int pageSize, Object... values)
     {
-        ParsedStatement.Prepared prepared = prepareInternal(query);
+        Prepared prepared = prepareInternal(query);
         if (!(prepared.statement instanceof SelectStatement))
             throw new IllegalArgumentException("Only SELECTs can be paged");
 
         SelectStatement select = (SelectStatement)prepared.statement;
-        QueryPager pager = select.getQuery(makeInternalOptions(prepared, values), FBUtilities.nowInSeconds()).getPager(null, ProtocolVersion.CURRENT);
+        QueryPager pager = select.getQuery(makeInternalOptions(prepared.statement, values), FBUtilities.nowInSeconds()).getPager(null, ProtocolVersion.CURRENT);
         return UntypedResultSet.create(select, pager, pageSize);
     }
 
     /**
-     * Same than executeInternal, but to use for queries we know are only executed once so that the
+     * Same than executeLocally, but to use for queries we know are only executed once so that the
      * created statement object is not cached.
      */
     public static UntypedResultSet executeOnceInternal(String query, Object... values)
     {
-        ParsedStatement.Prepared prepared = parseStatement(query, internalQueryState().getClientState());
-        prepared.statement.validate(internalQueryState().getClientState());
-        ResultMessage result = prepared.statement.executeInternal(internalQueryState(), makeInternalOptions(prepared, values));
+        CQLStatement statement = parseStatement(query, internalQueryState().getClientState());
+        statement.validate(internalQueryState().getClientState());
+        ResultMessage result = statement.executeLocally(internalQueryState(), makeInternalOptions(statement, values));
         if (result instanceof ResultMessage.Rows)
             return UntypedResultSet.create(((ResultMessage.Rows)result).result);
         else
@@ -351,16 +350,16 @@ public class QueryProcessor implements QueryHandler
     }
 
     /**
-     * A special version of executeInternal that takes the time used as "now" for the query in argument.
+     * A special version of executeLocally that takes the time used as "now" for the query in argument.
      * Note that this only make sense for Selects so this only accept SELECT statements and is only useful in rare
      * cases.
      */
     public static UntypedResultSet executeInternalWithNow(int nowInSec, long queryStartNanoTime, String query, Object... values)
     {
-        ParsedStatement.Prepared prepared = prepareInternal(query);
+        Prepared prepared = prepareInternal(query);
         assert prepared.statement instanceof SelectStatement;
         SelectStatement select = (SelectStatement)prepared.statement;
-        ResultMessage result = select.executeInternal(internalQueryState(), makeInternalOptions(prepared, values), nowInSec, queryStartNanoTime);
+        ResultMessage result = select.executeInternal(internalQueryState(), makeInternalOptions(prepared.statement, values), nowInSec, queryStartNanoTime);
         assert result instanceof ResultMessage.Rows;
         return UntypedResultSet.create(((ResultMessage.Rows)result).result);
     }
@@ -374,7 +373,7 @@ public class QueryProcessor implements QueryHandler
     {
         try (PartitionIterator iter = partitions)
         {
-            SelectStatement ss = (SelectStatement) getStatement(query, null).statement;
+            SelectStatement ss = (SelectStatement) getStatement(query, null);
             ResultSet cqlRows = ss.process(iter, FBUtilities.nowInSeconds());
             return UntypedResultSet.create(cqlRows);
         }
@@ -393,12 +392,12 @@ public class QueryProcessor implements QueryHandler
         if (existing != null)
             return existing;
 
-        ParsedStatement.Prepared prepared = getStatement(queryString, clientState);
-        prepared.rawCQLStatement = queryString;
-        int boundTerms = prepared.statement.getBoundTerms();
+        CQLStatement statement = getStatement(queryString, clientState);
+        Prepared prepared = new Prepared(statement, queryString);
+
+        int boundTerms = statement.getBindVariables().size();
         if (boundTerms > FBUtilities.MAX_UNSIGNED_SHORT)
             throw new InvalidRequestException(String.format("Too many markers(?). %d markers exceed the allowed maximum of %d", boundTerms, FBUtilities.MAX_UNSIGNED_SHORT));
-        assert boundTerms == prepared.boundNames.size();
 
         return storePreparedStatement(queryString, clientState.getRawKeyspace(), prepared);
     }
@@ -413,19 +412,19 @@ public class QueryProcessor implements QueryHandler
     throws InvalidRequestException
     {
         MD5Digest statementId = computeId(queryString, keyspace);
-        ParsedStatement.Prepared existing = preparedStatements.getIfPresent(statementId);
+        Prepared existing = preparedStatements.getIfPresent(statementId);
         if (existing == null)
             return null;
 
         checkTrue(queryString.equals(existing.rawCQLStatement),
                 String.format("MD5 hash collision: query with the same MD5 hash was already prepared. \n Existing: '%s'", existing.rawCQLStatement));
 
-        ResultSet.PreparedMetadata preparedMetadata = ResultSet.PreparedMetadata.fromPrepared(existing);
-        ResultSet.ResultMetadata resultMetadata = ResultSet.ResultMetadata.fromPrepared(existing);
+        ResultSet.PreparedMetadata preparedMetadata = ResultSet.PreparedMetadata.fromPrepared(existing.statement);
+        ResultSet.ResultMetadata resultMetadata = ResultSet.ResultMetadata.fromPrepared(existing.statement);
         return new ResultMessage.Prepared(statementId, resultMetadata.getResultMetadataId(), preparedMetadata, resultMetadata);
     }
 
-    private static ResultMessage.Prepared storePreparedStatement(String queryString, String keyspace, ParsedStatement.Prepared prepared)
+    private static ResultMessage.Prepared storePreparedStatement(String queryString, String keyspace, Prepared prepared)
     throws InvalidRequestException
     {
         // Concatenate the current keyspace so we don't mix prepared statements between keyspace (#5352).
@@ -440,8 +439,8 @@ public class QueryProcessor implements QueryHandler
         MD5Digest statementId = computeId(queryString, keyspace);
         preparedStatements.put(statementId, prepared);
         SystemKeyspace.writePreparedStatement(keyspace, statementId, queryString);
-        ResultSet.PreparedMetadata preparedMetadata = ResultSet.PreparedMetadata.fromPrepared(prepared);
-        ResultSet.ResultMetadata resultMetadata = ResultSet.ResultMetadata.fromPrepared(prepared);
+        ResultSet.PreparedMetadata preparedMetadata = ResultSet.PreparedMetadata.fromPrepared(prepared.statement);
+        ResultSet.ResultMetadata resultMetadata = ResultSet.ResultMetadata.fromPrepared(prepared.statement);
         return new ResultMessage.Prepared(statementId, resultMetadata.getResultMetadataId(), preparedMetadata, resultMetadata);
     }
 
@@ -460,11 +459,11 @@ public class QueryProcessor implements QueryHandler
     {
         List<ByteBuffer> variables = options.getValues();
         // Check to see if there are any bound variables to verify
-        if (!(variables.isEmpty() && (statement.getBoundTerms() == 0)))
+        if (!(variables.isEmpty() && statement.getBindVariables().isEmpty()))
         {
-            if (variables.size() != statement.getBoundTerms())
+            if (variables.size() != statement.getBindVariables().size())
                 throw new InvalidRequestException(String.format("there were %d markers(?) in CQL but %d bound variables",
-                                                                statement.getBoundTerms(),
+                                                                statement.getBindVariables().size(),
                                                                 variables.size()));
 
             // at this point there is a match in count between markers and variables that is non-zero
@@ -491,31 +490,31 @@ public class QueryProcessor implements QueryHandler
     throws RequestExecutionException, RequestValidationException
     {
         ClientState clientState = queryState.getClientState().cloneWithKeyspaceIfSet(options.getKeyspace());
-        batch.checkAccess(clientState);
+        batch.authorize(clientState);
         batch.validate();
         batch.validate(clientState);
         return batch.execute(queryState, options, queryStartNanoTime);
     }
 
-    public static ParsedStatement.Prepared getStatement(String queryStr, ClientState clientState)
+    public static CQLStatement getStatement(String queryStr, ClientState clientState)
     throws RequestValidationException
     {
         Tracing.trace("Parsing {}", queryStr);
-        ParsedStatement statement = parseStatement(queryStr);
+        CQLStatement.Raw statement = parseStatement(queryStr);
 
         // Set keyspace for statement that require login
-        if (statement instanceof CFStatement)
-            ((CFStatement) statement).prepareKeyspace(clientState);
+        if (statement instanceof QualifiedStatement)
+            ((QualifiedStatement) statement).setKeyspace(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
+    public static <T extends CQLStatement.Raw> T parseStatement(String queryStr, Class<T> klass, String type) throws SyntaxException
     {
         try
         {
-            ParsedStatement stmt = parseStatement(queryStr);
+            CQLStatement.Raw stmt = parseStatement(queryStr);
 
             if (!klass.isAssignableFrom(stmt.getClass()))
                 throw new IllegalArgumentException("Invalid query, must be a " + type + " statement but was: " + stmt.getClass());
@@ -527,7 +526,7 @@ public class QueryProcessor implements QueryHandler
             throw new IllegalArgumentException(e.getMessage(), e);
         }
     }
-    public static ParsedStatement parseStatement(String queryStr) throws SyntaxException
+    public static CQLStatement.Raw parseStatement(String queryStr) throws SyntaxException
     {
         try
         {
@@ -551,7 +550,7 @@ public class QueryProcessor implements QueryHandler
         }
     }
 
-    private static int measure(Object key, ParsedStatement.Prepared value)
+    private static int measure(Object key, Prepared value)
     {
         return Ints.checkedCast(ObjectSizes.measureDeep(key) + ObjectSizes.measureDeep(value));
     }
@@ -577,10 +576,10 @@ public class QueryProcessor implements QueryHandler
         {
             Predicate<Function> matchesFunction = f -> ksName.equals(f.name().keyspace) && functionName.equals(f.name().name);
 
-            for (Iterator<Map.Entry<MD5Digest, ParsedStatement.Prepared>> iter = preparedStatements.asMap().entrySet().iterator();
+            for (Iterator<Map.Entry<MD5Digest, Prepared>> iter = preparedStatements.asMap().entrySet().iterator();
                  iter.hasNext();)
             {
-                Map.Entry<MD5Digest, ParsedStatement.Prepared> pstmt = iter.next();
+                Map.Entry<MD5Digest, Prepared> pstmt = iter.next();
                 if (Iterables.any(pstmt.getValue().statement.getFunctions(), matchesFunction))
                 {
                     SystemKeyspace.removePreparedStatement(pstmt.getKey());
@@ -593,12 +592,12 @@ public class QueryProcessor implements QueryHandler
                                statement -> Iterables.any(statement.statement.getFunctions(), matchesFunction));
         }
 
-        private static void removeInvalidPersistentPreparedStatements(Iterator<Map.Entry<MD5Digest, ParsedStatement.Prepared>> iterator,
+        private static void removeInvalidPersistentPreparedStatements(Iterator<Map.Entry<MD5Digest, Prepared>> iterator,
                                                                       String ksName, String cfName)
         {
             while (iterator.hasNext())
             {
-                Map.Entry<MD5Digest, ParsedStatement.Prepared> entry = iterator.next();
+                Map.Entry<MD5Digest, Prepared> entry = iterator.next();
                 if (shouldInvalidate(ksName, cfName, entry.getValue().statement))
                 {
                     SystemKeyspace.removePreparedStatement(entry.getKey());
@@ -607,7 +606,7 @@ public class QueryProcessor implements QueryHandler
             }
         }
 
-        private static void removeInvalidPreparedStatements(Iterator<ParsedStatement.Prepared> iterator, String ksName, String cfName)
+        private static void removeInvalidPreparedStatements(Iterator<Prepared> iterator, String ksName, String cfName)
         {
             while (iterator.hasNext())
             {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/cql3/ResultSet.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/ResultSet.java b/src/java/org/apache/cassandra/cql3/ResultSet.java
index 9d79dea..455f7c4 100644
--- a/src/java/org/apache/cassandra/cql3/ResultSet.java
+++ b/src/java/org/apache/cassandra/cql3/ResultSet.java
@@ -31,7 +31,6 @@ import java.util.Objects;
 import com.google.common.annotations.VisibleForTesting;
 
 import io.netty.buffer.ByteBuf;
-import org.apache.cassandra.cql3.statements.ParsedStatement;
 import org.apache.cassandra.cql3.statements.SelectStatement;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.service.pager.PagingState;
@@ -306,10 +305,8 @@ public class ResultSet
             return resultMetadataId;
         }
 
-        public static ResultMetadata fromPrepared(ParsedStatement.Prepared prepared)
+        public static ResultMetadata fromPrepared(CQLStatement statement)
         {
-            CQLStatement statement = prepared.statement;
-
             if (statement instanceof SelectStatement)
                 return ((SelectStatement)statement).getResultMetadata();
 
@@ -569,9 +566,9 @@ public class ResultSet
             return sb.toString();
         }
 
-        public static PreparedMetadata fromPrepared(ParsedStatement.Prepared prepared)
+        public static PreparedMetadata fromPrepared(CQLStatement statement)
         {
-            return new PreparedMetadata(prepared.boundNames, prepared.partitionKeyBindIndexes);
+            return new PreparedMetadata(statement.getBindVariables(), statement.getPartitionKeyBindVariableIndexes());
         }
 
         private static class Codec implements CBCodec<PreparedMetadata>

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/cql3/SingleColumnRelation.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/SingleColumnRelation.java b/src/java/org/apache/cassandra/cql3/SingleColumnRelation.java
index 7bd7aac..d9c5b26 100644
--- a/src/java/org/apache/cassandra/cql3/SingleColumnRelation.java
+++ b/src/java/org/apache/cassandra/cql3/SingleColumnRelation.java
@@ -20,6 +20,7 @@ package org.apache.cassandra.cql3;
 import java.util.Collections;
 import java.util.ArrayList;
 import java.util.List;
+import java.util.Objects;
 
 import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.schema.TableMetadata;
@@ -149,12 +150,34 @@ public final class SingleColumnRelation extends Relation
             entityAsString = String.format("%s[%s]", entityAsString, mapKey);
 
         if (isIN())
-            return String.format("%s IN %s", entityAsString, inValues);
+            return String.format("%s IN %s", entityAsString, Tuples.tupleToString(inValues));
 
         return String.format("%s %s %s", entityAsString, relationType, value);
     }
 
     @Override
+    public int hashCode()
+    {
+        return Objects.hash(relationType, entity, mapKey, value, inValues);
+    }
+
+    @Override
+    public boolean equals(Object o)
+    {
+        if (this == o)
+            return true;
+
+        if (!(o instanceof SingleColumnRelation))
+            return false;
+
+        SingleColumnRelation scr = (SingleColumnRelation) o;
+        return Objects.equals(entity, scr.entity)
+            && Objects.equals(mapKey, scr.mapKey)
+            && Objects.equals(value, scr.value)
+            && Objects.equals(inValues, scr.inValues);
+    }
+
+    @Override
     protected Restriction newEQRestriction(TableMetadata table, VariableSpecifications boundNames)
     {
         ColumnMetadata columnDef = entity.prepare(table);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/cql3/Term.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/Term.java b/src/java/org/apache/cassandra/cql3/Term.java
index f1737d1..f536baa 100644
--- a/src/java/org/apache/cassandra/cql3/Term.java
+++ b/src/java/org/apache/cassandra/cql3/Term.java
@@ -124,6 +124,18 @@ public interface Term
         {
             return getText();
         }
+
+        @Override
+        public int hashCode()
+        {
+            return getText().hashCode();
+        }
+
+        @Override
+        public boolean equals(Object o)
+        {
+            return this == o || (o instanceof Raw && getText().equals(((Raw) o).getText()));
+        }
     }
 
     public abstract class MultiColumnRaw extends Term.Raw

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/cql3/TokenRelation.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/TokenRelation.java b/src/java/org/apache/cassandra/cql3/TokenRelation.java
index 62f603b..4e3313d 100644
--- a/src/java/org/apache/cassandra/cql3/TokenRelation.java
+++ b/src/java/org/apache/cassandra/cql3/TokenRelation.java
@@ -20,6 +20,7 @@ package org.apache.cassandra.cql3;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
+import java.util.Objects;
 import java.util.stream.Collectors;
 
 import com.google.common.base.Joiner;
@@ -143,6 +144,25 @@ public final class TokenRelation extends Relation
         return String.format("token%s %s %s", Tuples.tupleToString(entities), relationType, value);
     }
 
+    @Override
+    public int hashCode()
+    {
+        return Objects.hash(relationType, entities, value);
+    }
+
+    @Override
+    public boolean equals(Object o)
+    {
+        if (this == o)
+            return true;
+
+        if (!(o instanceof TokenRelation))
+            return false;
+
+        TokenRelation tr = (TokenRelation) o;
+        return entities.equals(tr.entities) && value.equals(tr.value);
+    }
+
     /**
      * Returns the definition of the columns to which apply the token restriction.
      *

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/cql3/VariableSpecifications.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/VariableSpecifications.java b/src/java/org/apache/cassandra/cql3/VariableSpecifications.java
index 96290a6..e58290e 100644
--- a/src/java/org/apache/cassandra/cql3/VariableSpecifications.java
+++ b/src/java/org/apache/cassandra/cql3/VariableSpecifications.java
@@ -27,13 +27,13 @@ import org.apache.cassandra.schema.TableMetadata;
 public class VariableSpecifications
 {
     private final List<ColumnIdentifier> variableNames;
-    private final ColumnSpecification[] specs;
+    private final List<ColumnSpecification> specs;
     private final ColumnMetadata[] targetColumns;
 
     public VariableSpecifications(List<ColumnIdentifier> variableNames)
     {
         this.variableNames = variableNames;
-        this.specs = new ColumnSpecification[variableNames.size()];
+        this.specs = Arrays.asList(new ColumnSpecification[variableNames.size()]);
         this.targetColumns = new ColumnMetadata[variableNames.size()];
     }
 
@@ -43,17 +43,17 @@ public class VariableSpecifications
      */
     public static VariableSpecifications empty()
     {
-        return new VariableSpecifications(Collections.<ColumnIdentifier> emptyList());
+        return new VariableSpecifications(Collections.emptyList());
     }
 
-    public int size()
+    public boolean isEmpty()
     {
-        return variableNames.size();
+        return variableNames.isEmpty();
     }
 
-    public List<ColumnSpecification> getSpecifications()
+    public List<ColumnSpecification> getBindVariables()
     {
-        return Arrays.asList(specs);
+        return specs;
     }
 
     /**
@@ -63,7 +63,7 @@ public class VariableSpecifications
      *
      * Callers of this method should ensure that all statements operate on the same table.
      */
-    public short[] getPartitionKeyBindIndexes(TableMetadata metadata)
+    public short[] getPartitionKeyBindVariableIndexes(TableMetadata metadata)
     {
         short[] partitionKeyPositions = new short[metadata.partitionKeyColumns().size()];
         boolean[] set = new boolean[partitionKeyPositions.length];
@@ -94,12 +94,12 @@ public class VariableSpecifications
         // Use the user name, if there is one
         if (bindMarkerName != null)
             spec = new ColumnSpecification(spec.ksName, spec.cfName, bindMarkerName, spec.type);
-        specs[bindIndex] = spec;
+        specs.set(bindIndex, spec);
     }
 
     @Override
     public String toString()
     {
-        return Arrays.toString(specs);
+        return specs.toString();
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/cql3/WhereClause.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/WhereClause.java b/src/java/org/apache/cassandra/cql3/WhereClause.java
index fb783a5..87041f9 100644
--- a/src/java/org/apache/cassandra/cql3/WhereClause.java
+++ b/src/java/org/apache/cassandra/cql3/WhereClause.java
@@ -15,20 +15,24 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.cassandra.cql3;
 
 import java.util.List;
+import java.util.Objects;
 
 import com.google.common.collect.ImmutableList;
 
+import org.antlr.runtime.RecognitionException;
+import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.cql3.restrictions.CustomIndexExpression;
-import org.apache.commons.lang3.builder.ToStringBuilder;
-import org.apache.commons.lang3.builder.ToStringStyle;
+
+import static java.lang.String.join;
+
+import static com.google.common.collect.Iterables.concat;
+import static com.google.common.collect.Iterables.transform;
 
 public final class WhereClause
 {
-
     private static final WhereClause EMPTY = new WhereClause(new Builder());
 
     public final List<Relation> relations;
@@ -36,9 +40,8 @@ public final class WhereClause
 
     private WhereClause(Builder builder)
     {
-        this.relations = builder.relations.build();
-        this.expressions = builder.expressions.build();
-
+        relations = builder.relations.build();
+        expressions = builder.expressions.build();
     }
 
     public static WhereClause empty()
@@ -51,6 +54,57 @@ public final class WhereClause
         return !expressions.isEmpty();
     }
 
+    /**
+     * Renames identifiers in all relations
+     * @param from the old identifier
+     * @param to the new identifier
+     * @return a new WhereClause with with "from" replaced by "to" in all relations
+     */
+    public WhereClause renameIdentifier(ColumnMetadata.Raw from, ColumnMetadata.Raw to)
+    {
+        WhereClause.Builder builder = new WhereClause.Builder();
+
+        relations.stream()
+                 .map(r -> r.renameIdentifier(from, to))
+                 .forEach(builder::add);
+
+        expressions.forEach(builder::add);
+
+        return builder.build();
+    }
+
+    public static WhereClause parse(String cql) throws RecognitionException
+    {
+        return CQLFragmentParser.parseAnyUnhandled(CqlParser::whereClause, cql).build();
+    }
+
+    @Override
+    public String toString()
+    {
+        return join(" AND ",
+                    concat(transform(relations, Relation::toString),
+                           transform(expressions, CustomIndexExpression::toString)));
+    }
+
+    @Override
+    public boolean equals(Object o)
+    {
+        if (this == o)
+            return true;
+
+        if (!(o instanceof WhereClause))
+            return false;
+
+        WhereClause wc = (WhereClause) o;
+        return relations.equals(wc.relations) && expressions.equals(wc.expressions);
+    }
+
+    @Override
+    public int hashCode()
+    {
+        return Objects.hash(relations, expressions);
+    }
+
     public static final class Builder
     {
         ImmutableList.Builder<Relation> relations = new ImmutableList.Builder<>();
@@ -73,10 +127,4 @@ public final class WhereClause
             return new WhereClause(this);
         }
     }
-    
-    @Override
-    public String toString()
-    {
-        return ToStringBuilder.reflectionToString(this, ToStringStyle.SHORT_PREFIX_STYLE);
-    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/cql3/functions/AbstractFunction.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/functions/AbstractFunction.java b/src/java/org/apache/cassandra/cql3/functions/AbstractFunction.java
index 5e10e9f..aea0d01 100644
--- a/src/java/org/apache/cassandra/cql3/functions/AbstractFunction.java
+++ b/src/java/org/apache/cassandra/cql3/functions/AbstractFunction.java
@@ -17,6 +17,7 @@
  */
 package org.apache.cassandra.cql3.functions;
 
+import java.nio.ByteBuffer;
 import java.util.List;
 
 import com.google.common.base.Objects;
@@ -25,6 +26,7 @@ import org.apache.cassandra.cql3.AssignmentTestable;
 import org.apache.cassandra.cql3.CQL3Type;
 import org.apache.cassandra.cql3.ColumnSpecification;
 import org.apache.cassandra.db.marshal.AbstractType;
+
 import org.apache.commons.lang3.text.StrBuilder;
 
 import static java.util.stream.Collectors.toList;
@@ -85,7 +87,7 @@ public abstract class AbstractFunction implements Function
         functions.add(this);
     }
 
-    public boolean hasReferenceTo(Function function)
+    public boolean referencesUserType(ByteBuffer name)
     {
         return false;
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/cql3/functions/Function.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/functions/Function.java b/src/java/org/apache/cassandra/cql3/functions/Function.java
index 5d258af..e13e906 100644
--- a/src/java/org/apache/cassandra/cql3/functions/Function.java
+++ b/src/java/org/apache/cassandra/cql3/functions/Function.java
@@ -17,10 +17,13 @@
  */
 package org.apache.cassandra.cql3.functions;
 
+import java.nio.ByteBuffer;
 import java.util.List;
+import java.util.Optional;
 
 import org.apache.cassandra.cql3.AssignmentTestable;
 import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.schema.Difference;
 import org.github.jamm.Unmetered;
 
 @Unmetered
@@ -46,7 +49,7 @@ public interface Function extends AssignmentTestable
 
     public void addFunctionsTo(List<Function> functions);
 
-    public boolean hasReferenceTo(Function function);
+    public boolean referencesUserType(ByteBuffer name);
 
     /**
      * Returns the name of the function to use within a ResultSet.
@@ -55,4 +58,9 @@ public interface Function extends AssignmentTestable
      * @return the name of the function to use within a ResultSet
      */
     public String columnName(List<String> columnNames);
+
+    public default Optional<Difference> compare(Function other)
+    {
+        throw new UnsupportedOperationException();
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/cql3/functions/UDAggregate.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/functions/UDAggregate.java b/src/java/org/apache/cassandra/cql3/functions/UDAggregate.java
index 1a3174c..1a49b33 100644
--- a/src/java/org/apache/cassandra/cql3/functions/UDAggregate.java
+++ b/src/java/org/apache/cassandra/cql3/functions/UDAggregate.java
@@ -21,16 +21,23 @@ import java.nio.ByteBuffer;
 import java.util.*;
 
 import com.google.common.base.Objects;
+import com.google.common.collect.Lists;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import com.datastax.driver.core.TypeCodec;
 import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.UserType;
+import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.schema.Difference;
 import org.apache.cassandra.schema.Functions;
 import org.apache.cassandra.tracing.Tracing;
 import org.apache.cassandra.transport.ProtocolVersion;
 
+import static com.google.common.collect.Iterables.any;
+import static com.google.common.collect.Iterables.transform;
+
 /**
  * Base class for user-defined-aggregates.
  */
@@ -55,13 +62,13 @@ public class UDAggregate extends AbstractFunction implements AggregateFunction
         super(name, argTypes, returnType);
         this.stateFunction = stateFunc;
         this.finalFunction = finalFunc;
-        this.stateType = stateFunc != null ? stateFunc.returnType() : null;
-        this.stateTypeCodec = stateType != null ? UDHelper.codecFor(UDHelper.driverType(stateType)) : null;
-        this.returnTypeCodec = returnType != null ? UDHelper.codecFor(UDHelper.driverType(returnType)) : null;
+        this.stateType = stateFunc.returnType();
+        this.stateTypeCodec = UDHelper.codecFor(UDHelper.driverType(stateType));
+        this.returnTypeCodec = UDHelper.codecFor(UDHelper.driverType(returnType));
         this.initcond = initcond;
     }
 
-    public static UDAggregate create(Functions functions,
+    public static UDAggregate create(Collection<UDFunction> functions,
                                      FunctionName name,
                                      List<AbstractType<?>> argTypes,
                                      AbstractType<?> returnType,
@@ -69,7 +76,6 @@ public class UDAggregate extends AbstractFunction implements AggregateFunction
                                      FunctionName finalFunc,
                                      AbstractType<?> stateType,
                                      ByteBuffer initcond)
-    throws InvalidRequestException
     {
         List<AbstractType<?>> stateTypes = new ArrayList<>(argTypes.size() + 1);
         stateTypes.add(stateType);
@@ -78,27 +84,17 @@ public class UDAggregate extends AbstractFunction implements AggregateFunction
         return new UDAggregate(name,
                                argTypes,
                                returnType,
-                               resolveScalar(functions, name, stateFunc, stateTypes),
-                               finalFunc != null ? resolveScalar(functions, name, finalFunc, finalTypes) : null,
+                               findFunction(name, functions, stateFunc, stateTypes),
+                               null == finalFunc ? null : findFunction(name, functions, finalFunc, finalTypes),
                                initcond);
     }
 
-    public static UDAggregate createBroken(FunctionName name,
-                                           List<AbstractType<?>> argTypes,
-                                           AbstractType<?> returnType,
-                                           ByteBuffer initcond,
-                                           InvalidRequestException reason)
+    private static UDFunction findFunction(FunctionName udaName, Collection<UDFunction> functions, FunctionName name, List<AbstractType<?>> arguments)
     {
-        return new UDAggregate(name, argTypes, returnType, null, null, initcond)
-        {
-            public Aggregate newAggregate() throws InvalidRequestException
-            {
-                throw new InvalidRequestException(String.format("Aggregate '%s' exists but hasn't been loaded successfully for the following reason: %s. "
-                                                                + "Please see the server log for more details",
-                                                                this,
-                                                                reason.getMessage()));
-            }
-        };
+        return functions.stream()
+                        .filter(f -> f.name().equals(name) && Functions.typesMatch(f.argTypes(), arguments))
+                        .findFirst()
+                        .orElseThrow(() -> new ConfigurationException(String.format("Unable to find function %s referenced by UDA %s", name, udaName)));
     }
 
     public boolean hasReferenceTo(Function function)
@@ -107,16 +103,37 @@ public class UDAggregate extends AbstractFunction implements AggregateFunction
     }
 
     @Override
+    public boolean referencesUserType(ByteBuffer name)
+    {
+        return any(argTypes(), t -> t.referencesUserType(name))
+            || returnType.referencesUserType(name)
+            || (null != stateType && stateType.referencesUserType(name))
+            || stateFunction.referencesUserType(name)
+            || (null != finalFunction && finalFunction.referencesUserType(name));
+    }
+
+    public UDAggregate withUpdatedUserType(Collection<UDFunction> udfs, UserType udt)
+    {
+        if (!referencesUserType(udt.name))
+            return this;
+
+        return new UDAggregate(name,
+                               Lists.newArrayList(transform(argTypes, t -> t.withUpdatedUserType(udt))),
+                               returnType.withUpdatedUserType(udt),
+                               findFunction(name, udfs, stateFunction.name(), stateFunction.argTypes()),
+                               null == finalFunction ? null : findFunction(name, udfs, finalFunction.name(), finalFunction.argTypes()),
+                               initcond);
+    }
+
+    @Override
     public void addFunctionsTo(List<Function> functions)
     {
         functions.add(this);
-        if (stateFunction != null)
-        {
-            stateFunction.addFunctionsTo(functions);
 
-            if (finalFunction != null)
-                finalFunction.addFunctionsTo(functions);
-        }
+        stateFunction.addFunctionsTo(functions);
+
+        if (finalFunction != null)
+            finalFunction.addFunctionsTo(functions);
     }
 
     public boolean isAggregate()
@@ -214,23 +231,6 @@ public class UDAggregate extends AbstractFunction implements AggregateFunction
         };
     }
 
-    private static ScalarFunction resolveScalar(Functions functions, FunctionName aName, FunctionName fName, List<AbstractType<?>> argTypes) throws InvalidRequestException
-    {
-        Optional<Function> fun = functions.find(fName, argTypes);
-        if (!fun.isPresent())
-            throw new InvalidRequestException(String.format("Referenced state function '%s %s' for aggregate '%s' does not exist",
-                                                            fName,
-                                                            Arrays.toString(UDHelper.driverTypes(argTypes)),
-                                                            aName));
-
-        if (!(fun.get() instanceof ScalarFunction))
-            throw new InvalidRequestException(String.format("Referenced state function '%s %s' for aggregate '%s' is not a scalar function",
-                                                            fName,
-                                                            Arrays.toString(UDHelper.driverTypes(argTypes)),
-                                                            aName));
-        return (ScalarFunction) fun.get();
-    }
-
     @Override
     public boolean equals(Object o)
     {
@@ -238,13 +238,83 @@ public class UDAggregate extends AbstractFunction implements AggregateFunction
             return false;
 
         UDAggregate that = (UDAggregate) o;
-        return Objects.equal(name, that.name)
-            && Functions.typesMatch(argTypes, that.argTypes)
-            && Functions.typesMatch(returnType, that.returnType)
+        return equalsWithoutTypesAndFunctions(that)
+            && argTypes.equals(that.argTypes)
+            && returnType.equals(that.returnType)
             && Objects.equal(stateFunction, that.stateFunction)
             && Objects.equal(finalFunction, that.finalFunction)
-            && ((stateType == that.stateType) || ((stateType != null) && stateType.equals(that.stateType, true)))  // ignore freezing
-            && Objects.equal(initcond, that.initcond);
+            && ((stateType == that.stateType) || ((stateType != null) && stateType.equals(that.stateType)));
+    }
+
+    private boolean equalsWithoutTypesAndFunctions(UDAggregate other)
+    {
+        return name.equals(other.name)
+            && argTypes.size() == other.argTypes.size()
+            && Objects.equal(initcond, other.initcond);
+    }
+
+    @Override
+    public Optional<Difference> compare(Function function)
+    {
+        if (!(function instanceof UDAggregate))
+            throw new IllegalArgumentException();
+
+        UDAggregate other = (UDAggregate) function;
+
+        if (!equalsWithoutTypesAndFunctions(other)
+        || ((null == finalFunction) != (null == other.finalFunction))
+        || ((null == stateType) != (null == other.stateType)))
+            return Optional.of(Difference.SHALLOW);
+
+        boolean differsDeeply = false;
+
+        if (null != finalFunction && !finalFunction.equals(other.finalFunction))
+        {
+            if (finalFunction.name().equals(other.finalFunction.name()))
+                differsDeeply = true;
+            else
+                return Optional.of(Difference.SHALLOW);
+        }
+
+        if (null != stateType && !stateType.equals(other.stateType))
+        {
+            if (stateType.asCQL3Type().toString().equals(other.stateType.asCQL3Type().toString()))
+                differsDeeply = true;
+            else
+                return Optional.of(Difference.SHALLOW);
+        }
+
+        if (!returnType.equals(other.returnType))
+        {
+            if (returnType.asCQL3Type().toString().equals(other.returnType.asCQL3Type().toString()))
+                differsDeeply = true;
+            else
+                return Optional.of(Difference.SHALLOW);
+        }
+
+        for (int i = 0; i < argTypes().size(); i++)
+        {
+            AbstractType<?> thisType = argTypes.get(i);
+            AbstractType<?> thatType = other.argTypes.get(i);
+
+            if (!thisType.equals(thatType))
+            {
+                if (thisType.asCQL3Type().toString().equals(thatType.asCQL3Type().toString()))
+                    differsDeeply = true;
+                else
+                    return Optional.of(Difference.SHALLOW);
+            }
+        }
+
+        if (!stateFunction.equals(other.stateFunction))
+        {
+            if (stateFunction.name().equals(other.stateFunction.name()))
+                differsDeeply = true;
+            else
+                return Optional.of(Difference.SHALLOW);
+        }
+
+        return differsDeeply ? Optional.of(Difference.DEEP) : Optional.empty();
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/cql3/functions/UDFunction.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/functions/UDFunction.java b/src/java/org/apache/cassandra/cql3/functions/UDFunction.java
index b6fedcc..8c5a07e 100644
--- a/src/java/org/apache/cassandra/cql3/functions/UDFunction.java
+++ b/src/java/org/apache/cassandra/cql3/functions/UDFunction.java
@@ -27,6 +27,7 @@ import java.util.Collections;
 import java.util.Enumeration;
 import java.util.HashSet;
 import java.util.List;
+import java.util.Optional;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutionException;
@@ -37,6 +38,7 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 
 import com.google.common.base.Objects;
+import com.google.common.collect.Lists;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -44,6 +46,8 @@ import com.datastax.driver.core.DataType;
 import com.datastax.driver.core.TypeCodec;
 import org.apache.cassandra.config.Config;
 import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.marshal.UserType;
+import org.apache.cassandra.schema.Difference;
 import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.cql3.ColumnIdentifier;
 import org.apache.cassandra.db.marshal.AbstractType;
@@ -56,6 +60,9 @@ import org.apache.cassandra.tracing.Tracing;
 import org.apache.cassandra.transport.ProtocolVersion;
 import org.apache.cassandra.utils.JVMStabilityInspector;
 
+import static com.google.common.collect.Iterables.any;
+import static com.google.common.collect.Iterables.transform;
+
 /**
  * Base class for User Defined Functions.
  */
@@ -214,6 +221,24 @@ public abstract class UDFunction extends AbstractFunction implements ScalarFunct
                                              keyspaceMetadata);
     }
 
+    public static UDFunction tryCreate(FunctionName name,
+                                       List<ColumnIdentifier> argNames,
+                                       List<AbstractType<?>> argTypes,
+                                       AbstractType<?> returnType,
+                                       boolean calledOnNullInput,
+                                       String language,
+                                       String body)
+    {
+        try
+        {
+            return create(name, argNames, argTypes, returnType, calledOnNullInput, language, body);
+        }
+        catch (InvalidRequestException e)
+        {
+            return createBrokenFunction(name, argNames, argTypes, returnType, calledOnNullInput, language, body, e);
+        }
+    }
+
     public static UDFunction create(FunctionName name,
                                     List<ColumnIdentifier> argNames,
                                     List<AbstractType<?>> argTypes,
@@ -222,7 +247,7 @@ public abstract class UDFunction extends AbstractFunction implements ScalarFunct
                                     String language,
                                     String body)
     {
-        UDFunction.assertUdfsEnabled(language);
+        assertUdfsEnabled(language);
 
         switch (language)
         {
@@ -399,7 +424,7 @@ public abstract class UDFunction extends AbstractFunction implements ScalarFunct
     }
 
     /**
-     * Like {@link #executeAsync(int, List)} but the first parameter is already in non-serialized form.
+     * Like {@link #executeAsync(ProtocolVersion, List)} but the first parameter is already in non-serialized form.
      * Remaining parameters (2nd paramters and all others) are in {@code parameters}.
      * This is used to prevent superfluous (de)serialization of the state of aggregates.
      * Means: scalar functions of aggregates are called using this variant.
@@ -582,18 +607,83 @@ public abstract class UDFunction extends AbstractFunction implements ScalarFunct
     }
 
     @Override
+    public boolean referencesUserType(ByteBuffer name)
+    {
+        return any(argTypes(), t -> t.referencesUserType(name)) || returnType.referencesUserType(name);
+    }
+
+    public UDFunction withUpdatedUserType(UserType udt)
+    {
+        if (!referencesUserType(udt.name))
+            return this;
+
+        return tryCreate(name,
+                         argNames,
+                         Lists.newArrayList(transform(argTypes, t -> t.withUpdatedUserType(udt))),
+                         returnType.withUpdatedUserType(udt),
+                         calledOnNullInput,
+                         language,
+                         body);
+    }
+
+    @Override
     public boolean equals(Object o)
     {
         if (!(o instanceof UDFunction))
             return false;
 
         UDFunction that = (UDFunction)o;
-        return Objects.equal(name, that.name)
-            && Objects.equal(argNames, that.argNames)
-            && Functions.typesMatch(argTypes, that.argTypes)
-            && Functions.typesMatch(returnType, that.returnType)
-            && Objects.equal(language, that.language)
-            && Objects.equal(body, that.body);
+        return equalsWithoutTypes(that)
+            && argTypes.equals(that.argTypes)
+            && returnType.equals(that.returnType);
+    }
+
+    private boolean equalsWithoutTypes(UDFunction other)
+    {
+        return name.equals(other.name)
+            && argTypes.size() == other.argTypes.size()
+            && argNames.equals(other.argNames)
+            && body.equals(other.body)
+            && language.equals(other.language)
+            && calledOnNullInput == other.calledOnNullInput;
+    }
+
+    @Override
+    public Optional<Difference> compare(Function function)
+    {
+        if (!(function instanceof UDFunction))
+            throw new IllegalArgumentException();
+
+        UDFunction other = (UDFunction) function;
+
+        if (!equalsWithoutTypes(other))
+            return Optional.of(Difference.SHALLOW);
+
+        boolean typesDifferDeeply = false;
+
+        if (!returnType.equals(other.returnType))
+        {
+            if (returnType.asCQL3Type().toString().equals(other.returnType.asCQL3Type().toString()))
+                typesDifferDeeply = true;
+            else
+                return Optional.of(Difference.SHALLOW);
+        }
+
+        for (int i = 0; i < argTypes().size(); i++)
+        {
+            AbstractType<?> thisType = argTypes.get(i);
+            AbstractType<?> thatType = other.argTypes.get(i);
+
+            if (!thisType.equals(thatType))
+            {
+                if (thisType.asCQL3Type().toString().equals(thatType.asCQL3Type().toString()))
+                    typesDifferDeeply = true;
+                else
+                    return Optional.of(Difference.SHALLOW);
+            }
+        }
+
+        return typesDifferDeeply ? Optional.of(Difference.DEEP) : Optional.empty();
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/cql3/restrictions/CustomIndexExpression.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/restrictions/CustomIndexExpression.java b/src/java/org/apache/cassandra/cql3/restrictions/CustomIndexExpression.java
index 8048862..539715c 100644
--- a/src/java/org/apache/cassandra/cql3/restrictions/CustomIndexExpression.java
+++ b/src/java/org/apache/cassandra/cql3/restrictions/CustomIndexExpression.java
@@ -15,26 +15,25 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.cassandra.cql3.restrictions;
 
-import org.apache.cassandra.schema.TableMetadata;
+import java.util.Objects;
+
 import org.apache.cassandra.cql3.*;
 import org.apache.cassandra.db.filter.RowFilter;
 import org.apache.cassandra.db.marshal.AbstractType;
-import org.apache.commons.lang3.builder.ToStringBuilder;
-import org.apache.commons.lang3.builder.ToStringStyle;
+import org.apache.cassandra.schema.TableMetadata;
 
 public class CustomIndexExpression
 {
     private final ColumnIdentifier valueColId = new ColumnIdentifier("custom index expression", false);
 
-    public final IndexName targetIndex;
+    public final QualifiedName targetIndex;
     public final Term.Raw valueRaw;
 
     private Term value;
 
-    public CustomIndexExpression(IndexName targetIndex, Term.Raw value)
+    public CustomIndexExpression(QualifiedName targetIndex, Term.Raw value)
     {
         this.targetIndex = targetIndex;
         this.valueRaw = value;
@@ -51,14 +50,33 @@ public class CustomIndexExpression
     {
         filter.addCustomIndexExpression(table,
                                         table.indexes
-                                             .get(targetIndex.getIdx())
+                                             .get(targetIndex.getName())
                                              .orElseThrow(() -> IndexRestrictions.indexNotFound(targetIndex, table)),
                                         value.bindAndGet(options));
     }
-    
+
     @Override
     public String toString()
     {
-        return ToStringBuilder.reflectionToString(this, ToStringStyle.SHORT_PREFIX_STYLE);
+        return String.format("expr(%s,%s)", targetIndex, valueRaw);
+    }
+
+    @Override
+    public int hashCode()
+    {
+        return Objects.hash(targetIndex, valueRaw);
+    }
+
+    @Override
+    public boolean equals(Object o)
+    {
+        if (this == o)
+            return true;
+
+        if (!(o instanceof CustomIndexExpression))
+            return false;
+
+        CustomIndexExpression cie = (CustomIndexExpression) o;
+        return targetIndex.equals(cie.targetIndex) && valueRaw.equals(cie.valueRaw);
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/cql3/restrictions/IndexRestrictions.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/restrictions/IndexRestrictions.java b/src/java/org/apache/cassandra/cql3/restrictions/IndexRestrictions.java
index ac2e58a..fd89d1b 100644
--- a/src/java/org/apache/cassandra/cql3/restrictions/IndexRestrictions.java
+++ b/src/java/org/apache/cassandra/cql3/restrictions/IndexRestrictions.java
@@ -21,8 +21,8 @@ package org.apache.cassandra.cql3.restrictions;
 import java.util.ArrayList;
 import java.util.List;
 
+import org.apache.cassandra.cql3.QualifiedName;
 import org.apache.cassandra.schema.TableMetadata;
-import org.apache.cassandra.cql3.IndexName;
 import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.commons.lang3.builder.ToStringBuilder;
 import org.apache.commons.lang3.builder.ToStringStyle;
@@ -63,24 +63,24 @@ public class IndexRestrictions
         return customExpressions;
     }
 
-    static InvalidRequestException invalidIndex(IndexName indexName, TableMetadata table)
+    static InvalidRequestException invalidIndex(QualifiedName indexName, TableMetadata table)
     {
-        return new InvalidRequestException(String.format(INVALID_INDEX, indexName.getIdx(), table.toString()));
+        return new InvalidRequestException(String.format(INVALID_INDEX, indexName.getName(), table));
     }
 
-    static InvalidRequestException indexNotFound(IndexName indexName, TableMetadata table)
+    static InvalidRequestException indexNotFound(QualifiedName indexName, TableMetadata table)
     {
-        return new InvalidRequestException(String.format(INDEX_NOT_FOUND, indexName.getIdx(), table.toString()));
+        return new InvalidRequestException(String.format(INDEX_NOT_FOUND, indexName.getName(), table));
     }
 
-    static InvalidRequestException nonCustomIndexInExpression(IndexName indexName)
+    static InvalidRequestException nonCustomIndexInExpression(QualifiedName indexName)
     {
-        return new InvalidRequestException(String.format(NON_CUSTOM_INDEX_IN_EXPRESSION, indexName.getIdx()));
+        return new InvalidRequestException(String.format(NON_CUSTOM_INDEX_IN_EXPRESSION, indexName.getName()));
     }
 
-    static InvalidRequestException customExpressionNotSupported(IndexName indexName)
+    static InvalidRequestException customExpressionNotSupported(QualifiedName indexName)
     {
-        return new InvalidRequestException(String.format(CUSTOM_EXPRESSION_NOT_SUPPORTED, indexName.getIdx()));
+        return new InvalidRequestException(String.format(CUSTOM_EXPRESSION_NOT_SUPPORTED, indexName.getName()));
     }
     
     @Override

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/cql3/restrictions/StatementRestrictions.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/restrictions/StatementRestrictions.java b/src/java/org/apache/cassandra/cql3/restrictions/StatementRestrictions.java
index af1a964..5a41da0 100644
--- a/src/java/org/apache/cassandra/cql3/restrictions/StatementRestrictions.java
+++ b/src/java/org/apache/cassandra/cql3/restrictions/StatementRestrictions.java
@@ -131,6 +131,22 @@ public final class StatementRestrictions
                                  boolean allowFiltering,
                                  boolean forView)
     {
+        this(type, table, whereClause, boundNames, selectsOnlyStaticColumns, type.allowUseOfSecondaryIndices(), allowFiltering, forView);
+    }
+
+    /*
+     * We want to override allowUseOfSecondaryIndices flag from the StatementType for MV statements
+     * to avoid initing the Keyspace and SecondaryIndexManager.
+     */
+    public StatementRestrictions(StatementType type,
+                                 TableMetadata table,
+                                 WhereClause whereClause,
+                                 VariableSpecifications boundNames,
+                                 boolean selectsOnlyStaticColumns,
+                                 boolean allowUseOfSecondaryIndices,
+                                 boolean allowFiltering,
+                                 boolean forView)
+    {
         this(type, table, allowFiltering);
 
         IndexRegistry indexRegistry = null;
@@ -153,8 +169,7 @@ public final class StatementRestrictions
                 if (!forView)
                     throw new InvalidRequestException("Unsupported restriction: " + relation);
 
-                for (ColumnMetadata def : relation.toRestriction(table, boundNames).getColumnDefs())
-                    this.notNullColumns.add(def);
+                this.notNullColumns.addAll(relation.toRestriction(table, boundNames).getColumnDefs());
             }
             else if (relation.isLIKE())
             {
@@ -178,7 +193,7 @@ public final class StatementRestrictions
         boolean hasQueriableClusteringColumnIndex = false;
         boolean hasQueriableIndex = false;
 
-        if (type.allowUseOfSecondaryIndices())
+        if (allowUseOfSecondaryIndices)
         {
             if (whereClause.containsCustomExpressions())
                 processCustomIndexExpressions(whereClause.expressions, boundNames, indexRegistry);
@@ -566,19 +581,15 @@ public final class StatementRestrictions
 
         CustomIndexExpression expression = expressions.get(0);
 
-        CFName cfName = expression.targetIndex.getCfName();
-        if (cfName.hasKeyspace()
-            && !expression.targetIndex.getKeyspace().equals(table.keyspace))
-            throw IndexRestrictions.invalidIndex(expression.targetIndex, table);
+        QualifiedName name = expression.targetIndex;
 
-        if (cfName.getColumnFamily() != null && !cfName.getColumnFamily().equals(table.name))
+        if (name.hasKeyspace() && !name.getKeyspace().equals(table.keyspace))
             throw IndexRestrictions.invalidIndex(expression.targetIndex, table);
 
-        if (!table.indexes.has(expression.targetIndex.getIdx()))
+        if (!table.indexes.has(expression.targetIndex.getName()))
             throw IndexRestrictions.indexNotFound(expression.targetIndex, table);
 
-        Index index = indexRegistry.getIndex(table.indexes.get(expression.targetIndex.getIdx()).get());
-
+        Index index = indexRegistry.getIndex(table.indexes.get(expression.targetIndex.getName()).get());
         if (!index.getIndexMetadata().isCustom())
             throw IndexRestrictions.nonCustomIndexInExpression(expression.targetIndex);
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/cql3/selection/Selectable.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/selection/Selectable.java b/src/java/org/apache/cassandra/cql3/selection/Selectable.java
index 998baca..220bb89 100644
--- a/src/java/org/apache/cassandra/cql3/selection/Selectable.java
+++ b/src/java/org/apache/cassandra/cql3/selection/Selectable.java
@@ -1296,6 +1296,12 @@ public interface Selectable extends AssignmentTestable
             {
                 return new WithElementSelection(selected.prepare(cfm), element);
             }
+
+            @Override
+            public String toString()
+            {
+                return String.format("%s[%s]", selected, element);
+            }
         }
     }
 
@@ -1379,6 +1385,12 @@ public interface Selectable extends AssignmentTestable
             {
                 return new WithSliceSelection(selected.prepare(cfm), from, to);
             }
+
+            @Override
+            public String toString()
+            {
+                return String.format("%s[%s..%s]", selected, from == null ? "" : from, to == null ? "" : to);
+            }
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/cql3/statements/AlterKeyspaceStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/AlterKeyspaceStatement.java b/src/java/org/apache/cassandra/cql3/statements/AlterKeyspaceStatement.java
deleted file mode 100644
index 00d2b94..0000000
--- a/src/java/org/apache/cassandra/cql3/statements/AlterKeyspaceStatement.java
+++ /dev/null
@@ -1,129 +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.statements;
-
-import org.apache.cassandra.audit.AuditLogEntryType;
-import org.apache.cassandra.auth.Permission;
-import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.exceptions.*;
-import org.apache.cassandra.locator.AbstractReplicationStrategy;
-import org.apache.cassandra.locator.LocalStrategy;
-import org.apache.cassandra.schema.KeyspaceMetadata;
-import org.apache.cassandra.schema.KeyspaceParams;
-import org.apache.cassandra.schema.MigrationManager;
-import org.apache.cassandra.schema.Schema;
-import org.apache.cassandra.schema.SchemaConstants;
-import org.apache.cassandra.service.ClientState;
-import org.apache.cassandra.service.ClientWarn;
-import org.apache.cassandra.service.QueryState;
-import org.apache.cassandra.service.StorageService;
-import org.apache.cassandra.transport.Event;
-import org.apache.commons.lang3.builder.ToStringBuilder;
-import org.apache.commons.lang3.builder.ToStringStyle;
-
-public class AlterKeyspaceStatement extends SchemaAlteringStatement
-{
-    private final String name;
-    private final KeyspaceAttributes attrs;
-
-    public AlterKeyspaceStatement(String name, KeyspaceAttributes attrs)
-    {
-        super();
-        this.name = name;
-        this.attrs = attrs;
-    }
-
-    @Override
-    public String keyspace()
-    {
-        return name;
-    }
-
-    public void checkAccess(ClientState state) throws UnauthorizedException, InvalidRequestException
-    {
-        state.hasKeyspaceAccess(name, Permission.ALTER);
-    }
-
-    public void validate(ClientState state) throws RequestValidationException
-    {
-        KeyspaceMetadata ksm = Schema.instance.getKeyspaceMetadata(name);
-        if (ksm == null)
-            throw new InvalidRequestException("Unknown keyspace " + name);
-        if (SchemaConstants.isLocalSystemKeyspace(ksm.name))
-            throw new InvalidRequestException("Cannot alter system keyspace");
-        if (ksm.isVirtual())
-            throw new InvalidRequestException("Cannot alter virtual keyspaces");
-
-        attrs.validate();
-
-        if (attrs.getReplicationStrategyClass() == null && !attrs.getReplicationOptions().isEmpty())
-            throw new ConfigurationException("Missing replication strategy class");
-
-        if (attrs.getReplicationStrategyClass() != null)
-        {
-            // The strategy is validated through KSMetaData.validate() in announceKeyspaceUpdate below.
-            // However, for backward compatibility with thrift, this doesn't validate unexpected options yet,
-            // so doing proper validation here.
-            KeyspaceParams params = attrs.asAlteredKeyspaceParams(ksm.params);
-            params.validate(name);
-            if (params.replication.klass.equals(LocalStrategy.class))
-                throw new ConfigurationException("Unable to use given strategy class: LocalStrategy is reserved for internal use.");
-            warnIfIncreasingRF(ksm, params);
-        }
-    }
-
-    private void warnIfIncreasingRF(KeyspaceMetadata ksm, KeyspaceParams params)
-    {
-        AbstractReplicationStrategy oldStrategy = AbstractReplicationStrategy.createReplicationStrategy(ksm.name,
-                                                                                                        ksm.params.replication.klass,
-                                                                                                        StorageService.instance.getTokenMetadata(),
-                                                                                                        DatabaseDescriptor.getEndpointSnitch(),
-                                                                                                        ksm.params.replication.options);
-        AbstractReplicationStrategy newStrategy = AbstractReplicationStrategy.createReplicationStrategy(keyspace(),
-                                                                                                        params.replication.klass,
-                                                                                                        StorageService.instance.getTokenMetadata(),
-                                                                                                        DatabaseDescriptor.getEndpointSnitch(),
-                                                                                                        params.replication.options);
-        if (newStrategy.getReplicationFactor() > oldStrategy.getReplicationFactor())
-            ClientWarn.instance.warn("When increasing replication factor you need to run a full (-full) repair to distribute the data.");
-    }
-
-    public Event.SchemaChange announceMigration(QueryState queryState, boolean isLocalOnly) throws RequestValidationException
-    {
-        KeyspaceMetadata oldKsm = Schema.instance.getKeyspaceMetadata(name);
-        // In the (very) unlikely case the keyspace was dropped since validate()
-        if (oldKsm == null)
-            throw new InvalidRequestException("Unknown keyspace " + name);
-
-        KeyspaceMetadata newKsm = oldKsm.withSwapped(attrs.asAlteredKeyspaceParams(oldKsm.params));
-        MigrationManager.announceKeyspaceUpdate(newKsm, isLocalOnly);
-        return new Event.SchemaChange(Event.SchemaChange.Change.UPDATED, keyspace());
-    }
-    
-    @Override
-    public String toString()
-    {
-        return ToStringBuilder.reflectionToString(this, ToStringStyle.SHORT_PREFIX_STYLE);
-    }
-
-    @Override
-    public AuditLogContext getAuditLogContext()
-    {
-        return new AuditLogContext(AuditLogEntryType.ALTER_KEYSPACE, keyspace(), null);
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/cql3/statements/AlterRoleStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/AlterRoleStatement.java b/src/java/org/apache/cassandra/cql3/statements/AlterRoleStatement.java
index 957ac97..7a748e8 100644
--- a/src/java/org/apache/cassandra/cql3/statements/AlterRoleStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/AlterRoleStatement.java
@@ -17,6 +17,7 @@
  */
 package org.apache.cassandra.cql3.statements;
 
+import org.apache.cassandra.audit.AuditLogContext;
 import org.apache.cassandra.audit.AuditLogEntryType;
 import org.apache.cassandra.auth.*;
 import org.apache.cassandra.auth.IRoleManager.Option;
@@ -58,13 +59,13 @@ public class AlterRoleStatement extends AuthenticationStatement
         if (opts.isEmpty() && dcPermissions == null)
             throw new InvalidRequestException("ALTER [ROLE|USER] can't be empty");
 
-        // validate login here before checkAccess to avoid leaking user existence to anonymous users.
+        // validate login here before authorize to avoid leaking user existence to anonymous users.
         state.ensureNotAnonymous();
         if (!DatabaseDescriptor.getRoleManager().isExistingRole(role))
             throw new InvalidRequestException(String.format("%s doesn't exist", role.getRoleName()));
     }
 
-    public void checkAccess(ClientState state) throws UnauthorizedException
+    public void authorize(ClientState state) throws UnauthorizedException
     {
         AuthenticatedUser user = state.getUser();
         boolean isSuper = user.isSuper();


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


[09/13] cassandra git commit: Make all DDL statements idempotent and not dependent on global state

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/cql3/statements/DropKeyspaceStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/DropKeyspaceStatement.java b/src/java/org/apache/cassandra/cql3/statements/DropKeyspaceStatement.java
deleted file mode 100644
index 2f302c0..0000000
--- a/src/java/org/apache/cassandra/cql3/statements/DropKeyspaceStatement.java
+++ /dev/null
@@ -1,92 +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.statements;
-
-import org.apache.cassandra.audit.AuditLogEntryType;
-import org.apache.cassandra.auth.Permission;
-import org.apache.cassandra.db.virtual.VirtualKeyspaceRegistry;
-import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.exceptions.InvalidRequestException;
-import org.apache.cassandra.exceptions.RequestValidationException;
-import org.apache.cassandra.exceptions.UnauthorizedException;
-import org.apache.cassandra.schema.MigrationManager;
-import org.apache.cassandra.schema.Schema;
-import org.apache.cassandra.service.ClientState;
-import org.apache.cassandra.service.QueryState;
-import org.apache.cassandra.transport.Event;
-import org.apache.commons.lang3.builder.ToStringBuilder;
-import org.apache.commons.lang3.builder.ToStringStyle;
-
-public class DropKeyspaceStatement extends SchemaAlteringStatement
-{
-    private final String keyspace;
-    private final boolean ifExists;
-
-    public DropKeyspaceStatement(String keyspace, boolean ifExists)
-    {
-        super();
-        this.keyspace = keyspace;
-        this.ifExists = ifExists;
-    }
-
-    public void checkAccess(ClientState state) throws UnauthorizedException, InvalidRequestException
-    {
-        state.hasKeyspaceAccess(keyspace, Permission.DROP);
-    }
-
-    public void validate(ClientState state) throws RequestValidationException
-    {
-        Schema.validateKeyspaceNotSystem(keyspace);
-    }
-
-    @Override
-    public String keyspace()
-    {
-        return keyspace;
-    }
-
-    public Event.SchemaChange announceMigration(QueryState queryState, boolean isLocalOnly) throws ConfigurationException
-    {
-        if (null != VirtualKeyspaceRegistry.instance.getKeyspaceNullable(keyspace))
-            throw new InvalidRequestException("Cannot drop virtual keyspaces");
-
-        try
-        {
-            MigrationManager.announceKeyspaceDrop(keyspace, isLocalOnly);
-            return new Event.SchemaChange(Event.SchemaChange.Change.DROPPED, keyspace());
-        }
-        catch(ConfigurationException e)
-        {
-            if (ifExists)
-                return null;
-            throw e;
-        }
-    }
-    
-    @Override
-    public String toString()
-    {
-        return ToStringBuilder.reflectionToString(this, ToStringStyle.SHORT_PREFIX_STYLE);
-    }
-
-    @Override
-    public AuditLogContext getAuditLogContext()
-    {
-        return new AuditLogContext(AuditLogEntryType.DROP_KEYSPACE, keyspace());
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/cql3/statements/DropRoleStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/DropRoleStatement.java b/src/java/org/apache/cassandra/cql3/statements/DropRoleStatement.java
index 88b8b1a..058ab01 100644
--- a/src/java/org/apache/cassandra/cql3/statements/DropRoleStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/DropRoleStatement.java
@@ -17,6 +17,7 @@
  */
 package org.apache.cassandra.cql3.statements;
 
+import org.apache.cassandra.audit.AuditLogContext;
 import org.apache.cassandra.audit.AuditLogEntryType;
 import org.apache.cassandra.auth.*;
 import org.apache.cassandra.config.DatabaseDescriptor;
@@ -38,7 +39,7 @@ public class DropRoleStatement extends AuthenticationStatement
         this.ifExists = ifExists;
     }
 
-    public void checkAccess(ClientState state) throws UnauthorizedException
+    public void authorize(ClientState state) throws UnauthorizedException
     {
         super.checkPermission(state, Permission.DROP, role);
 
@@ -52,7 +53,7 @@ public class DropRoleStatement extends AuthenticationStatement
 
     public void validate(ClientState state) throws RequestValidationException
     {
-        // validate login here before checkAccess to avoid leaking user existence to anonymous users.
+        // validate login here before authorize to avoid leaking user existence to anonymous users.
         state.ensureNotAnonymous();
 
         if (!ifExists && !DatabaseDescriptor.getRoleManager().isExistingRole(role))

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/cql3/statements/DropTableStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/DropTableStatement.java b/src/java/org/apache/cassandra/cql3/statements/DropTableStatement.java
deleted file mode 100644
index beb1002..0000000
--- a/src/java/org/apache/cassandra/cql3/statements/DropTableStatement.java
+++ /dev/null
@@ -1,122 +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.statements;
-
-import org.apache.cassandra.audit.AuditLogEntryType;
-import org.apache.cassandra.auth.Permission;
-import org.apache.cassandra.cql3.CFName;
-import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.exceptions.InvalidRequestException;
-import org.apache.cassandra.exceptions.UnauthorizedException;
-import org.apache.cassandra.schema.KeyspaceMetadata;
-import org.apache.cassandra.schema.MigrationManager;
-import org.apache.cassandra.schema.Schema;
-import org.apache.cassandra.schema.TableMetadata;
-import org.apache.cassandra.schema.ViewMetadata;
-import org.apache.cassandra.service.ClientState;
-import org.apache.cassandra.service.QueryState;
-import org.apache.cassandra.transport.Event;
-import org.apache.commons.lang3.builder.ToStringBuilder;
-import org.apache.commons.lang3.builder.ToStringStyle;
-
-public class DropTableStatement extends SchemaAlteringStatement
-{
-    private final boolean ifExists;
-
-    public DropTableStatement(CFName name, boolean ifExists)
-    {
-        super(name);
-        this.ifExists = ifExists;
-    }
-
-    public void checkAccess(ClientState state) throws UnauthorizedException, InvalidRequestException
-    {
-        try
-        {
-            state.hasColumnFamilyAccess(keyspace(), columnFamily(), Permission.DROP);
-        }
-        catch (InvalidRequestException e)
-        {
-            if (!ifExists)
-                throw e;
-        }
-    }
-
-    public void validate(ClientState state)
-    {
-        // validated in announceMigration()
-    }
-
-    public Event.SchemaChange announceMigration(QueryState queryState, boolean isLocalOnly) throws ConfigurationException
-    {
-        try
-        {
-            KeyspaceMetadata ksm = Schema.instance.getKeyspaceMetadata(keyspace());
-            if (ksm == null)
-                throw new ConfigurationException(String.format("Cannot drop table in unknown keyspace '%s'", keyspace()));
-            TableMetadata metadata = ksm.getTableOrViewNullable(columnFamily());
-            if (metadata != null)
-            {
-                if (metadata.isView())
-                    throw new InvalidRequestException("Cannot use DROP TABLE on Materialized View");
-
-                if (metadata.isVirtual())
-                    throw new InvalidRequestException("Cannot drop virtual tables");
-
-                boolean rejectDrop = false;
-                StringBuilder messageBuilder = new StringBuilder();
-                for (ViewMetadata def : ksm.views)
-                {
-                    if (def.baseTableId.equals(metadata.id))
-                    {
-                        if (rejectDrop)
-                            messageBuilder.append(',');
-                        rejectDrop = true;
-                        messageBuilder.append(def.name);
-                    }
-                }
-                if (rejectDrop)
-                {
-                    throw new InvalidRequestException(String.format("Cannot drop table when materialized views still depend on it (%s.{%s})",
-                                                                    keyspace(),
-                                                                    messageBuilder.toString()));
-                }
-            }
-            MigrationManager.announceTableDrop(keyspace(), columnFamily(), isLocalOnly);
-            return new Event.SchemaChange(Event.SchemaChange.Change.DROPPED, Event.SchemaChange.Target.TABLE, keyspace(), columnFamily());
-        }
-        catch (ConfigurationException e)
-        {
-            if (ifExists)
-                return null;
-            throw e;
-        }
-    }
-    
-    @Override
-    public String toString()
-    {
-        return ToStringBuilder.reflectionToString(this, ToStringStyle.SHORT_PREFIX_STYLE);
-    }
-
-    @Override
-    public AuditLogContext getAuditLogContext()
-    {
-        return new AuditLogContext(AuditLogEntryType.DROP_TABLE, keyspace(), cfName.getColumnFamily());
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/cql3/statements/DropTriggerStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/DropTriggerStatement.java b/src/java/org/apache/cassandra/cql3/statements/DropTriggerStatement.java
deleted file mode 100644
index 1d9e3e4..0000000
--- a/src/java/org/apache/cassandra/cql3/statements/DropTriggerStatement.java
+++ /dev/null
@@ -1,100 +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.statements;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.cassandra.audit.AuditLogEntryType;
-import org.apache.cassandra.cql3.CFName;
-import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.exceptions.InvalidRequestException;
-import org.apache.cassandra.exceptions.RequestValidationException;
-import org.apache.cassandra.exceptions.UnauthorizedException;
-import org.apache.cassandra.schema.MigrationManager;
-import org.apache.cassandra.schema.Schema;
-import org.apache.cassandra.schema.TableMetadata;
-import org.apache.cassandra.schema.Triggers;
-import org.apache.cassandra.service.ClientState;
-import org.apache.cassandra.service.QueryState;
-import org.apache.cassandra.transport.Event;
-import org.apache.commons.lang3.builder.ToStringBuilder;
-import org.apache.commons.lang3.builder.ToStringStyle;
-
-public class DropTriggerStatement extends SchemaAlteringStatement
-{
-    private static final Logger logger = LoggerFactory.getLogger(DropTriggerStatement.class);
-
-    private final String triggerName;
-
-    private final boolean ifExists;
-
-    public DropTriggerStatement(CFName name, String triggerName, boolean ifExists)
-    {
-        super(name);
-        this.triggerName = triggerName;
-        this.ifExists = ifExists;
-    }
-
-    public void checkAccess(ClientState state) throws UnauthorizedException
-    {
-        state.ensureIsSuper("Only superusers are allowed to perfrom DROP TRIGGER queries");
-    }
-
-    public void validate(ClientState state) throws RequestValidationException
-    {
-        Schema.instance.validateTable(keyspace(), columnFamily());
-    }
-
-    public Event.SchemaChange announceMigration(QueryState queryState, boolean isLocalOnly) throws ConfigurationException, InvalidRequestException
-    {
-        TableMetadata current = Schema.instance.getTableMetadata(keyspace(), columnFamily());
-        Triggers triggers = current.triggers;
-
-        if (!triggers.get(triggerName).isPresent())
-        {
-            if (ifExists)
-                return null;
-            else
-                throw new InvalidRequestException(String.format("Trigger %s was not found", triggerName));
-        }
-
-        logger.info("Dropping trigger with name {}", triggerName);
-
-        TableMetadata updated =
-            current.unbuild()
-                   .triggers(triggers.without(triggerName))
-                   .build();
-
-        MigrationManager.announceTableUpdate(updated, isLocalOnly);
-
-        return new Event.SchemaChange(Event.SchemaChange.Change.UPDATED, Event.SchemaChange.Target.TABLE, keyspace(), columnFamily());
-    }
-    
-    @Override
-    public String toString()
-    {
-        return ToStringBuilder.reflectionToString(this, ToStringStyle.SHORT_PREFIX_STYLE);
-    }
-
-    @Override
-    public AuditLogContext getAuditLogContext()
-    {
-        return new AuditLogContext(AuditLogEntryType.DROP_TRIGGER, keyspace(), triggerName);
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/cql3/statements/DropTypeStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/DropTypeStatement.java b/src/java/org/apache/cassandra/cql3/statements/DropTypeStatement.java
deleted file mode 100644
index cc4ca1c..0000000
--- a/src/java/org/apache/cassandra/cql3/statements/DropTypeStatement.java
+++ /dev/null
@@ -1,138 +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.statements;
-
-import org.apache.cassandra.audit.AuditLogEntryType;
-import org.apache.cassandra.auth.Permission;
-import org.apache.cassandra.cql3.*;
-import org.apache.cassandra.cql3.functions.Function;
-import org.apache.cassandra.db.marshal.*;
-import org.apache.cassandra.exceptions.*;
-import org.apache.cassandra.schema.ColumnMetadata;
-import org.apache.cassandra.schema.KeyspaceMetadata;
-import org.apache.cassandra.schema.MigrationManager;
-import org.apache.cassandra.schema.Schema;
-import org.apache.cassandra.schema.TableMetadata;
-import org.apache.cassandra.service.ClientState;
-import org.apache.cassandra.service.QueryState;
-import org.apache.cassandra.transport.Event;
-import org.apache.commons.lang3.builder.ToStringBuilder;
-import org.apache.commons.lang3.builder.ToStringStyle;
-
-public class DropTypeStatement extends SchemaAlteringStatement
-{
-    private final UTName name;
-    private final boolean ifExists;
-
-    public DropTypeStatement(UTName name, boolean ifExists)
-    {
-        this.name = name;
-        this.ifExists = ifExists;
-    }
-
-    @Override
-    public void prepareKeyspace(ClientState state) throws InvalidRequestException
-    {
-        if (!name.hasKeyspace())
-            name.setKeyspace(state.getKeyspace());
-    }
-
-    public void checkAccess(ClientState state) throws UnauthorizedException, InvalidRequestException
-    {
-        state.hasKeyspaceAccess(keyspace(), Permission.DROP);
-    }
-
-    public void validate(ClientState state) throws RequestValidationException
-    {
-        KeyspaceMetadata ksm = Schema.instance.getKeyspaceMetadata(name.getKeyspace());
-        if (ksm == null)
-        {
-            if (ifExists)
-                return;
-            else
-                throw new InvalidRequestException(String.format("Cannot drop type in unknown keyspace %s", name.getKeyspace()));
-        }
-
-        if (!ksm.types.get(name.getUserTypeName()).isPresent())
-        {
-            if (ifExists)
-                return;
-            else
-                throw new InvalidRequestException(String.format("No user type named %s exists.", name));
-        }
-
-        // We don't want to drop a type unless it's not used anymore (mainly because
-        // if someone drops a type and recreates one with the same name but different
-        // definition with the previous name still in use, things can get messy).
-        // We have two places to check: 1) other user type that can nest the one
-        // we drop and 2) existing tables referencing the type (maybe in a nested
-        // way).
-
-        for (Function function : ksm.functions)
-        {
-            if (function.returnType().referencesUserType(name.getStringTypeName()))
-                throw new InvalidRequestException(String.format("Cannot drop user type %s as it is still used by function %s", name, function));
-
-            for (AbstractType<?> argType : function.argTypes())
-                if (argType.referencesUserType(name.getStringTypeName()))
-                    throw new InvalidRequestException(String.format("Cannot drop user type %s as it is still used by function %s", name, function));
-        }
-
-        for (UserType ut : ksm.types)
-            if (!ut.name.equals(name.getUserTypeName()) && ut.referencesUserType(name.getStringTypeName()))
-                throw new InvalidRequestException(String.format("Cannot drop user type %s as it is still used by user type %s", name, ut.getNameAsString()));
-
-        for (TableMetadata table : ksm.tablesAndViews())
-            for (ColumnMetadata def : table.columns())
-                if (def.type.referencesUserType(name.getStringTypeName()))
-                    throw new InvalidRequestException(String.format("Cannot drop user type %s as it is still used by table %s", name, table.toString()));
-    }
-
-    @Override
-    public String keyspace()
-    {
-        return name.getKeyspace();
-    }
-
-    public Event.SchemaChange announceMigration(QueryState queryState, boolean isLocalOnly) throws InvalidRequestException, ConfigurationException
-    {
-        KeyspaceMetadata ksm = Schema.instance.getKeyspaceMetadata(name.getKeyspace());
-        if (ksm == null)
-            return null; // do not assert (otherwise IF EXISTS case fails)
-
-        UserType toDrop = ksm.types.getNullable(name.getUserTypeName());
-        // Can be null with ifExists
-        if (toDrop == null)
-            return null;
-
-        MigrationManager.announceTypeDrop(toDrop, isLocalOnly);
-        return new Event.SchemaChange(Event.SchemaChange.Change.DROPPED, Event.SchemaChange.Target.TYPE, keyspace(), name.getStringTypeName());
-    }
-
-    @Override
-    public String toString()
-    {
-        return ToStringBuilder.reflectionToString(this, ToStringStyle.SHORT_PREFIX_STYLE);
-    }
-
-    @Override
-    public AuditLogContext getAuditLogContext()
-    {
-        return new AuditLogContext(AuditLogEntryType.DROP_TYPE, keyspace(), name.getStringTypeName());
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/cql3/statements/DropViewStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/DropViewStatement.java b/src/java/org/apache/cassandra/cql3/statements/DropViewStatement.java
deleted file mode 100644
index 218807f..0000000
--- a/src/java/org/apache/cassandra/cql3/statements/DropViewStatement.java
+++ /dev/null
@@ -1,84 +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.statements;
-
-import org.apache.cassandra.audit.AuditLogEntryType;
-import org.apache.cassandra.auth.Permission;
-import org.apache.cassandra.cql3.CFName;
-import org.apache.cassandra.db.view.View;
-import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.exceptions.InvalidRequestException;
-import org.apache.cassandra.exceptions.UnauthorizedException;
-import org.apache.cassandra.schema.MigrationManager;
-import org.apache.cassandra.schema.TableMetadataRef;
-import org.apache.cassandra.service.ClientState;
-import org.apache.cassandra.service.QueryState;
-import org.apache.cassandra.transport.Event;
-import org.apache.commons.lang3.builder.ToStringBuilder;
-import org.apache.commons.lang3.builder.ToStringStyle;
-
-public class DropViewStatement extends SchemaAlteringStatement
-{
-    public final boolean ifExists;
-
-    public DropViewStatement(CFName cf, boolean ifExists)
-    {
-        super(cf);
-        this.ifExists = ifExists;
-    }
-
-    public void checkAccess(ClientState state) throws UnauthorizedException, InvalidRequestException
-    {
-        TableMetadataRef baseTable = View.findBaseTable(keyspace(), columnFamily());
-        if (baseTable != null)
-            state.hasColumnFamilyAccess(keyspace(), baseTable.name, Permission.ALTER);
-    }
-
-    public void validate(ClientState state)
-    {
-        // validated in findIndexedCf()
-    }
-
-    public Event.SchemaChange announceMigration(QueryState queryState, boolean isLocalOnly) throws InvalidRequestException, ConfigurationException
-    {
-        try
-        {
-            MigrationManager.announceViewDrop(keyspace(), columnFamily(), isLocalOnly);
-            return new Event.SchemaChange(Event.SchemaChange.Change.DROPPED, Event.SchemaChange.Target.TABLE, keyspace(), columnFamily());
-        }
-        catch (ConfigurationException e)
-        {
-            if (ifExists)
-                return null;
-            throw e;
-        }
-    }
-    
-    @Override
-    public String toString()
-    {
-        return ToStringBuilder.reflectionToString(this, ToStringStyle.SHORT_PREFIX_STYLE);
-    }
-
-    @Override
-    public AuditLogContext getAuditLogContext()
-    {
-        return new AuditLogContext(AuditLogEntryType.DROP_VIEW, keyspace(), cfName.getColumnFamily());
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/cql3/statements/GrantPermissionsStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/GrantPermissionsStatement.java b/src/java/org/apache/cassandra/cql3/statements/GrantPermissionsStatement.java
index f5ac5ca..3db20e3 100644
--- a/src/java/org/apache/cassandra/cql3/statements/GrantPermissionsStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/GrantPermissionsStatement.java
@@ -19,6 +19,7 @@ package org.apache.cassandra.cql3.statements;
 
 import java.util.Set;
 
+import org.apache.cassandra.audit.AuditLogContext;
 import org.apache.cassandra.audit.AuditLogEntryType;
 import org.apache.cassandra.auth.IResource;
 import org.apache.cassandra.auth.Permission;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/cql3/statements/GrantRoleStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/GrantRoleStatement.java b/src/java/org/apache/cassandra/cql3/statements/GrantRoleStatement.java
index f234237..d6240c5 100644
--- a/src/java/org/apache/cassandra/cql3/statements/GrantRoleStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/GrantRoleStatement.java
@@ -17,6 +17,7 @@
  */
 package org.apache.cassandra.cql3.statements;
 
+import org.apache.cassandra.audit.AuditLogContext;
 import org.apache.cassandra.audit.AuditLogEntryType;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.cql3.RoleName;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/cql3/statements/IndexPropDefs.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/IndexPropDefs.java b/src/java/org/apache/cassandra/cql3/statements/IndexPropDefs.java
deleted file mode 100644
index b8ce7ec..0000000
--- a/src/java/org/apache/cassandra/cql3/statements/IndexPropDefs.java
+++ /dev/null
@@ -1,76 +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.statements;
-
-import java.util.*;
-
-import org.apache.cassandra.exceptions.InvalidRequestException;
-import org.apache.cassandra.exceptions.RequestValidationException;
-import org.apache.cassandra.exceptions.SyntaxException;
-
-public class IndexPropDefs extends PropertyDefinitions
-{
-    public static final String KW_OPTIONS = "options";
-
-    public static final Set<String> keywords = new HashSet<>();
-    public static final Set<String> obsoleteKeywords = new HashSet<>();
-
-    public boolean isCustom;
-    public String customClass;
-
-    static
-    {
-        keywords.add(KW_OPTIONS);
-    }
-
-    public void validate() throws RequestValidationException
-    {
-        validate(keywords, obsoleteKeywords);
-
-        if (isCustom && customClass == null)
-            throw new InvalidRequestException("CUSTOM index requires specifiying the index class");
-
-        if (!isCustom && customClass != null)
-            throw new InvalidRequestException("Cannot specify index class for a non-CUSTOM index");
-
-        if (!isCustom && !properties.isEmpty())
-            throw new InvalidRequestException("Cannot specify options for a non-CUSTOM index");
-
-        if (getRawOptions().containsKey(IndexTarget.CUSTOM_INDEX_OPTION_NAME))
-            throw new InvalidRequestException(String.format("Cannot specify %s as a CUSTOM option",
-                                                            IndexTarget.CUSTOM_INDEX_OPTION_NAME));
-
-        if (getRawOptions().containsKey(IndexTarget.TARGET_OPTION_NAME))
-            throw new InvalidRequestException(String.format("Cannot specify %s as a CUSTOM option",
-                                                            IndexTarget.TARGET_OPTION_NAME));
-
-    }
-
-    public Map<String, String> getRawOptions() throws SyntaxException
-    {
-        Map<String, String> options = getMap(KW_OPTIONS);
-        return options == null ? Collections.<String, String>emptyMap() : options;
-    }
-
-    public Map<String, String> getOptions() throws SyntaxException
-    {
-        Map<String, String> options = new HashMap<>(getRawOptions());
-        options.put(IndexTarget.CUSTOM_INDEX_OPTION_NAME, customClass);
-        return options;
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/cql3/statements/IndexTarget.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/IndexTarget.java b/src/java/org/apache/cassandra/cql3/statements/IndexTarget.java
deleted file mode 100644
index 26ca3df..0000000
--- a/src/java/org/apache/cassandra/cql3/statements/IndexTarget.java
+++ /dev/null
@@ -1,135 +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.statements;
-
-import org.apache.cassandra.schema.ColumnMetadata;
-import org.apache.cassandra.schema.TableMetadata;
-import org.apache.cassandra.cql3.ColumnIdentifier;
-import org.apache.commons.lang3.builder.ToStringBuilder;
-import org.apache.commons.lang3.builder.ToStringStyle;
-
-public class IndexTarget
-{
-    public static final String TARGET_OPTION_NAME = "target";
-    public static final String CUSTOM_INDEX_OPTION_NAME = "class_name";
-
-    public final ColumnIdentifier column;
-    public final Type type;
-
-    public IndexTarget(ColumnIdentifier column, Type type)
-    {
-        this.column = column;
-        this.type = type;
-    }
-
-    public String asCqlString()
-    {
-        return type == Type.SIMPLE
-               ? column.toCQLString()
-               : String.format("%s(%s)", type.toString(), column.toCQLString());
-    }
-
-    public static class Raw
-    {
-        private final ColumnMetadata.Raw column;
-        private final Type type;
-
-        private Raw(ColumnMetadata.Raw column, Type type)
-        {
-            this.column = column;
-            this.type = type;
-        }
-
-        public static Raw simpleIndexOn(ColumnMetadata.Raw c)
-        {
-            return new Raw(c, Type.SIMPLE);
-        }
-
-        public static Raw valuesOf(ColumnMetadata.Raw c)
-        {
-            return new Raw(c, Type.VALUES);
-        }
-
-        public static Raw keysOf(ColumnMetadata.Raw c)
-        {
-            return new Raw(c, Type.KEYS);
-        }
-
-        public static Raw keysAndValuesOf(ColumnMetadata.Raw c)
-        {
-            return new Raw(c, Type.KEYS_AND_VALUES);
-        }
-
-        public static Raw fullCollection(ColumnMetadata.Raw c)
-        {
-            return new Raw(c, Type.FULL);
-        }
-
-        public IndexTarget prepare(TableMetadata table)
-        {
-            // Until we've prepared the target column, we can't be certain about the target type
-            // because (for backwards compatibility) an index on a collection's values uses the
-            // same syntax as an index on a regular column (i.e. the 'values' in
-            // 'CREATE INDEX on table(values(collection));' is optional). So we correct the target type
-            // when the target column is a collection & the target type is SIMPLE.
-            ColumnMetadata columnDef = column.prepare(table);
-            Type actualType = (type == Type.SIMPLE && columnDef.type.isCollection()) ? Type.VALUES : type;
-            return new IndexTarget(columnDef.name, actualType);
-        }
-    }
-
-    public enum Type
-    {
-        VALUES, KEYS, KEYS_AND_VALUES, FULL, SIMPLE;
-
-        public String toString()
-        {
-            switch (this)
-            {
-                case KEYS: return "keys";
-                case KEYS_AND_VALUES: return "entries";
-                case FULL: return "full";
-                case VALUES: return "values";
-                case SIMPLE: return "";
-                default: return "";
-            }
-        }
-
-        public static Type fromString(String s)
-        {
-            if ("".equals(s))
-                return SIMPLE;
-            else if ("values".equals(s))
-                return VALUES;
-            else if ("keys".equals(s))
-                return KEYS;
-            else if ("entries".equals(s))
-                return KEYS_AND_VALUES;
-            else if ("full".equals(s))
-                return FULL;
-
-            throw new AssertionError("Unrecognized index target type " + s);
-        }
-    }
-    
-    @Override
-    public String toString()
-    {
-        return ToStringBuilder.reflectionToString(this, ToStringStyle.SHORT_PREFIX_STYLE);
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/cql3/statements/KeyspaceAttributes.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/KeyspaceAttributes.java b/src/java/org/apache/cassandra/cql3/statements/KeyspaceAttributes.java
deleted file mode 100644
index d2280ce..0000000
--- a/src/java/org/apache/cassandra/cql3/statements/KeyspaceAttributes.java
+++ /dev/null
@@ -1,91 +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.statements;
-
-import java.util.*;
-
-import com.google.common.collect.ImmutableSet;
-
-import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.schema.KeyspaceParams;
-import org.apache.cassandra.schema.KeyspaceParams.Option;
-import org.apache.cassandra.schema.ReplicationParams;
-
-public final class KeyspaceAttributes extends PropertyDefinitions
-{
-    private static final Set<String> validKeywords;
-    private static final Set<String> obsoleteKeywords;
-
-    static
-    {
-        ImmutableSet.Builder<String> validBuilder = ImmutableSet.builder();
-        for (Option option : Option.values())
-            validBuilder.add(option.toString());
-        validKeywords = validBuilder.build();
-        obsoleteKeywords = ImmutableSet.of();
-    }
-
-    public void validate()
-    {
-        validate(validKeywords, obsoleteKeywords);
-
-        Map<String, String> replicationOptions = getAllReplicationOptions();
-        if (!replicationOptions.isEmpty() && !replicationOptions.containsKey(ReplicationParams.CLASS))
-            throw new ConfigurationException("Missing replication strategy class");
-    }
-
-    public String getReplicationStrategyClass()
-    {
-        return getAllReplicationOptions().get(ReplicationParams.CLASS);
-    }
-
-    public Map<String, String> getReplicationOptions()
-    {
-        Map<String, String> replication = new HashMap<>(getAllReplicationOptions());
-        replication.remove(ReplicationParams.CLASS);
-        return replication;
-    }
-
-    public Map<String, String> getAllReplicationOptions()
-    {
-        Map<String, String> replication = getMap(Option.REPLICATION.toString());
-        return replication == null
-             ? Collections.emptyMap()
-             : replication;
-    }
-
-    public KeyspaceParams asNewKeyspaceParams()
-    {
-        boolean durableWrites = getBoolean(Option.DURABLE_WRITES.toString(), KeyspaceParams.DEFAULT_DURABLE_WRITES);
-        return KeyspaceParams.create(durableWrites, getAllReplicationOptions());
-    }
-
-    public KeyspaceParams asAlteredKeyspaceParams(KeyspaceParams previous)
-    {
-        boolean durableWrites = getBoolean(Option.DURABLE_WRITES.toString(), previous.durableWrites);
-        ReplicationParams replication = getReplicationStrategyClass() == null
-                                      ? previous.replication
-                                      : ReplicationParams.fromMap(getAllReplicationOptions());
-        return new KeyspaceParams(durableWrites, replication);
-    }
-
-    public boolean hasOption(Option option)
-    {
-        return hasProperty(option.toString());
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/cql3/statements/ListPermissionsStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/ListPermissionsStatement.java b/src/java/org/apache/cassandra/cql3/statements/ListPermissionsStatement.java
index 80195c0..4b5aa60 100644
--- a/src/java/org/apache/cassandra/cql3/statements/ListPermissionsStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/ListPermissionsStatement.java
@@ -19,6 +19,7 @@ package org.apache.cassandra.cql3.statements;
 
 import java.util.*;
 
+import org.apache.cassandra.audit.AuditLogContext;
 import org.apache.cassandra.audit.AuditLogEntryType;
 import org.apache.cassandra.auth.*;
 import org.apache.cassandra.config.DatabaseDescriptor;
@@ -79,7 +80,7 @@ public class ListPermissionsStatement extends AuthorizationStatement
             throw new InvalidRequestException(String.format("%s doesn't exist", grantee));
    }
 
-    public void checkAccess(ClientState state)
+    public void authorize(ClientState state)
     {
         // checked in validate
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/cql3/statements/ListRolesStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/ListRolesStatement.java b/src/java/org/apache/cassandra/cql3/statements/ListRolesStatement.java
index 8b51c59..8a75f8a 100644
--- a/src/java/org/apache/cassandra/cql3/statements/ListRolesStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/ListRolesStatement.java
@@ -24,6 +24,7 @@ import java.util.Set;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
 
+import org.apache.cassandra.audit.AuditLogContext;
 import org.apache.cassandra.audit.AuditLogEntryType;
 import org.apache.cassandra.auth.*;
 import org.apache.cassandra.config.DatabaseDescriptor;
@@ -74,7 +75,7 @@ public class ListRolesStatement extends AuthorizationStatement
             throw new InvalidRequestException(String.format("%s doesn't exist", grantee));
     }
 
-    public void checkAccess(ClientState state) throws InvalidRequestException
+    public void authorize(ClientState state) throws InvalidRequestException
     {
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/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 65e1e2d..c388c48 100644
--- a/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
@@ -29,7 +29,6 @@ import org.apache.cassandra.auth.Permission;
 import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.schema.TableMetadata;
-import org.apache.cassandra.schema.ColumnMetadata.Raw;
 import org.apache.cassandra.schema.ViewMetadata;
 import org.apache.cassandra.cql3.*;
 import org.apache.cassandra.cql3.conditions.ColumnCondition;
@@ -77,7 +76,8 @@ public abstract class ModificationStatement implements CQLStatement
 
     protected final StatementType type;
 
-    private final int boundTerms;
+    protected final VariableSpecifications bindVariables;
+
     public final TableMetadata metadata;
     private final Attributes attrs;
 
@@ -94,7 +94,7 @@ public abstract class ModificationStatement implements CQLStatement
     private final RegularAndStaticColumns requiresRead;
 
     public ModificationStatement(StatementType type,
-                                 int boundTerms,
+                                 VariableSpecifications bindVariables,
                                  TableMetadata metadata,
                                  Operations operations,
                                  StatementRestrictions restrictions,
@@ -102,7 +102,7 @@ public abstract class ModificationStatement implements CQLStatement
                                  Attributes attrs)
     {
         this.type = type;
-        this.boundTerms = boundTerms;
+        this.bindVariables = bindVariables;
         this.metadata = metadata;
         this.restrictions = restrictions;
         this.operations = operations;
@@ -147,6 +147,19 @@ public abstract class ModificationStatement implements CQLStatement
         this.requiresRead = requiresReadBuilder.build();
     }
 
+    @Override
+    public List<ColumnSpecification> getBindVariables()
+    {
+        return bindVariables.getBindVariables();
+    }
+
+    @Override
+    public short[] getPartitionKeyBindVariableIndexes()
+    {
+        return bindVariables.getPartitionKeyBindVariableIndexes(metadata);
+    }
+
+    @Override
     public Iterable<Function> getFunctions()
     {
         List<Function> functions = new ArrayList<>();
@@ -179,11 +192,6 @@ public abstract class ModificationStatement implements CQLStatement
 
     public abstract void addUpdateForKey(PartitionUpdate.Builder updateBuilder, Slice slice, UpdateParameters params);
 
-    public int getBoundTerms()
-    {
-        return boundTerms;
-    }
-
     public String keyspace()
     {
         return metadata.keyspace;
@@ -224,28 +232,28 @@ public abstract class ModificationStatement implements CQLStatement
         return attrs.getTimeToLive(options, metadata);
     }
 
-    public void checkAccess(ClientState state) throws InvalidRequestException, UnauthorizedException
+    public void authorize(ClientState state) throws InvalidRequestException, UnauthorizedException
     {
-        state.hasColumnFamilyAccess(metadata, Permission.MODIFY);
+        state.ensureTablePermission(metadata, Permission.MODIFY);
 
         // CAS updates can be used to simulate a SELECT query, so should require Permission.SELECT as well.
         if (hasConditions())
-            state.hasColumnFamilyAccess(metadata, Permission.SELECT);
+            state.ensureTablePermission(metadata, Permission.SELECT);
 
         // MV updates need to get the current state from the table, and might update the views
         // Require Permission.SELECT on the base table, and Permission.MODIFY on the views
         Iterator<ViewMetadata> views = View.findAll(keyspace(), columnFamily()).iterator();
         if (views.hasNext())
         {
-            state.hasColumnFamilyAccess(metadata, Permission.SELECT);
+            state.ensureTablePermission(metadata, Permission.SELECT);
             do
             {
-                state.hasColumnFamilyAccess(views.next().metadata, Permission.MODIFY);
+                state.ensureTablePermission(views.next().metadata, Permission.MODIFY);
             } while (views.hasNext());
         }
 
         for (Function function : getFunctions())
-            state.ensureHasPermission(Permission.EXECUTE, function);
+            state.ensurePermission(Permission.EXECUTE, function);
     }
 
     public void validate(ClientState state) throws InvalidRequestException
@@ -597,7 +605,7 @@ public abstract class ModificationStatement implements CQLStatement
         return builder.build();
     }
 
-    public ResultMessage executeInternal(QueryState queryState, QueryOptions options) throws RequestValidationException, RequestExecutionException
+    public ResultMessage executeLocally(QueryState queryState, QueryOptions options) throws RequestValidationException, RequestExecutionException
     {
         return hasConditions()
                ? executeInternalWithCondition(queryState, options)
@@ -799,7 +807,7 @@ public abstract class ModificationStatement implements CQLStatement
         return builder.build();
     }
 
-    public static abstract class Parsed extends CFStatement
+    public static abstract class Parsed extends QualifiedStatement
     {
         protected final StatementType type;
         private final Attributes.Raw attrs;
@@ -807,7 +815,7 @@ public abstract class ModificationStatement implements CQLStatement
         private final boolean ifNotExists;
         private final boolean ifExists;
 
-        protected Parsed(CFName name,
+        protected Parsed(QualifiedName name,
                          StatementType type,
                          Attributes.Raw attrs,
                          List<Pair<ColumnMetadata.Raw, ColumnCondition.Raw>> conditions,
@@ -817,39 +825,36 @@ public abstract class ModificationStatement implements CQLStatement
             super(name);
             this.type = type;
             this.attrs = attrs;
-            this.conditions = conditions == null ? Collections.<Pair<ColumnMetadata.Raw, ColumnCondition.Raw>>emptyList() : conditions;
+            this.conditions = conditions == null ? Collections.emptyList() : conditions;
             this.ifNotExists = ifNotExists;
             this.ifExists = ifExists;
         }
 
-        public ParsedStatement.Prepared prepare()
+        public ModificationStatement prepare(ClientState state)
         {
-            VariableSpecifications boundNames = getBoundVariables();
-            ModificationStatement statement = prepare(boundNames);
-            TableMetadata metadata = Schema.instance.validateTable(keyspace(), columnFamily());
-            return new ParsedStatement.Prepared(statement, boundNames, boundNames.getPartitionKeyBindIndexes(metadata));
+            return prepare(bindVariables);
         }
 
-        public ModificationStatement prepare(VariableSpecifications boundNames)
+        public ModificationStatement prepare(VariableSpecifications bindVariables)
         {
-            TableMetadata metadata = Schema.instance.validateTable(keyspace(), columnFamily());
+            TableMetadata metadata = Schema.instance.validateTable(keyspace(), name());
 
-            Attributes preparedAttributes = attrs.prepare(keyspace(), columnFamily());
-            preparedAttributes.collectMarkerSpecification(boundNames);
+            Attributes preparedAttributes = attrs.prepare(keyspace(), name());
+            preparedAttributes.collectMarkerSpecification(bindVariables);
 
-            Conditions preparedConditions = prepareConditions(metadata, boundNames);
+            Conditions preparedConditions = prepareConditions(metadata, bindVariables);
 
-            return prepareInternal(metadata, boundNames, preparedConditions, preparedAttributes);
+            return prepareInternal(metadata, bindVariables, preparedConditions, preparedAttributes);
         }
 
         /**
          * Returns the column conditions.
          *
          * @param metadata the column family meta data
-         * @param boundNames the bound names
+         * @param bindVariables the bound names
          * @return the column conditions.
          */
-        private Conditions prepareConditions(TableMetadata metadata, VariableSpecifications boundNames)
+        private Conditions prepareConditions(TableMetadata metadata, VariableSpecifications bindVariables)
         {
             // To have both 'IF EXISTS'/'IF NOT EXISTS' and some other conditions doesn't make sense.
             // So far this is enforced by the parser, but let's assert it for sanity if ever the parse changes.
@@ -870,17 +875,17 @@ public abstract class ModificationStatement implements CQLStatement
             if (conditions.isEmpty())
                 return Conditions.EMPTY_CONDITION;
 
-            return prepareColumnConditions(metadata, boundNames);
+            return prepareColumnConditions(metadata, bindVariables);
         }
 
         /**
          * Returns the column conditions.
          *
          * @param metadata the column family meta data
-         * @param boundNames the bound names
+         * @param bindVariables the bound names
          * @return the column conditions.
          */
-        private ColumnConditions prepareColumnConditions(TableMetadata metadata, VariableSpecifications boundNames)
+        private ColumnConditions prepareColumnConditions(TableMetadata metadata, VariableSpecifications bindVariables)
         {
             checkNull(attrs.timestamp, "Cannot provide custom timestamp for conditional updates");
 
@@ -890,7 +895,7 @@ public abstract class ModificationStatement implements CQLStatement
             {
                 ColumnMetadata def = entry.left.prepare(metadata);
                 ColumnCondition condition = entry.right.prepare(keyspace(), def, metadata);
-                condition.collectMarkerSpecification(boundNames);
+                condition.collectMarkerSpecification(bindVariables);
 
                 checkFalse(def.isPrimaryKeyColumn(), "PRIMARY KEY column '%s' cannot have IF conditions", def.name);
                 builder.add(condition);
@@ -899,7 +904,7 @@ public abstract class ModificationStatement implements CQLStatement
         }
 
         protected abstract ModificationStatement prepareInternal(TableMetadata metadata,
-                                                                 VariableSpecifications boundNames,
+                                                                 VariableSpecifications bindVariables,
                                                                  Conditions conditions,
                                                                  Attributes attrs);
 
@@ -933,7 +938,7 @@ public abstract class ModificationStatement implements CQLStatement
          * @param rawId the raw <code>ColumnIdentifier</code>
          * @return the <code>ColumnMetadata</code> corresponding to the specified raw <code>ColumnIdentifier</code>
          */
-        protected static ColumnMetadata getColumnDefinition(TableMetadata metadata, Raw rawId)
+        protected static ColumnMetadata getColumnDefinition(TableMetadata metadata, ColumnMetadata.Raw rawId)
         {
             return rawId.prepare(metadata);
         }
@@ -942,7 +947,7 @@ public abstract class ModificationStatement implements CQLStatement
         {
             ImmutableList.Builder<Pair<ColumnMetadata.Raw, ColumnCondition.Raw>> builder = ImmutableList.builderWithExpectedSize(conditions.size());
 
-            for (Pair<Raw, ColumnCondition.Raw> condition : conditions)
+            for (Pair<ColumnMetadata.Raw, ColumnCondition.Raw> condition : conditions)
                 builder.add(Pair.create(condition.left, condition.right));
 
             return builder.build();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/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
deleted file mode 100644
index 34bfc3d..0000000
--- a/src/java/org/apache/cassandra/cql3/statements/ParsedStatement.java
+++ /dev/null
@@ -1,88 +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.statements;
-
-import java.util.*;
-
-import org.apache.cassandra.cql3.*;
-import org.apache.cassandra.cql3.functions.Function;
-import org.apache.cassandra.exceptions.RequestValidationException;
-import org.apache.cassandra.utils.*;
-
-public abstract class ParsedStatement
-{
-    private VariableSpecifications variables;
-
-    public VariableSpecifications getBoundVariables()
-    {
-        return variables;
-    }
-
-    // Used by the parser and preparable statement
-    public void setBoundVariables(List<ColumnIdentifier> boundNames)
-    {
-        this.variables = new VariableSpecifications(boundNames);
-    }
-
-    public void setBoundVariables(VariableSpecifications variables)
-    {
-        this.variables = variables;
-    }
-
-    public abstract Prepared prepare() throws RequestValidationException;
-
-    public static class Prepared
-    {
-        /**
-         * Contains the CQL statement source if the statement has been "regularly" perpared via
-         * {@link org.apache.cassandra.cql3.QueryProcessor#prepare(java.lang.String, org.apache.cassandra.service.ClientState)} /
-         * {@link QueryHandler#prepare(java.lang.String, org.apache.cassandra.service.ClientState, java.util.Map)}.
-         * Other usages of this class may or may not contain the CQL statement source.
-         */
-        public String rawCQLStatement;
-
-        public final MD5Digest resultMetadataId;
-        public final List<ColumnSpecification> boundNames;
-        public final CQLStatement statement;
-        public final short[] partitionKeyBindIndexes;
-
-        protected Prepared(CQLStatement statement, List<ColumnSpecification> boundNames, short[] partitionKeyBindIndexes)
-        {
-            this.statement = statement;
-            this.boundNames = boundNames;
-            this.partitionKeyBindIndexes = partitionKeyBindIndexes;
-            this.resultMetadataId = ResultSet.ResultMetadata.fromPrepared(this).getResultMetadataId();
-            this.rawCQLStatement = "";
-        }
-
-        public Prepared(CQLStatement statement, VariableSpecifications names, short[] partitionKeyBindIndexes)
-        {
-            this(statement, names.getSpecifications(), partitionKeyBindIndexes);
-        }
-
-        public Prepared(CQLStatement statement)
-        {
-            this(statement, Collections.<ColumnSpecification>emptyList(), null);
-        }
-    }
-
-    public Iterable<Function> getFunctions()
-    {
-        return Collections.emptyList();
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/cql3/statements/PermissionsManagementStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/PermissionsManagementStatement.java b/src/java/org/apache/cassandra/cql3/statements/PermissionsManagementStatement.java
index 6c21012..aa7e85b 100644
--- a/src/java/org/apache/cassandra/cql3/statements/PermissionsManagementStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/PermissionsManagementStatement.java
@@ -45,14 +45,14 @@ public abstract class PermissionsManagementStatement extends AuthorizationStatem
 
     public void validate(ClientState state) throws RequestValidationException
     {
-        // validate login here before checkAccess to avoid leaking user existence to anonymous users.
+        // validate login here before authorize to avoid leaking user existence to anonymous users.
         state.ensureNotAnonymous();
 
         if (!DatabaseDescriptor.getRoleManager().isExistingRole(grantee))
             throw new InvalidRequestException(String.format("Role %s doesn't exist", grantee.getRoleName()));
 
         // if a keyspace is omitted when GRANT/REVOKE ON TABLE <table>, we need to correct the resource.
-        // called both here and in checkAccess(), as in some cases we do not call the latter.
+        // called both here and in authorize(), as in some cases we do not call the latter.
         resource = maybeCorrectResource(resource, state);
 
         // altering permissions on builtin functions is not supported
@@ -66,17 +66,17 @@ public abstract class PermissionsManagementStatement extends AuthorizationStatem
             throw new InvalidRequestException(String.format("Resource %s doesn't exist", resource));
     }
 
-    public void checkAccess(ClientState state) throws UnauthorizedException
+    public void authorize(ClientState state) throws UnauthorizedException
     {
         // if a keyspace is omitted when GRANT/REVOKE ON TABLE <table>, we need to correct the resource.
         resource = maybeCorrectResource(resource, state);
 
         // check that the user has AUTHORIZE permission on the resource or its parents, otherwise reject GRANT/REVOKE.
-        state.ensureHasPermission(Permission.AUTHORIZE, resource);
+        state.ensurePermission(Permission.AUTHORIZE, resource);
 
         // check that the user has [a single permission or all in case of ALL] on the resource or its parents.
         for (Permission p : permissions)
-            state.ensureHasPermission(p, resource);
+            state.ensurePermission(p, resource);
     }
     
     @Override

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/cql3/statements/QualifiedStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/QualifiedStatement.java b/src/java/org/apache/cassandra/cql3/statements/QualifiedStatement.java
new file mode 100644
index 0000000..a9c1f19
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/statements/QualifiedStatement.java
@@ -0,0 +1,76 @@
+/*
+ * 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.statements;
+
+import org.apache.commons.lang.builder.ToStringBuilder;
+import org.apache.commons.lang.builder.ToStringStyle;
+
+import org.apache.cassandra.cql3.CQLStatement;
+import org.apache.cassandra.cql3.QualifiedName;
+import org.apache.cassandra.service.ClientState;
+
+/**
+ * Abstract class for statements that work on sub-keyspace level (tables, views, indexes, functions, etc.)
+ */
+public abstract class QualifiedStatement extends CQLStatement.Raw
+{
+    final QualifiedName qualifiedName;
+
+    QualifiedStatement(QualifiedName qualifiedName)
+    {
+        this.qualifiedName = qualifiedName;
+    }
+
+    public void setKeyspace(ClientState state)
+    {
+        if (!qualifiedName.hasKeyspace())
+        {
+            // XXX: We explicitly only want to call state.getKeyspace() in this case, as we don't want to throw
+            // if not logged in any keyspace but a keyspace is explicitly set on the statement. So don't move
+            // the call outside the 'if' or replace the method by 'setKeyspace(state.getKeyspace())'
+            qualifiedName.setKeyspace(state.getKeyspace(), true);
+        }
+    }
+
+    // Only for internal calls, use the version with ClientState for user queries. In particular, the
+    // version with ClientState throws an exception if the statement does not have keyspace set *and*
+    // ClientState has no keyspace
+    public void setKeyspace(String keyspace)
+    {
+        qualifiedName.setKeyspace(keyspace, true);
+    }
+
+    public String keyspace()
+    {
+        if (!qualifiedName.hasKeyspace())
+            throw new IllegalStateException("Statement must have keyspace set");
+
+        return qualifiedName.getKeyspace();
+    }
+
+    public String name()
+    {
+        return qualifiedName.getName();
+    }
+    
+    @Override
+    public String toString()
+    {
+        return ToStringBuilder.reflectionToString(this, ToStringStyle.SHORT_PREFIX_STYLE);
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/cql3/statements/RevokePermissionsStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/RevokePermissionsStatement.java b/src/java/org/apache/cassandra/cql3/statements/RevokePermissionsStatement.java
index eb528fc..57d0631 100644
--- a/src/java/org/apache/cassandra/cql3/statements/RevokePermissionsStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/RevokePermissionsStatement.java
@@ -19,6 +19,7 @@ package org.apache.cassandra.cql3.statements;
 
 import java.util.Set;
 
+import org.apache.cassandra.audit.AuditLogContext;
 import org.apache.cassandra.audit.AuditLogEntryType;
 import org.apache.cassandra.auth.IResource;
 import org.apache.cassandra.auth.Permission;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/cql3/statements/RevokeRoleStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/RevokeRoleStatement.java b/src/java/org/apache/cassandra/cql3/statements/RevokeRoleStatement.java
index c497121..651743f 100644
--- a/src/java/org/apache/cassandra/cql3/statements/RevokeRoleStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/RevokeRoleStatement.java
@@ -17,6 +17,7 @@
  */
 package org.apache.cassandra.cql3.statements;
 
+import org.apache.cassandra.audit.AuditLogContext;
 import org.apache.cassandra.audit.AuditLogEntryType;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.cql3.RoleName;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/cql3/statements/RoleManagementStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/RoleManagementStatement.java b/src/java/org/apache/cassandra/cql3/statements/RoleManagementStatement.java
index 2c920bf..a5274dd 100644
--- a/src/java/org/apache/cassandra/cql3/statements/RoleManagementStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/RoleManagementStatement.java
@@ -39,7 +39,7 @@ public abstract class RoleManagementStatement extends AuthenticationStatement
         this.grantee = RoleResource.role(grantee.getName());
     }
 
-    public void checkAccess(ClientState state) throws UnauthorizedException
+    public void authorize(ClientState state) throws UnauthorizedException
     {
         super.checkPermission(state, Permission.AUTHORIZE, role);
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/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
deleted file mode 100644
index 4a20451..0000000
--- a/src/java/org/apache/cassandra/cql3/statements/SchemaAlteringStatement.java
+++ /dev/null
@@ -1,123 +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.statements;
-
-import org.apache.cassandra.auth.AuthenticatedUser;
-import org.apache.cassandra.cql3.CFName;
-import org.apache.cassandra.cql3.CQLStatement;
-import org.apache.cassandra.cql3.QueryOptions;
-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.transport.Event;
-import org.apache.cassandra.transport.messages.ResultMessage;
-
-/**
- * Abstract class for statements that alter the schema.
- */
-public abstract class SchemaAlteringStatement extends CFStatement implements CQLStatement
-{
-    private final boolean isColumnFamilyLevel;
-
-    protected SchemaAlteringStatement()
-    {
-        super(null);
-        this.isColumnFamilyLevel = false;
-    }
-
-    protected SchemaAlteringStatement(CFName name)
-    {
-        super(name);
-        this.isColumnFamilyLevel = true;
-    }
-
-    public int getBoundTerms()
-    {
-        return 0;
-    }
-
-    @Override
-    public void prepareKeyspace(ClientState state) throws InvalidRequestException
-    {
-        if (isColumnFamilyLevel)
-            super.prepareKeyspace(state);
-    }
-
-    @Override
-    public Prepared prepare()
-    {
-        return new Prepared(this);
-    }
-
-    /**
-     * Schema alteration may result in a new database object (keyspace, table, role, function) being created capable of
-     * having permissions GRANTed on it. The creator of the object (the primary role assigned to the AuthenticatedUser
-     * performing the operation) is automatically granted ALL applicable permissions on the object. This is a hook for
-     * subclasses to override in order to perform that grant when the statement is executed.
-     */
-    protected void grantPermissionsToCreator(QueryState state)
-    {
-        // no-op by default
-    }
-
-    /**
-     * Announces the migration to other nodes in the cluster.
-     *
-     * @return the schema change event corresponding to the execution of this statement, or {@code null} if no schema change
-     * has occurred (when IF NOT EXISTS is used, for example)
-     *
-     * @throws RequestValidationException
-     */
-    protected abstract Event.SchemaChange announceMigration(QueryState queryState, boolean isLocalOnly) throws RequestValidationException;
-
-    public ResultMessage execute(QueryState state, QueryOptions options, long queryStartNanoTime) throws RequestValidationException
-    {
-        // If an IF [NOT] EXISTS clause was used, this may not result in an actual schema change.  To avoid doing
-        // extra work in the drivers to handle schema changes, we return an empty message in this case. (CASSANDRA-7600)
-        Event.SchemaChange ce = announceMigration(state, false);
-        if (ce == null)
-            return new ResultMessage.Void();
-
-        // when a schema alteration results in a new db object being created, we grant permissions on the new
-        // object to the user performing the request if:
-        // * the user is not anonymous
-        // * the configured IAuthorizer supports granting of permissions (not all do, AllowAllAuthorizer doesn't and
-        //   custom external implementations may not)
-        AuthenticatedUser user = state.getClientState().getUser();
-        if (user != null && !user.isAnonymous() && ce.change == Event.SchemaChange.Change.CREATED)
-        {
-            try
-            {
-                grantPermissionsToCreator(state);
-            }
-            catch (UnsupportedOperationException e)
-            {
-                // not a problem, grant is an optional method on IAuthorizer
-            }
-        }
-
-        return new ResultMessage.SchemaChange(ce);
-    }
-
-    public ResultMessage executeInternal(QueryState state, QueryOptions options)
-    {
-        Event.SchemaChange ce = announceMigration(state, true);
-        return ce == null ? new ResultMessage.Void() : new ResultMessage.SchemaChange(ce);
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/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 ef3db51..a5105f2 100644
--- a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
@@ -25,8 +25,8 @@ import com.google.common.base.MoreObjects;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.cassandra.audit.AuditLogContext;
 import org.apache.cassandra.audit.AuditLogEntryType;
-import org.apache.cassandra.audit.IAuditLogContext;
 import org.apache.cassandra.auth.Permission;
 import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.schema.Schema;
@@ -91,7 +91,7 @@ public class SelectStatement implements CQLStatement
 
     public static final int DEFAULT_PAGE_SIZE = 10000;
 
-    private final int boundTerms;
+    public final VariableSpecifications bindVariables;
     public final TableMetadata table;
     public final Parameters parameters;
     private final Selection selection;
@@ -120,7 +120,7 @@ public class SelectStatement implements CQLStatement
                                                                        false);
 
     public SelectStatement(TableMetadata table,
-                           int boundTerms,
+                           VariableSpecifications bindVariables,
                            Parameters parameters,
                            Selection selection,
                            StatementRestrictions restrictions,
@@ -131,7 +131,7 @@ public class SelectStatement implements CQLStatement
                            Term perPartitionLimit)
     {
         this.table = table;
-        this.boundTerms = boundTerms;
+        this.bindVariables = bindVariables;
         this.selection = selection;
         this.restrictions = restrictions;
         this.isReversed = isReversed;
@@ -142,6 +142,19 @@ public class SelectStatement implements CQLStatement
         this.perPartitionLimit = perPartitionLimit;
     }
 
+    @Override
+    public List<ColumnSpecification> getBindVariables()
+    {
+        return bindVariables.getBindVariables();
+    }
+
+    @Override
+    public short[] getPartitionKeyBindVariableIndexes()
+    {
+        return bindVariables.getPartitionKeyBindVariableIndexes(table);
+    }
+
+    @Override
     public Iterable<Function> getFunctions()
     {
         List<Function> functions = new ArrayList<>();
@@ -176,7 +189,7 @@ public class SelectStatement implements CQLStatement
     static SelectStatement forSelection(TableMetadata table, Selection selection)
     {
         return new SelectStatement(table,
-                                   0,
+                                   VariableSpecifications.empty(),
                                    defaultParameters,
                                    selection,
                                    StatementRestrictions.empty(StatementType.SELECT, table),
@@ -192,26 +205,21 @@ public class SelectStatement implements CQLStatement
         return selection.getResultMetadata();
     }
 
-    public int getBoundTerms()
-    {
-        return boundTerms;
-    }
-
-    public void checkAccess(ClientState state) throws InvalidRequestException, UnauthorizedException
+    public void authorize(ClientState state) throws InvalidRequestException, UnauthorizedException
     {
         if (table.isView())
         {
             TableMetadataRef baseTable = View.findBaseTable(keyspace(), columnFamily());
             if (baseTable != null)
-                state.hasColumnFamilyAccess(baseTable, Permission.SELECT);
+                state.ensureTablePermission(baseTable, Permission.SELECT);
         }
         else
         {
-            state.hasColumnFamilyAccess(table, Permission.SELECT);
+            state.ensureTablePermission(table, Permission.SELECT);
         }
 
         for (Function function : getFunctions())
-            state.ensureHasPermission(Permission.EXECUTE, function);
+            state.ensurePermission(Permission.EXECUTE, function);
     }
 
     public void validate(ClientState state) throws InvalidRequestException
@@ -418,7 +426,7 @@ public class SelectStatement implements CQLStatement
         return new ResultMessage.Rows(rset);
     }
 
-    public ResultMessage.Rows executeInternal(QueryState state, QueryOptions options) throws RequestExecutionException, RequestValidationException
+    public ResultMessage.Rows executeLocally(QueryState state, QueryOptions options) throws RequestExecutionException, RequestValidationException
     {
         return executeInternal(state, options, FBUtilities.nowInSeconds(), System.nanoTime());
     }
@@ -911,7 +919,7 @@ public class SelectStatement implements CQLStatement
         Collections.sort(cqlRows.rows, orderingComparator);
     }
 
-    public static class RawStatement extends CFStatement
+    public static class RawStatement extends QualifiedStatement
     {
         public final Parameters parameters;
         public final List<RawSelector> selectClause;
@@ -919,7 +927,8 @@ public class SelectStatement implements CQLStatement
         public final Term.Raw limit;
         public final Term.Raw perPartitionLimit;
 
-        public RawStatement(CFName cfName, Parameters parameters,
+        public RawStatement(QualifiedName cfName,
+                            Parameters parameters,
                             List<RawSelector> selectClause,
                             WhereClause whereClause,
                             Term.Raw limit,
@@ -933,20 +942,19 @@ public class SelectStatement implements CQLStatement
             this.perPartitionLimit = perPartitionLimit;
         }
 
-        public ParsedStatement.Prepared prepare() throws InvalidRequestException
+        public SelectStatement prepare(ClientState state)
         {
             return prepare(false);
         }
 
-        public ParsedStatement.Prepared prepare(boolean forView) throws InvalidRequestException
+        public SelectStatement prepare(boolean forView) throws InvalidRequestException
         {
-            TableMetadata table = Schema.instance.validateTable(keyspace(), columnFamily());
-            VariableSpecifications boundNames = getBoundVariables();
+            TableMetadata table = Schema.instance.validateTable(keyspace(), name());
 
             List<Selectable> selectables = RawSelector.toSelectables(selectClause, table);
             boolean containsOnlyStaticColumns = selectOnlyStaticColumns(table, selectables);
 
-            StatementRestrictions restrictions = prepareRestrictions(table, boundNames, containsOnlyStaticColumns, forView);
+            StatementRestrictions restrictions = prepareRestrictions(table, bindVariables, containsOnlyStaticColumns, forView);
 
             // If we order post-query, the sorted column needs to be in the ResultSet for sorting,
             // even if we don't ultimately ship them to the client (CASSANDRA-4911).
@@ -956,7 +964,7 @@ public class SelectStatement implements CQLStatement
 
             Selection selection = prepareSelection(table,
                                                    selectables,
-                                                   boundNames,
+                                                   bindVariables,
                                                    resultSetOrderingColumns,
                                                    restrictions);
 
@@ -989,18 +997,16 @@ public class SelectStatement implements CQLStatement
 
             checkNeedsFiltering(restrictions);
 
-            SelectStatement stmt = new SelectStatement(table,
-                                                       boundNames.size(),
-                                                       parameters,
-                                                       selection,
-                                                       restrictions,
-                                                       isReversed,
-                                                       aggregationSpec,
-                                                       orderingComparator,
-                                                       prepareLimit(boundNames, limit, keyspace(), limitReceiver()),
-                                                       prepareLimit(boundNames, perPartitionLimit, keyspace(), perPartitionLimitReceiver()));
-
-            return new ParsedStatement.Prepared(stmt, boundNames, boundNames.getPartitionKeyBindIndexes(table));
+            return new SelectStatement(table,
+                                       bindVariables,
+                                       parameters,
+                                       selection,
+                                       restrictions,
+                                       isReversed,
+                                       aggregationSpec,
+                                       orderingComparator,
+                                       prepareLimit(bindVariables, limit, keyspace(), limitReceiver()),
+                                       prepareLimit(bindVariables, perPartitionLimit, keyspace(), perPartitionLimitReceiver()));
         }
 
         private Selection prepareSelection(TableMetadata table,
@@ -1258,19 +1264,19 @@ public class SelectStatement implements CQLStatement
 
         private ColumnSpecification limitReceiver()
         {
-            return new ColumnSpecification(keyspace(), columnFamily(), new ColumnIdentifier("[limit]", true), Int32Type.instance);
+            return new ColumnSpecification(keyspace(), name(), new ColumnIdentifier("[limit]", true), Int32Type.instance);
         }
 
         private ColumnSpecification perPartitionLimitReceiver()
         {
-            return new ColumnSpecification(keyspace(), columnFamily(), new ColumnIdentifier("[per_partition_limit]", true), Int32Type.instance);
+            return new ColumnSpecification(keyspace(), name(), new ColumnIdentifier("[per_partition_limit]", true), Int32Type.instance);
         }
 
         @Override
         public String toString()
         {
             return MoreObjects.toStringHelper(this)
-                              .add("name", cfName)
+                              .add("name", qualifiedName)
                               .add("selectClause", selectClause)
                               .add("whereClause", whereClause)
                               .add("isDistinct", parameters.isDistinct)


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


[13/13] cassandra git commit: Make all DDL statements idempotent and not dependent on global state

Posted by al...@apache.org.
Make all DDL statements idempotent and not dependent on global state

patch by Aleksey Yeschenko; reviewed by Sam Tunnicliffe for
CASSANDRA-13426


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

Branch: refs/heads/trunk
Commit: 207c80c1fd63dfbd8ca7e615ec8002ee8983c5d6
Parents: d3a994b
Author: Aleksey Yeschenko <al...@apache.org>
Authored: Thu Nov 10 01:17:36 2016 +0000
Committer: Aleksey Yeshchenko <al...@apple.com>
Committed: Tue Jul 17 16:39:41 2018 +0100

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 NEWS.txt                                        |   3 +
 src/antlr/Cql.g                                 |  10 +-
 src/antlr/Parser.g                              | 324 +++++++-------
 .../apache/cassandra/audit/AuditLogContext.java |  42 ++
 .../apache/cassandra/audit/AuditLogManager.java |   8 +-
 .../cassandra/audit/IAuditLogContext.java       |  53 ---
 .../org/apache/cassandra/auth/AuthKeyspace.java |   2 +-
 .../cassandra/auth/AuthenticatedUser.java       |   2 +-
 .../cassandra/auth/CassandraAuthorizer.java     |  20 +-
 .../auth/CassandraNetworkAuthorizer.java        |   2 +-
 .../cassandra/auth/CassandraRoleManager.java    |   3 +-
 .../apache/cassandra/auth/FunctionResource.java |  10 +
 .../cassandra/auth/PasswordAuthenticator.java   |   2 +-
 .../cassandra/auth/jmx/AuthorizationProxy.java  |   2 +-
 src/java/org/apache/cassandra/cql3/CFName.java  |  39 --
 .../org/apache/cassandra/cql3/CQL3Type.java     |  97 ++--
 .../org/apache/cassandra/cql3/CQLStatement.java |  64 ++-
 .../CustomPayloadMirroringQueryHandler.java     |   3 +-
 .../org/apache/cassandra/cql3/IndexName.java    |  47 --
 .../cassandra/cql3/KeyspaceElementName.java     |  74 ----
 .../cassandra/cql3/MultiColumnRelation.java     |  23 +
 .../apache/cassandra/cql3/QualifiedName.java    | 116 +++++
 .../org/apache/cassandra/cql3/QueryHandler.java |  29 +-
 .../apache/cassandra/cql3/QueryProcessor.java   | 131 +++---
 .../org/apache/cassandra/cql3/ResultSet.java    |   9 +-
 .../cassandra/cql3/SingleColumnRelation.java    |  25 +-
 src/java/org/apache/cassandra/cql3/Term.java    |  12 +
 .../apache/cassandra/cql3/TokenRelation.java    |  20 +
 .../cassandra/cql3/VariableSpecifications.java  |  20 +-
 .../org/apache/cassandra/cql3/WhereClause.java  |  74 +++-
 .../cql3/functions/AbstractFunction.java        |   4 +-
 .../cassandra/cql3/functions/Function.java      |  10 +-
 .../cassandra/cql3/functions/UDAggregate.java   | 170 ++++---
 .../cassandra/cql3/functions/UDFunction.java    | 106 ++++-
 .../restrictions/CustomIndexExpression.java     |  36 +-
 .../cql3/restrictions/IndexRestrictions.java    |  18 +-
 .../restrictions/StatementRestrictions.java     |  33 +-
 .../cassandra/cql3/selection/Selectable.java    |  12 +
 .../cql3/statements/AlterKeyspaceStatement.java | 129 ------
 .../cql3/statements/AlterRoleStatement.java     |   5 +-
 .../cql3/statements/AlterTableStatement.java    | 317 -------------
 .../statements/AlterTableStatementColumn.java   |  74 ----
 .../cql3/statements/AlterTypeStatement.java     | 195 --------
 .../cql3/statements/AlterViewStatement.java     | 103 -----
 .../statements/AuthenticationStatement.java     |  18 +-
 .../cql3/statements/AuthorizationStatement.java |  17 +-
 .../cql3/statements/BatchStatement.java         |  80 ++--
 .../cassandra/cql3/statements/CFProperties.java |  64 ---
 .../cassandra/cql3/statements/CFStatement.java  |  74 ----
 .../statements/CreateAggregateStatement.java    | 281 ------------
 .../statements/CreateFunctionStatement.java     | 202 ---------
 .../cql3/statements/CreateIndexStatement.java   | 275 ------------
 .../statements/CreateKeyspaceStatement.java     | 152 -------
 .../cql3/statements/CreateRoleStatement.java    |   7 +-
 .../cql3/statements/CreateTableStatement.java   | 431 ------------------
 .../cql3/statements/CreateTriggerStatement.java | 116 -----
 .../cql3/statements/CreateTypeStatement.java    | 152 -------
 .../cql3/statements/CreateViewStatement.java    | 390 ----------------
 .../cql3/statements/DeleteStatement.java        |  15 +-
 .../cql3/statements/DropAggregateStatement.java | 167 -------
 .../cql3/statements/DropFunctionStatement.java  | 203 ---------
 .../cql3/statements/DropIndexStatement.java     | 131 ------
 .../cql3/statements/DropKeyspaceStatement.java  |  92 ----
 .../cql3/statements/DropRoleStatement.java      |   5 +-
 .../cql3/statements/DropTableStatement.java     | 122 -----
 .../cql3/statements/DropTriggerStatement.java   | 100 -----
 .../cql3/statements/DropTypeStatement.java      | 138 ------
 .../cql3/statements/DropViewStatement.java      |  84 ----
 .../statements/GrantPermissionsStatement.java   |   1 +
 .../cql3/statements/GrantRoleStatement.java     |   1 +
 .../cql3/statements/IndexPropDefs.java          |  76 ----
 .../cassandra/cql3/statements/IndexTarget.java  | 135 ------
 .../cql3/statements/KeyspaceAttributes.java     |  91 ----
 .../statements/ListPermissionsStatement.java    |   3 +-
 .../cql3/statements/ListRolesStatement.java     |   3 +-
 .../cql3/statements/ModificationStatement.java  |  83 ++--
 .../cql3/statements/ParsedStatement.java        |  88 ----
 .../PermissionsManagementStatement.java         |  10 +-
 .../cql3/statements/QualifiedStatement.java     |  76 ++++
 .../statements/RevokePermissionsStatement.java  |   1 +
 .../cql3/statements/RevokeRoleStatement.java    |   1 +
 .../statements/RoleManagementStatement.java     |   2 +-
 .../statements/SchemaAlteringStatement.java     | 123 ------
 .../cql3/statements/SelectStatement.java        |  82 ++--
 .../cql3/statements/TableAttributes.java        | 195 --------
 .../cql3/statements/TruncateStatement.java      |  32 +-
 .../cql3/statements/UpdateStatement.java        |  42 +-
 .../cassandra/cql3/statements/UseStatement.java |  16 +-
 .../schema/AlterKeyspaceStatement.java          | 114 +++++
 .../statements/schema/AlterSchemaStatement.java | 153 +++++++
 .../statements/schema/AlterTableStatement.java  | 441 +++++++++++++++++++
 .../statements/schema/AlterTypeStatement.java   | 229 ++++++++++
 .../statements/schema/AlterViewStatement.java   | 112 +++++
 .../schema/CreateAggregateStatement.java        | 329 ++++++++++++++
 .../schema/CreateFunctionStatement.java         | 250 +++++++++++
 .../statements/schema/CreateIndexStatement.java | 233 ++++++++++
 .../schema/CreateKeyspaceStatement.java         | 115 +++++
 .../statements/schema/CreateTableStatement.java | 348 +++++++++++++++
 .../schema/CreateTriggerStatement.java          | 120 +++++
 .../statements/schema/CreateTypeStatement.java  | 151 +++++++
 .../statements/schema/CreateViewStatement.java  | 413 +++++++++++++++++
 .../schema/DropAggregateStatement.java          | 174 ++++++++
 .../schema/DropFunctionStatement.java           | 182 ++++++++
 .../statements/schema/DropIndexStatement.java   | 110 +++++
 .../schema/DropKeyspaceStatement.java           |  83 ++++
 .../statements/schema/DropTableStatement.java   | 112 +++++
 .../statements/schema/DropTriggerStatement.java | 104 +++++
 .../statements/schema/DropTypeStatement.java    | 149 +++++++
 .../statements/schema/DropViewStatement.java    |  98 +++++
 .../cql3/statements/schema/IndexAttributes.java |  77 ++++
 .../cql3/statements/schema/IndexTarget.java     | 133 ++++++
 .../statements/schema/KeyspaceAttributes.java   |  85 ++++
 .../cql3/statements/schema/TableAttributes.java | 196 +++++++++
 src/java/org/apache/cassandra/db/Keyspace.java  |   9 +-
 .../org/apache/cassandra/db/SystemKeyspace.java |   2 +-
 .../org/apache/cassandra/db/TableCQLHelper.java |   2 +-
 .../db/marshal/AbstractCompositeType.java       |   6 -
 .../cassandra/db/marshal/AbstractType.java      |  36 +-
 .../cassandra/db/marshal/CollectionType.java    |   7 +-
 .../cassandra/db/marshal/CompositeType.java     |  42 +-
 .../db/marshal/DynamicCompositeType.java        |  44 +-
 .../apache/cassandra/db/marshal/ListType.java   |  43 +-
 .../apache/cassandra/db/marshal/MapType.java    |  37 +-
 .../cassandra/db/marshal/ReversedType.java      |  42 +-
 .../apache/cassandra/db/marshal/SetType.java    |  34 +-
 .../apache/cassandra/db/marshal/TupleType.java  |  26 +-
 .../apache/cassandra/db/marshal/UserType.java   |  83 ++--
 .../apache/cassandra/db/view/TableViews.java    |   2 +-
 src/java/org/apache/cassandra/db/view/View.java | 108 ++---
 .../apache/cassandra/db/view/ViewManager.java   |   6 +-
 .../hadoop/cql3/CqlBulkRecordWriter.java        |   2 +-
 .../cassandra/index/SecondaryIndexManager.java  |   2 +-
 .../apache/cassandra/index/TargetParser.java    |   2 +-
 .../index/internal/CassandraIndex.java          |   2 +-
 .../apache/cassandra/index/sasi/SASIIndex.java  |   2 +-
 .../cassandra/io/sstable/CQLSSTableWriter.java  |  46 +-
 .../apache/cassandra/repair/RepairRunnable.java |   3 +-
 .../repair/SystemDistributedKeyspace.java       |   2 +-
 .../apache/cassandra/schema/ColumnMetadata.java |  30 +-
 .../cassandra/schema/CompressionParams.java     |  22 +-
 src/java/org/apache/cassandra/schema/Diff.java  |  59 +++
 .../org/apache/cassandra/schema/Difference.java |  38 ++
 .../org/apache/cassandra/schema/Functions.java  | 146 ++++--
 .../apache/cassandra/schema/IndexMetadata.java  |  14 +-
 .../org/apache/cassandra/schema/Indexes.java    |  30 --
 .../cassandra/schema/KeyspaceMetadata.java      | 136 +++++-
 .../apache/cassandra/schema/KeyspaceParams.java |   4 +-
 .../org/apache/cassandra/schema/Keyspaces.java  |  85 +++-
 .../cassandra/schema/MigrationManager.java      | 219 ++-------
 .../org/apache/cassandra/schema/Schema.java     | 250 ++++++-----
 .../apache/cassandra/schema/SchemaKeyspace.java | 277 +++++-------
 .../cassandra/schema/SchemaTransformation.java  |  31 ++
 .../org/apache/cassandra/schema/TableId.java    |   2 +-
 .../apache/cassandra/schema/TableMetadata.java  | 145 ++++--
 .../cassandra/schema/TableMetadataRef.java      |   2 +-
 .../apache/cassandra/schema/TableParams.java    |   2 +-
 .../org/apache/cassandra/schema/Tables.java     |  91 +++-
 src/java/org/apache/cassandra/schema/Types.java |  74 +++-
 .../apache/cassandra/schema/ViewMetadata.java   | 135 +++---
 src/java/org/apache/cassandra/schema/Views.java | 117 +++--
 .../apache/cassandra/service/ClientState.java   |  74 ++--
 .../cassandra/service/StorageService.java       |   6 +-
 .../cassandra/tools/SSTableOfflineRelevel.java  |   2 +-
 .../apache/cassandra/tracing/TraceKeyspace.java |   2 +-
 .../org/apache/cassandra/transport/Event.java   |  13 +
 .../transport/messages/BatchMessage.java        |  28 +-
 .../transport/messages/ExecuteMessage.java      |  14 +-
 .../transport/messages/PrepareMessage.java      |   8 +-
 .../transport/messages/QueryMessage.java        |  10 +-
 .../transport/messages/ResultMessage.java       |   3 -
 .../cassandra/triggers/TriggerExecutor.java     |   8 +-
 .../test/microbench/MutationBench.java          |   2 +-
 .../unit/org/apache/cassandra/SchemaLoader.java |  16 +-
 .../auth/CassandraNetworkAuthorizerTest.java    |  10 +-
 .../org/apache/cassandra/cql3/CQLTester.java    |  20 +-
 .../cassandra/cql3/PstmtPersistenceTest.java    |   9 +-
 .../apache/cassandra/cql3/ViewComplexTest.java  |  12 +-
 .../cassandra/cql3/ViewFilteringTest.java       |   2 +-
 .../apache/cassandra/cql3/ViewSchemaTest.java   |   6 +-
 .../org/apache/cassandra/cql3/ViewTest.java     |  39 +-
 .../selection/SelectionColumnMappingTest.java   |   8 +-
 .../cql3/statements/AlterRoleStatementTest.java |   3 +-
 .../statements/CreateRoleStatementTest.java     |   3 +-
 .../statements/CreateUserStatementTest.java     |   3 +-
 .../validation/entities/CollectionsTest.java    |  10 +-
 .../cql3/validation/entities/CountersTest.java  |   2 +-
 .../entities/FrozenCollectionsTest.java         |   4 +-
 .../validation/entities/SecondaryIndexTest.java | 116 ++---
 .../cql3/validation/entities/TypeTest.java      |   4 +-
 .../cql3/validation/entities/UFAuthTest.java    |  32 +-
 .../entities/UFIdentificationTest.java          |   9 +-
 .../cql3/validation/entities/UFJavaTest.java    |   4 +-
 .../cql3/validation/entities/UFTest.java        |  53 ++-
 .../cql3/validation/entities/UFTypesTest.java   |  30 +-
 .../cql3/validation/entities/UserTypesTest.java |   6 +-
 .../validation/entities/VirtualTableTest.java   |  20 +-
 .../validation/operations/AggregationTest.java  | 109 ++---
 .../cql3/validation/operations/AlterTest.java   |  19 +-
 .../cql3/validation/operations/CreateTest.java  |  67 +--
 .../cql3/validation/operations/DropTest.java    |   4 +-
 .../operations/InsertUpdateIfConditionTest.java |   6 +-
 .../SelectSingleColumnRelationTest.java         |   2 +-
 .../cql3/validation/operations/SelectTest.java  |   4 +-
 .../apache/cassandra/db/DirectoriesTest.java    |   2 +-
 .../apache/cassandra/db/RangeTombstoneTest.java |   2 +-
 .../apache/cassandra/db/RowIndexEntryTest.java  |   2 +-
 .../apache/cassandra/db/SecondaryIndexTest.java |   2 +-
 .../apache/cassandra/db/TableCQLHelperTest.java |   4 +-
 .../compaction/AbstractPendingRepairTest.java   |   8 +-
 .../db/compaction/CompactionTaskTest.java       |   2 +-
 .../db/compaction/CompactionsPurgeTest.java     |   2 +-
 ...tionManagerGetSSTablesForValidationTest.java |   5 +-
 .../db/repair/PendingAntiCompactionTest.java    |   6 +-
 .../db/streaming/CassandraStreamHeaderTest.java |   2 +-
 .../streaming/CassandraStreamManagerTest.java   |   2 +-
 .../cassandra/hints/HintWriteTTLTest.java       |   2 +-
 .../apache/cassandra/index/CustomIndexTest.java |   8 +-
 .../index/internal/CassandraIndexTest.java      |   4 +-
 .../index/internal/CustomCassandraIndex.java    |   2 +-
 .../cassandra/index/sasi/SASIIndexTest.java     |   2 +-
 .../repair/StreamingRepairTaskTest.java         |   2 +-
 .../consistent/CoordinatorSessionsTest.java     |   4 +-
 .../repair/consistent/LocalSessionTest.java     |   4 +-
 .../cassandra/schema/IndexMetadataTest.java     |   7 +-
 .../cassandra/schema/MigrationManagerTest.java  |   2 +-
 .../cassandra/schema/SchemaKeyspaceTest.java    |   2 +-
 .../cassandra/service/QueryPagerTest.java       |   2 +-
 .../cassandra/transport/MessagePayloadTest.java |   3 +-
 .../cassandra/triggers/TriggersSchemaTest.java  |   2 +-
 .../io/sstable/StressCQLSSTableWriter.java      |  59 +--
 .../cassandra/stress/CompactionStress.java      |   4 +-
 .../apache/cassandra/stress/StressProfile.java  |  14 +-
 233 files changed, 7806 insertions(+), 7024 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index b66a1f6..63831d7 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 4.0
+ * Make all DDL statements idempotent and not dependent on global state (CASSANDRA-13426)
  * Bump the hints messaging version to match the current one (CASSANDRA-14536)
  * OffsetAwareConfigurationLoader doesn't set ssl storage port causing bind errors in CircleCI (CASSANDRA-14546)
  * Report why native_transport_port fails to bind (CASSANDRA-14544)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/NEWS.txt
----------------------------------------------------------------------
diff --git a/NEWS.txt b/NEWS.txt
index e1e76de..da94422 100644
--- a/NEWS.txt
+++ b/NEWS.txt
@@ -172,6 +172,9 @@ Materialized Views
      now be logged when they are created. (See https://www.mail-archive.com/dev@cassandra.apache.org/msg11511.html)
    - An 'enable_materialized_views' flag has been added to cassandra.yaml to allow operators to prevent creation of
      views
+   - CREATE MATERIALIZED VIEW syntax has become stricter. Partition key columns are no longer implicitly considered
+     to be NOT NULL, and no base primary key columns get automatically included in view definition. You have to
+     specify them explicitly now.
 
 3.11.3
 =====

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/antlr/Cql.g
----------------------------------------------------------------------
diff --git a/src/antlr/Cql.g b/src/antlr/Cql.g
index 17a11aa..272c63b 100644
--- a/src/antlr/Cql.g
+++ b/src/antlr/Cql.g
@@ -28,25 +28,21 @@ import Parser,Lexer;
 @header {
     package org.apache.cassandra.cql3;
 
-    import java.util.ArrayList;
-    import java.util.Arrays;
     import java.util.Collections;
     import java.util.EnumSet;
     import java.util.HashSet;
-    import java.util.HashMap;
     import java.util.LinkedHashMap;
     import java.util.List;
     import java.util.Map;
     import java.util.Set;
 
     import org.apache.cassandra.auth.*;
-    import org.apache.cassandra.cql3.*;
     import org.apache.cassandra.cql3.conditions.*;
     import org.apache.cassandra.cql3.functions.*;
     import org.apache.cassandra.cql3.restrictions.CustomIndexExpression;
     import org.apache.cassandra.cql3.selection.*;
     import org.apache.cassandra.cql3.statements.*;
-    import org.apache.cassandra.db.marshal.CollectionType;
+    import org.apache.cassandra.cql3.statements.schema.*;
     import org.apache.cassandra.exceptions.ConfigurationException;
     import org.apache.cassandra.exceptions.InvalidRequestException;
     import org.apache.cassandra.exceptions.SyntaxException;
@@ -95,8 +91,6 @@ import Parser,Lexer;
 
 @lexer::header {
     package org.apache.cassandra.cql3;
-
-    import org.apache.cassandra.exceptions.SyntaxException;
 }
 
 @lexer::members {
@@ -135,6 +129,6 @@ import Parser,Lexer;
     }
 }
 
-query returns [ParsedStatement stmnt]
+query returns [CQLStatement.Raw stmnt]
     : st=cqlStatement (';')* EOF { $stmnt = st; }
     ;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/antlr/Parser.g
----------------------------------------------------------------------
diff --git a/src/antlr/Parser.g b/src/antlr/Parser.g
index f32fd48..1cfc7d1 100644
--- a/src/antlr/Parser.g
+++ b/src/antlr/Parser.g
@@ -204,8 +204,8 @@ options {
 
 /** STATEMENTS **/
 
-cqlStatement returns [ParsedStatement stmt]
-    @after{ if (stmt != null) stmt.setBoundVariables(bindVariables); }
+cqlStatement returns [CQLStatement.Raw stmt]
+    @after{ if (stmt != null) stmt.setBindVariables(bindVariables); }
     : st1= selectStatement                 { $stmt = st1; }
     | st2= insertStatement                 { $stmt = st2; }
     | st3= updateStatement                 { $stmt = st3; }
@@ -436,8 +436,8 @@ selectionFunctionArgs returns [List<Selectable.Raw> a]
 
 sident returns [Selectable.Raw id]
     : t=IDENT              { $id = Selectable.RawIdentifier.forUnquoted($t.text); }
-    | t=QUOTED_NAME        { $id = ColumnMetadata.RawIdentifier.forQuoted($t.text); }
-    | k=unreserved_keyword { $id = ColumnMetadata.RawIdentifier.forUnquoted(k); }
+    | t=QUOTED_NAME        { $id = Selectable.RawIdentifier.forQuoted($t.text); }
+    | k=unreserved_keyword { $id = Selectable.RawIdentifier.forUnquoted(k); }
     ;
 
 whereClause returns [WhereClause.Builder clause]
@@ -451,7 +451,7 @@ relationOrExpression [WhereClause.Builder clause]
     ;
 
 customIndexExpression [WhereClause.Builder clause]
-    @init{IndexName name = new IndexName();}
+    @init{QualifiedName name = new QualifiedName();}
     : 'expr(' idxName[name] ',' t=term ')' { clause.add(new CustomIndexExpression(name, t));}
     ;
 
@@ -478,7 +478,7 @@ insertStatement returns [ModificationStatement.Parsed expr]
         | K_JSON st2=jsonInsertStatement[cf] { $expr = st2; })
     ;
 
-normalInsertStatement [CFName cf] returns [UpdateStatement.ParsedInsert expr]
+normalInsertStatement [QualifiedName qn] returns [UpdateStatement.ParsedInsert expr]
     @init {
         Attributes.Raw attrs = new Attributes.Raw();
         List<ColumnMetadata.Raw> columnNames  = new ArrayList<>();
@@ -491,11 +491,11 @@ normalInsertStatement [CFName cf] returns [UpdateStatement.ParsedInsert expr]
       ( K_IF K_NOT K_EXISTS { ifNotExists = true; } )?
       ( usingClause[attrs] )?
       {
-          $expr = new UpdateStatement.ParsedInsert(cf, attrs, columnNames, values, ifNotExists);
+          $expr = new UpdateStatement.ParsedInsert(qn, attrs, columnNames, values, ifNotExists);
       }
     ;
 
-jsonInsertStatement [CFName cf] returns [UpdateStatement.ParsedInsertJson expr]
+jsonInsertStatement [QualifiedName qn] returns [UpdateStatement.ParsedInsertJson expr]
     @init {
         Attributes.Raw attrs = new Attributes.Raw();
         boolean ifNotExists = false;
@@ -506,7 +506,7 @@ jsonInsertStatement [CFName cf] returns [UpdateStatement.ParsedInsertJson expr]
       ( K_IF K_NOT K_EXISTS { ifNotExists = true; } )?
       ( usingClause[attrs] )?
       {
-          $expr = new UpdateStatement.ParsedInsertJson(cf, attrs, val, defaultUnset, ifNotExists);
+          $expr = new UpdateStatement.ParsedInsertJson(qn, attrs, val, defaultUnset, ifNotExists);
       }
     ;
 
@@ -649,12 +649,12 @@ batchStatementObjective returns [ModificationStatement.Parsed statement]
     | d=deleteStatement  { $statement = d; }
     ;
 
-createAggregateStatement returns [CreateAggregateStatement expr]
+createAggregateStatement returns [CreateAggregateStatement.Raw stmt]
     @init {
         boolean orReplace = false;
         boolean ifNotExists = false;
 
-        List<CQL3Type.Raw> argsTypes = new ArrayList<>();
+        List<CQL3Type.Raw> argTypes = new ArrayList<>();
     }
     : K_CREATE (K_OR K_REPLACE { orReplace = true; })?
       K_AGGREGATE
@@ -662,8 +662,8 @@ createAggregateStatement returns [CreateAggregateStatement expr]
       fn=functionName
       '('
         (
-          v=comparatorType { argsTypes.add(v); }
-          ( ',' v=comparatorType { argsTypes.add(v); } )*
+          v=comparatorType { argTypes.add(v); }
+          ( ',' v=comparatorType { argTypes.add(v); } )*
         )?
       ')'
       K_SFUNC sfunc = allowedFunctionName
@@ -674,14 +674,14 @@ createAggregateStatement returns [CreateAggregateStatement expr]
       (
         K_INITCOND ival = term
       )?
-      { $expr = new CreateAggregateStatement(fn, argsTypes, sfunc, stype, ffunc, ival, orReplace, ifNotExists); }
+      { $stmt = new CreateAggregateStatement.Raw(fn, argTypes, stype, sfunc, ffunc, ival, orReplace, ifNotExists); }
     ;
 
-dropAggregateStatement returns [DropAggregateStatement expr]
+dropAggregateStatement returns [DropAggregateStatement.Raw stmt]
     @init {
         boolean ifExists = false;
-        List<CQL3Type.Raw> argsTypes = new ArrayList<>();
-        boolean argsPresent = false;
+        List<CQL3Type.Raw> argTypes = new ArrayList<>();
+        boolean argsSpecified = false;
     }
     : K_DROP K_AGGREGATE
       (K_IF K_EXISTS { ifExists = true; } )?
@@ -689,22 +689,22 @@ dropAggregateStatement returns [DropAggregateStatement expr]
       (
         '('
           (
-            v=comparatorType { argsTypes.add(v); }
-            ( ',' v=comparatorType { argsTypes.add(v); } )*
+            v=comparatorType { argTypes.add(v); }
+            ( ',' v=comparatorType { argTypes.add(v); } )*
           )?
         ')'
-        { argsPresent = true; }
+        { argsSpecified = true; }
       )?
-      { $expr = new DropAggregateStatement(fn, argsTypes, argsPresent, ifExists); }
+      { $stmt = new DropAggregateStatement.Raw(fn, argTypes, argsSpecified, ifExists); }
     ;
 
-createFunctionStatement returns [CreateFunctionStatement expr]
+createFunctionStatement returns [CreateFunctionStatement.Raw stmt]
     @init {
         boolean orReplace = false;
         boolean ifNotExists = false;
 
-        List<ColumnIdentifier> argsNames = new ArrayList<>();
-        List<CQL3Type.Raw> argsTypes = new ArrayList<>();
+        List<ColumnIdentifier> argNames = new ArrayList<>();
+        List<CQL3Type.Raw> argTypes = new ArrayList<>();
         boolean calledOnNullInput = false;
     }
     : K_CREATE (K_OR K_REPLACE { orReplace = true; })?
@@ -713,23 +713,24 @@ createFunctionStatement returns [CreateFunctionStatement expr]
       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=noncol_ident v=comparatorType { argNames.add(k); argTypes.add(v); }
+          ( ',' k=noncol_ident v=comparatorType { argNames.add(k); argTypes.add(v); } )*
         )?
       ')'
       ( (K_RETURNS K_NULL) | (K_CALLED { calledOnNullInput=true; })) K_ON K_NULL K_INPUT
-      K_RETURNS rt = comparatorType
+      K_RETURNS returnType = 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); }
+      { $stmt = new CreateFunctionStatement.Raw(
+          fn, argNames, argTypes, returnType, calledOnNullInput, $language.text.toLowerCase(), $body.text, orReplace, ifNotExists);
+      }
     ;
 
-dropFunctionStatement returns [DropFunctionStatement expr]
+dropFunctionStatement returns [DropFunctionStatement.Raw stmt]
     @init {
         boolean ifExists = false;
-        List<CQL3Type.Raw> argsTypes = new ArrayList<>();
-        boolean argsPresent = false;
+        List<CQL3Type.Raw> argTypes = new ArrayList<>();
+        boolean argsSpecified = false;
     }
     : K_DROP K_FUNCTION
       (K_IF K_EXISTS { ifExists = true; } )?
@@ -737,72 +738,71 @@ dropFunctionStatement returns [DropFunctionStatement expr]
       (
         '('
           (
-            v=comparatorType { argsTypes.add(v); }
-            ( ',' v=comparatorType { argsTypes.add(v); } )*
+            v=comparatorType { argTypes.add(v); }
+            ( ',' v=comparatorType { argTypes.add(v); } )*
           )?
         ')'
-        { argsPresent = true; }
+        { argsSpecified = true; }
       )?
-      { $expr = new DropFunctionStatement(fn, argsTypes, argsPresent, ifExists); }
+      { $stmt = new DropFunctionStatement.Raw(fn, argTypes, argsSpecified, ifExists); }
     ;
 
 /**
  * CREATE KEYSPACE [IF NOT EXISTS] <KEYSPACE> WITH attr1 = value1 AND attr2 = value2;
  */
-createKeyspaceStatement returns [CreateKeyspaceStatement expr]
+createKeyspaceStatement returns [CreateKeyspaceStatement.Raw stmt]
     @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); }
+      K_WITH properties[attrs] { $stmt = new CreateKeyspaceStatement.Raw(ks, attrs, ifNotExists); }
     ;
 
 /**
- * CREATE COLUMNFAMILY [IF NOT EXISTS] <CF> (
+ * CREATE TABLE [IF NOT EXISTS] <CF> (
  *     <name1> <type>,
  *     <name2> <type>,
  *     <name3> <type>
  * ) WITH <property> = <value> AND ...;
  */
-createTableStatement returns [CreateTableStatement.RawStatement expr]
+createTableStatement returns [CreateTableStatement.Raw stmt]
     @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]
+      cf=columnFamilyName { $stmt = new CreateTableStatement.Raw(cf, ifNotExists); }
+      tableDefinition[stmt]
     ;
 
-cfamDefinition[CreateTableStatement.RawStatement expr]
-    : '(' cfamColumns[expr] ( ',' cfamColumns[expr]? )* ')'
-      ( K_WITH cfamProperty[expr.properties] ( K_AND cfamProperty[expr.properties] )*)?
+tableDefinition[CreateTableStatement.Raw stmt]
+    : '(' tableColumns[stmt] ( ',' tableColumns[stmt]? )* ')'
+      ( K_WITH tableProperty[stmt] ( K_AND tableProperty[stmt] )*)?
     ;
 
-cfamColumns[CreateTableStatement.RawStatement expr]
+tableColumns[CreateTableStatement.Raw stmt]
     @init { boolean isStatic = false; }
-    : k=ident v=comparatorType (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); } )* ')'
+    : k=ident v=comparatorType (K_STATIC { isStatic = true; })? { $stmt.addColumn(k, v, isStatic); }
+        (K_PRIMARY K_KEY { $stmt.setPartitionKeyColumn(k); })?
+    | K_PRIMARY K_KEY '(' tablePartitionKey[stmt] (',' c=ident { $stmt.markClusteringColumn(c); } )* ')'
     ;
 
-pkDef[CreateTableStatement.RawStatement expr]
+tablePartitionKey[CreateTableStatement.Raw stmt]
     @init {List<ColumnIdentifier> l = new ArrayList<ColumnIdentifier>();}
-    @after{ $expr.addKeyAliases(l); }
+    @after{ $stmt.setPartitionKeyColumns(l); }
     : k1=ident { l.add(k1);}
     | '(' k1=ident { l.add(k1); } ( ',' kn=ident { l.add(kn); } )* ')'
     ;
 
-cfamProperty[CFProperties props]
-    : property[props.properties]
-    | 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])* ')'
+tableProperty[CreateTableStatement.Raw stmt]
+    : property[stmt.attrs]
+    | K_COMPACT K_STORAGE { throw new SyntaxException("COMPACT STORAGE tables are not allowed starting with version 4.0"); }
+    | K_CLUSTERING K_ORDER K_BY '(' tableClusteringOrder[stmt] (',' tableClusteringOrder[stmt])* ')'
     ;
 
-cfamOrdering[CFProperties props]
-    @init{ boolean reversed=false; }
-    : k=ident (K_ASC | K_DESC { reversed=true;} ) { $props.setOrdering(k, reversed); }
+tableClusteringOrder[CreateTableStatement.Raw stmt]
+    @init{ boolean ascending = true; }
+    : k=ident (K_ASC | K_DESC { ascending = false; } ) { $stmt.extendClusteringOrder(k, ascending); }
     ;
 
-
 /**
  * CREATE TYPE foo (
  *    <name1> <type1>,
@@ -810,34 +810,33 @@ cfamOrdering[CFProperties props]
  *    ....
  * )
  */
-createTypeStatement returns [CreateTypeStatement expr]
+createTypeStatement returns [CreateTypeStatement.Raw stmt]
     @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]? )* ')'
+         tn=userTypeName { $stmt = new CreateTypeStatement.Raw(tn, ifNotExists); }
+         '(' typeColumns[stmt] ( ',' typeColumns[stmt]? )* ')'
     ;
 
-typeColumns[CreateTypeStatement expr]
-    : k=fident v=comparatorType { $expr.addDefinition(k, v); }
+typeColumns[CreateTypeStatement.Raw stmt]
+    : k=fident v=comparatorType { $stmt.addField(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]
+createIndexStatement returns [CreateIndexStatement.Raw stmt]
     @init {
-        IndexPropDefs props = new IndexPropDefs();
+        IndexAttributes props = new IndexAttributes();
         boolean ifNotExists = false;
-        IndexName name = new IndexName();
+        QualifiedName name = new QualifiedName();
         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); }
+      { $stmt = new CreateIndexStatement.Raw(cf, name, targets, props, ifNotExists); }
     ;
 
 indexIdent [List<IndexTarget.Raw> targets]
@@ -856,106 +855,114 @@ indexIdent [List<IndexTarget.Raw> targets]
  *  PRIMARY KEY (<pkColumns>)
  *  WITH <property> = <value> AND ...;
  */
-createMaterializedViewStatement returns [CreateViewStatement expr]
+createMaterializedViewStatement returns [CreateViewStatement.Raw stmt]
     @init {
         boolean ifNotExists = false;
-        List<ColumnMetadata.Raw> partitionKeys = new ArrayList<>();
-        List<ColumnMetadata.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=selectors 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);
+             $stmt = new CreateViewStatement.Raw(basecf, cf, sclause, where, ifNotExists);
         }
-        ( K_WITH cfamProperty[expr.properties] ( K_AND cfamProperty[expr.properties] )*)?
+        viewPrimaryKey[stmt]
+        ( K_WITH viewProperty[stmt] ( K_AND viewProperty[stmt] )*)?
+    ;
+
+viewPrimaryKey[CreateViewStatement.Raw stmt]
+    : K_PRIMARY K_KEY '(' viewPartitionKey[stmt] (',' c=ident { $stmt.markClusteringColumn(c); } )* ')'
+    ;
+
+viewPartitionKey[CreateViewStatement.Raw stmt]
+    @init {List<ColumnIdentifier> l = new ArrayList<ColumnIdentifier>();}
+    @after{ $stmt.setPartitionKeyColumns(l); }
+    : k1=ident { l.add(k1);}
+    | '(' k1=ident { l.add(k1); } ( ',' kn=ident { l.add(kn); } )* ')'
+    ;
+
+viewProperty[CreateViewStatement.Raw stmt]
+    : property[stmt.attrs]
+    | K_COMPACT K_STORAGE { throw new SyntaxException("COMPACT STORAGE tables are not allowed starting with version 4.0"); }
+    | K_CLUSTERING K_ORDER K_BY '(' viewClusteringOrder[stmt] (',' viewClusteringOrder[stmt])* ')'
+    ;
+
+viewClusteringOrder[CreateViewStatement.Raw stmt]
+    @init{ boolean ascending = true; }
+    : k=ident (K_ASC | K_DESC { ascending = false; } ) { $stmt.extendClusteringOrder(k, ascending); }
     ;
 
 /**
  * CREATE TRIGGER triggerName ON columnFamily USING 'triggerClass';
  */
-createTriggerStatement returns [CreateTriggerStatement expr]
+createTriggerStatement returns [CreateTriggerStatement.Raw stmt]
     @init {
         boolean ifNotExists = false;
     }
     : 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.toString(), $cls.text, ifNotExists); }
+      { $stmt = new CreateTriggerStatement.Raw(cf, name.toString(), $cls.text, ifNotExists); }
     ;
 
 /**
  * DROP TRIGGER [IF EXISTS] triggerName ON columnFamily;
  */
-dropTriggerStatement returns [DropTriggerStatement expr]
+dropTriggerStatement returns [DropTriggerStatement.Raw stmt]
      @init { boolean ifExists = false; }
     : K_DROP K_TRIGGER (K_IF K_EXISTS { ifExists = true; } )? (name=ident) K_ON cf=columnFamilyName
-      { $expr = new DropTriggerStatement(cf, name.toString(), ifExists); }
+      { $stmt = new DropTriggerStatement.Raw(cf, name.toString(), ifExists); }
     ;
 
 /**
  * ALTER KEYSPACE <KS> WITH <property> = <value>;
  */
-alterKeyspaceStatement returns [AlterKeyspaceStatement expr]
+alterKeyspaceStatement returns [AlterKeyspaceStatement.Raw stmt]
     @init { KeyspaceAttributes attrs = new KeyspaceAttributes(); }
     : K_ALTER K_KEYSPACE ks=keyspaceName
-        K_WITH properties[attrs] { $expr = new AlterKeyspaceStatement(ks, attrs); }
+        K_WITH properties[attrs] { $stmt = new AlterKeyspaceStatement.Raw(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>;
+ * ALTER TABLE <table> ALTER <column> TYPE <newtype>;
+ * ALTER TABLE <table> ADD <column> <newtype>; | ALTER TABLE <table> ADD (<column> <newtype>,<column1> <newtype1>..... <column n> <newtype n>)
+ * ALTER TABLE <table> DROP <column>; | ALTER TABLE <table> DROP ( <column>,<column1>.....<column n>)
+ * ALTER TABLE <table> RENAME <column> TO <column>;
+ * ALTER TABLE <table> WITH <property> = <value>;
  */
-alterTableStatement returns [AlterTableStatement expr]
-    @init {
-        AlterTableStatement.Type type = null;
-        TableAttributes attrs = new TableAttributes();
-        Map<ColumnMetadata.Raw, ColumnMetadata.Raw> renames = new HashMap<ColumnMetadata.Raw, ColumnMetadata.Raw>();
-        List<AlterTableStatementColumn> colNameList = new ArrayList<AlterTableStatementColumn>();
-        Long deleteTimestamp = null;
-    }
-    : K_ALTER K_COLUMNFAMILY cf=columnFamilyName
-          ( 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 ( (        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=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);
-    }
+alterTableStatement returns [AlterTableStatement.Raw stmt]
+    : K_ALTER K_COLUMNFAMILY cf=columnFamilyName { $stmt = new AlterTableStatement.Raw(cf); }
+      (
+        K_ALTER id=cident K_TYPE v=comparatorType { $stmt.alter(id, v); }
+
+      | K_ADD  (        id=schema_cident  v=comparatorType  b=isStaticColumn { $stmt.add(id,  v,  b);  }
+               | ('('  id1=schema_cident v1=comparatorType b1=isStaticColumn { $stmt.add(id1, v1, b1); }
+                 ( ',' idn=schema_cident vn=comparatorType bn=isStaticColumn { $stmt.add(idn, vn, bn); } )* ')') )
+
+      | K_DROP (        id=schema_cident { $stmt.drop(id);  }
+               | ('('  id1=schema_cident { $stmt.drop(id1); }
+                 ( ',' idn=schema_cident { $stmt.drop(idn); } )* ')') )
+               ( K_USING K_TIMESTAMP t=INTEGER { $stmt.timestamp(Long.parseLong(Constants.Literal.integer($t.text).getText())); } )?
+
+      | K_RENAME id1=schema_cident K_TO toId1=schema_cident { $stmt.rename(id1, toId1); }
+         ( K_AND idn=schema_cident K_TO toIdn=schema_cident { $stmt.rename(idn, toIdn); } )*
+
+      | K_WITH properties[$stmt.attrs] { $stmt.attrs(); }
+      )
     ;
 
-cfisStatic returns [boolean isStaticColumn]
-    @init{
-        boolean isStatic = false;
-    }
-    : (K_STATIC { isStatic=true; })? { $isStaticColumn = isStatic;
-    }
+isStaticColumn returns [boolean isStaticColumn]
+    @init { boolean isStatic = false; }
+    : (K_STATIC { isStatic=true; })? { $isStaticColumn = isStatic; }
     ;
 
-alterMaterializedViewStatement returns [AlterViewStatement expr]
+alterMaterializedViewStatement returns [AlterViewStatement.Raw stmt]
     @init {
         TableAttributes attrs = new TableAttributes();
     }
     : K_ALTER K_MATERIALIZED K_VIEW name=columnFamilyName
           K_WITH properties[attrs]
     {
-        $expr = new AlterViewStatement(name, attrs);
+        $stmt = new AlterViewStatement.Raw(name, attrs);
     }
     ;
 
@@ -965,59 +972,58 @@ alterMaterializedViewStatement returns [AlterViewStatement expr]
  * 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 r=renamedColumns                { $expr = AlterTypeStatement.renames(name, r); }
-          )
-    ;
+alterTypeStatement returns [AlterTypeStatement.Raw stmt]
+    : K_ALTER K_TYPE name=userTypeName { $stmt = new AlterTypeStatement.Raw(name); }
+      (
+        K_ALTER   f=fident K_TYPE v=comparatorType { $stmt.alter(f, v); }
+
+      | K_ADD     f=fident v=comparatorType        { $stmt.add(f, v); }
 
-renamedColumns returns [Map<FieldIdentifier, FieldIdentifier> renames]
-    @init {$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); } )*
+      | K_RENAME f1=fident K_TO toF1=fident        { $stmt.rename(f1, toF1); }
+         ( K_AND fn=fident K_TO toFn=fident        { $stmt.rename(fn, toFn); } )*
+      )
     ;
 
 /**
  * DROP KEYSPACE [IF EXISTS] <KSP>;
  */
-dropKeyspaceStatement returns [DropKeyspaceStatement ksp]
+dropKeyspaceStatement returns [DropKeyspaceStatement.Raw stmt]
     @init { boolean ifExists = false; }
-    : K_DROP K_KEYSPACE (K_IF K_EXISTS { ifExists = true; } )? ks=keyspaceName { $ksp = new DropKeyspaceStatement(ks, ifExists); }
+    : K_DROP K_KEYSPACE (K_IF K_EXISTS { ifExists = true; } )? ks=keyspaceName { $stmt = new DropKeyspaceStatement.Raw(ks, ifExists); }
     ;
 
 /**
- * DROP COLUMNFAMILY [IF EXISTS] <CF>;
+ * DROP TABLE [IF EXISTS] <table>;
  */
-dropTableStatement returns [DropTableStatement stmt]
+dropTableStatement returns [DropTableStatement.Raw stmt]
     @init { boolean ifExists = false; }
-    : K_DROP K_COLUMNFAMILY (K_IF K_EXISTS { ifExists = true; } )? cf=columnFamilyName { $stmt = new DropTableStatement(cf, ifExists); }
+    : K_DROP K_COLUMNFAMILY (K_IF K_EXISTS { ifExists = true; } )? name=columnFamilyName { $stmt = new DropTableStatement.Raw(name, ifExists); }
     ;
 
 /**
  * DROP TYPE <name>;
  */
-dropTypeStatement returns [DropTypeStatement stmt]
+dropTypeStatement returns [DropTypeStatement.Raw stmt]
     @init { boolean ifExists = false; }
-    : K_DROP K_TYPE (K_IF K_EXISTS { ifExists = true; } )? name=userTypeName { $stmt = new DropTypeStatement(name, ifExists); }
+    : K_DROP K_TYPE (K_IF K_EXISTS { ifExists = true; } )? name=userTypeName { $stmt = new DropTypeStatement.Raw(name, ifExists); }
     ;
 
 /**
  * DROP INDEX [IF EXISTS] <INDEX_NAME>
  */
-dropIndexStatement returns [DropIndexStatement expr]
+dropIndexStatement returns [DropIndexStatement.Raw stmt]
     @init { boolean ifExists = false; }
     : K_DROP K_INDEX (K_IF K_EXISTS { ifExists = true; } )? index=indexName
-      { $expr = new DropIndexStatement(index, ifExists); }
+      { $stmt = new DropIndexStatement.Raw(index, ifExists); }
     ;
 
 /**
  * DROP MATERIALIZED VIEW [IF EXISTS] <view_name>
  */
-dropMaterializedViewStatement returns [DropViewStatement expr]
+dropMaterializedViewStatement returns [DropViewStatement.Raw stmt]
     @init { boolean ifExists = false; }
     : K_DROP K_MATERIALIZED K_VIEW (K_IF K_EXISTS { ifExists = true; } )? cf=columnFamilyName
-      { $expr = new DropViewStatement(cf, ifExists); }
+      { $stmt = new DropViewStatement.Raw(cf, ifExists); }
     ;
 
 /**
@@ -1110,7 +1116,7 @@ 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()); }
+      { $res = DataResource.table($cf.name.getKeyspace(), $cf.name.getName()); }
     ;
 
 jmxResource returns [JMXResource res]
@@ -1336,17 +1342,17 @@ noncol_ident returns [ColumnIdentifier id]
 
 // Keyspace & Column family names
 keyspaceName returns [String id]
-    @init { CFName name = new CFName(); }
+    @init { QualifiedName name = new QualifiedName(); }
     : ksName[name] { $id = name.getKeyspace(); }
     ;
 
-indexName returns [IndexName name]
-    @init { $name = new IndexName(); }
+indexName returns [QualifiedName name]
+    @init { $name = new QualifiedName(); }
     : (ksName[name] '.')? idxName[name]
     ;
 
-columnFamilyName returns [CFName name]
-    @init { $name = new CFName(); }
+columnFamilyName returns [QualifiedName name]
+    @init { $name = new QualifiedName(); }
     : (ksName[name] '.')? cfName[name]
     ;
 
@@ -1359,24 +1365,24 @@ userOrRoleName returns [RoleName name]
     : roleName[role] {$name = role;}
     ;
 
-ksName[KeyspaceElementName name]
+ksName[QualifiedName 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); }
+cfName[QualifiedName name]
+    : t=IDENT              { $name.setName($t.text, false); }
+    | t=QUOTED_NAME        { $name.setName($t.text, true); }
+    | k=unreserved_keyword { $name.setName(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); }
+idxName[QualifiedName name]
+    : t=IDENT              { $name.setName($t.text, false); }
+    | t=QUOTED_NAME        { $name.setName($t.text, true);}
+    | k=unreserved_keyword { $name.setName(k, false); }
     | QMARK {addRecognitionError("Bind variables cannot be used for index names");}
     ;
 
@@ -1714,7 +1720,7 @@ comparatorType returns [CQL3Type.Raw t]
     | K_FROZEN '<' f=comparatorType '>'
       {
         try {
-            $t = CQL3Type.Raw.frozen(f);
+            $t = f.freeze();
         } catch (InvalidRequestException e) {
             addRecognitionError(e.getMessage());
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/audit/AuditLogContext.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/audit/AuditLogContext.java b/src/java/org/apache/cassandra/audit/AuditLogContext.java
new file mode 100644
index 0000000..9b44cf3
--- /dev/null
+++ b/src/java/org/apache/cassandra/audit/AuditLogContext.java
@@ -0,0 +1,42 @@
+/*
+ * 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.audit;
+
+public class AuditLogContext
+{
+    public final AuditLogEntryType auditLogEntryType;
+    public final String keyspace;
+    public final String scope;
+
+    public AuditLogContext(AuditLogEntryType auditLogEntryType)
+    {
+        this(auditLogEntryType, null, null);
+    }
+
+    public AuditLogContext(AuditLogEntryType auditLogEntryType, String keyspace)
+    {
+        this(auditLogEntryType, keyspace, null);
+    }
+
+    public AuditLogContext(AuditLogEntryType auditLogEntryType, String keyspace, String scope)
+    {
+        this.auditLogEntryType = auditLogEntryType;
+        this.keyspace = keyspace;
+        this.scope = scope;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/audit/AuditLogManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/audit/AuditLogManager.java b/src/java/org/apache/cassandra/audit/AuditLogManager.java
index 090499c..9e6a0a1 100644
--- a/src/java/org/apache/cassandra/audit/AuditLogManager.java
+++ b/src/java/org/apache/cassandra/audit/AuditLogManager.java
@@ -31,8 +31,8 @@ import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.cql3.CQLStatement;
+import org.apache.cassandra.cql3.QueryHandler;
 import org.apache.cassandra.cql3.QueryOptions;
-import org.apache.cassandra.cql3.statements.ParsedStatement;
 import org.apache.cassandra.exceptions.AuthenticationException;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.exceptions.UnauthorizedException;
@@ -187,7 +187,7 @@ public class AuditLogManager
     /**
      * Logs Batch queries to both FQL and standard audit logger.
      */
-    public void logBatch(String batchTypeName, List<Object> queryOrIdList, List<List<ByteBuffer>> values, List<ParsedStatement.Prepared> prepared, QueryOptions options, QueryState state, long queryStartTimeMillis)
+    public void logBatch(String batchTypeName, List<Object> queryOrIdList, List<List<ByteBuffer>> values, List<QueryHandler.Prepared> prepared, QueryOptions options, QueryState state, long queryStartTimeMillis)
     {
         if (isAuditingEnabled())
         {
@@ -201,7 +201,7 @@ public class AuditLogManager
         if (isFQLEnabled())
         {
             List<String> queryStrings = new ArrayList<>(queryOrIdList.size());
-            for (ParsedStatement.Prepared prepStatment : prepared)
+            for (QueryHandler.Prepared prepStatment : prepared)
             {
                 queryStrings.add(prepStatment.rawCQLStatement);
             }
@@ -209,7 +209,7 @@ public class AuditLogManager
         }
     }
 
-    private static List<AuditLogEntry> buildEntriesForBatch(List<Object> queryOrIdList, List<ParsedStatement.Prepared> prepared, QueryState state, QueryOptions options, long queryStartTimeMillis)
+    private static List<AuditLogEntry> buildEntriesForBatch(List<Object> queryOrIdList, List<QueryHandler.Prepared> prepared, QueryState state, QueryOptions options, long queryStartTimeMillis)
     {
         List<AuditLogEntry> auditLogEntries = new ArrayList<>(queryOrIdList.size() + 1);
         UUID batchId = UUID.randomUUID();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/audit/IAuditLogContext.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/audit/IAuditLogContext.java b/src/java/org/apache/cassandra/audit/IAuditLogContext.java
deleted file mode 100644
index 55c3e04..0000000
--- a/src/java/org/apache/cassandra/audit/IAuditLogContext.java
+++ /dev/null
@@ -1,53 +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.audit;
-
-import org.apache.cassandra.cql3.CQLStatement;
-
-/**
- * Provides the context needed for audit logging statements.
- * {@link CQLStatement} implements this interface such that every CQL command provides the context needed for AuditLog.
- */
-public interface IAuditLogContext
-{
-    AuditLogContext getAuditLogContext();
-
-    static class AuditLogContext
-    {
-        public final AuditLogEntryType auditLogEntryType;
-        public final String keyspace;
-        public final String scope;
-
-        public AuditLogContext(AuditLogEntryType auditLogEntryType)
-        {
-            this(auditLogEntryType,null,null);
-        }
-
-        public AuditLogContext(AuditLogEntryType auditLogEntryType, String keyspace)
-        {
-           this(auditLogEntryType,keyspace,null);
-        }
-
-        public AuditLogContext(AuditLogEntryType auditLogEntryType, String keyspace, String scope)
-        {
-            this.auditLogEntryType = auditLogEntryType;
-            this.keyspace = keyspace;
-            this.scope = scope;
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/auth/AuthKeyspace.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/auth/AuthKeyspace.java b/src/java/org/apache/cassandra/auth/AuthKeyspace.java
index 1f71bdc..3f95b19 100644
--- a/src/java/org/apache/cassandra/auth/AuthKeyspace.java
+++ b/src/java/org/apache/cassandra/auth/AuthKeyspace.java
@@ -19,7 +19,7 @@ package org.apache.cassandra.auth;
 
 import java.util.concurrent.TimeUnit;
 
-import org.apache.cassandra.cql3.statements.CreateTableStatement;
+import org.apache.cassandra.cql3.statements.schema.CreateTableStatement;
 import org.apache.cassandra.schema.TableId;
 import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.schema.SchemaConstants;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/auth/AuthenticatedUser.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/auth/AuthenticatedUser.java b/src/java/org/apache/cassandra/auth/AuthenticatedUser.java
index c608068..3d7c078 100644
--- a/src/java/org/apache/cassandra/auth/AuthenticatedUser.java
+++ b/src/java/org/apache/cassandra/auth/AuthenticatedUser.java
@@ -84,7 +84,7 @@ public class AuthenticatedUser
     /**
      * Some internal operations are performed on behalf of Cassandra itself, in those cases
      * the system user should be used where an identity is required
-     * see CreateRoleStatement#execute() and overrides of SchemaAlteringStatement#grantPermissionsToCreator()
+     * see CreateRoleStatement#execute() and overrides of AlterSchemaStatement#createdResources()
      */
     public boolean isSystem()
     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/auth/CassandraAuthorizer.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/auth/CassandraAuthorizer.java b/src/java/org/apache/cassandra/auth/CassandraAuthorizer.java
index 5140bcf..cebde13 100644
--- a/src/java/org/apache/cassandra/auth/CassandraAuthorizer.java
+++ b/src/java/org/apache/cassandra/auth/CassandraAuthorizer.java
@@ -27,7 +27,6 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.schema.SchemaConstants;
 import org.apache.cassandra.cql3.*;
 import org.apache.cassandra.cql3.statements.BatchStatement;
 import org.apache.cassandra.cql3.statements.ModificationStatement;
@@ -35,11 +34,8 @@ import org.apache.cassandra.cql3.statements.SelectStatement;
 import org.apache.cassandra.db.ConsistencyLevel;
 import org.apache.cassandra.db.marshal.UTF8Type;
 import org.apache.cassandra.exceptions.*;
+import org.apache.cassandra.schema.SchemaConstants;
 import org.apache.cassandra.service.ClientState;
-
-import org.apache.cassandra.cql3.QueryOptions;
-import org.apache.cassandra.cql3.QueryProcessor;
-import org.apache.cassandra.cql3.UntypedResultSet;
 import org.apache.cassandra.service.QueryState;
 import org.apache.cassandra.transport.messages.ResultMessage;
 import org.apache.cassandra.utils.ByteBufferUtil;
@@ -115,7 +111,7 @@ public class CassandraAuthorizer implements IAuthorizer
                                                               AuthKeyspace.RESOURCE_ROLE_INDEX,
                                                               escape(row.getString("resource")),
                                                               escape(revokee.getRoleName())),
-                                                ClientState.forInternalCalls()).statement);
+                                                ClientState.forInternalCalls()));
 
             }
 
@@ -123,7 +119,7 @@ public class CassandraAuthorizer implements IAuthorizer
                                                                      SchemaConstants.AUTH_KEYSPACE_NAME,
                                                                      AuthKeyspace.ROLE_PERMISSIONS,
                                                                      escape(revokee.getRoleName())),
-                                                       ClientState.forInternalCalls()).statement);
+                                                       ClientState.forInternalCalls()));
 
             executeLoggedBatch(statements);
         }
@@ -153,14 +149,14 @@ public class CassandraAuthorizer implements IAuthorizer
                                                                          AuthKeyspace.ROLE_PERMISSIONS,
                                                                          escape(row.getString("role")),
                                                                          escape(droppedResource.getName())),
-                                                           ClientState.forInternalCalls()).statement);
+                                                           ClientState.forInternalCalls()));
             }
 
             statements.add(QueryProcessor.getStatement(String.format("DELETE FROM %s.%s WHERE resource = '%s'",
                                                                      SchemaConstants.AUTH_KEYSPACE_NAME,
                                                                      AuthKeyspace.RESOURCE_ROLE_INDEX,
                                                                      escape(droppedResource.getName())),
-                                                                               ClientState.forInternalCalls()).statement);
+                                                      ClientState.forInternalCalls()));
 
             executeLoggedBatch(statements);
         }
@@ -173,8 +169,8 @@ public class CassandraAuthorizer implements IAuthorizer
     private void executeLoggedBatch(List<CQLStatement> statements)
     throws RequestExecutionException, RequestValidationException
     {
-        BatchStatement batch = new BatchStatement(0,
-                                                  BatchStatement.Type.LOGGED,
+        BatchStatement batch = new BatchStatement(BatchStatement.Type.LOGGED,
+                                                  VariableSpecifications.empty(),
                                                   Lists.newArrayList(Iterables.filter(statements, ModificationStatement.class)),
                                                   Attributes.none());
         processBatch(batch);
@@ -331,7 +327,7 @@ public class CassandraAuthorizer implements IAuthorizer
                                      SchemaConstants.AUTH_KEYSPACE_NAME,
                                      permissionsTable,
                                      entityname);
-        return (SelectStatement) QueryProcessor.getStatement(query, ClientState.forInternalCalls()).statement;
+        return (SelectStatement) QueryProcessor.getStatement(query, ClientState.forInternalCalls());
     }
 
     // We only worry about one character ('). Make sure it's properly escaped.

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/auth/CassandraNetworkAuthorizer.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/auth/CassandraNetworkAuthorizer.java b/src/java/org/apache/cassandra/auth/CassandraNetworkAuthorizer.java
index 9faa423..34a0140 100644
--- a/src/java/org/apache/cassandra/auth/CassandraNetworkAuthorizer.java
+++ b/src/java/org/apache/cassandra/auth/CassandraNetworkAuthorizer.java
@@ -46,7 +46,7 @@ public class CassandraNetworkAuthorizer implements INetworkAuthorizer
         String query = String.format("SELECT dcs FROM %s.%s WHERE role = ?",
                                      SchemaConstants.AUTH_KEYSPACE_NAME,
                                      AuthKeyspace.NETWORK_PERMISSIONS);
-        authorizeUserStatement = (SelectStatement) QueryProcessor.getStatement(query, ClientState.forInternalCalls()).statement;
+        authorizeUserStatement = (SelectStatement) QueryProcessor.getStatement(query, ClientState.forInternalCalls());
     }
 
     @VisibleForTesting

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/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 1271699..f5dd457 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.cql3.statements.SelectStatement;
 import org.apache.cassandra.db.ConsistencyLevel;
 import org.apache.cassandra.db.marshal.UTF8Type;
 import org.apache.cassandra.exceptions.*;
+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;
@@ -364,7 +365,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());
         }
         catch (RequestValidationException e)
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/auth/FunctionResource.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/auth/FunctionResource.java b/src/java/org/apache/cassandra/auth/FunctionResource.java
index 2c09c05..61c6a29 100644
--- a/src/java/org/apache/cassandra/auth/FunctionResource.java
+++ b/src/java/org/apache/cassandra/auth/FunctionResource.java
@@ -134,6 +134,11 @@ public class FunctionResource implements IResource
         return new FunctionResource(keyspace, name, argTypes);
     }
 
+    public static FunctionResource function(Function function)
+    {
+        return new FunctionResource(function.name().keyspace, function.name().name, function.argTypes());
+    }
+
     /**
      * Creates a FunctionResource representing a specific, keyspace-scoped function.
      * This variant is used to create an instance during parsing of a CQL statement.
@@ -157,6 +162,11 @@ public class FunctionResource implements IResource
         return new FunctionResource(keyspace, name, abstractTypes);
     }
 
+    public static FunctionResource functionFromCql(FunctionName name, List<CQL3Type.Raw> argTypes)
+    {
+        return functionFromCql(name.keyspace, name.name, argTypes);
+    }
+
     /**
      * Parses a resource name into a FunctionResource instance.
      *

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/auth/PasswordAuthenticator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/auth/PasswordAuthenticator.java b/src/java/org/apache/cassandra/auth/PasswordAuthenticator.java
index b1604e7..27a68a0 100644
--- a/src/java/org/apache/cassandra/auth/PasswordAuthenticator.java
+++ b/src/java/org/apache/cassandra/auth/PasswordAuthenticator.java
@@ -160,7 +160,7 @@ public class PasswordAuthenticator implements IAuthenticator
 
     private static SelectStatement prepare(String query)
     {
-        return (SelectStatement) QueryProcessor.getStatement(query, ClientState.forInternalCalls()).statement;
+        return (SelectStatement) QueryProcessor.getStatement(query, ClientState.forInternalCalls());
     }
 
     private class PlainTextSaslAuthenticator implements SaslNegotiator

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/auth/jmx/AuthorizationProxy.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/auth/jmx/AuthorizationProxy.java b/src/java/org/apache/cassandra/auth/jmx/AuthorizationProxy.java
index d9b63c6..ef00027 100644
--- a/src/java/org/apache/cassandra/auth/jmx/AuthorizationProxy.java
+++ b/src/java/org/apache/cassandra/auth/jmx/AuthorizationProxy.java
@@ -53,7 +53,7 @@ import org.apache.cassandra.service.StorageService;
  *
  * Because an ObjectName may contain wildcards, meaning it represents a set of individual MBeans,
  * JMX resources don't fit well with the hierarchical approach modelled by other IResource
- * implementations and utilised by ClientState::ensureHasPermission etc. To enable grants to use
+ * implementations and utilised by ClientState::ensurePermission etc. To enable grants to use
  * pattern-type ObjectNames, this class performs its own custom matching and filtering of resources
  * rather than pushing that down to the configured IAuthorizer. To that end, during authorization
  * it pulls back all permissions for the active subject, filtering them to retain only grants on

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/cql3/CFName.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/CFName.java b/src/java/org/apache/cassandra/cql3/CFName.java
deleted file mode 100644
index 3f4a118..0000000
--- a/src/java/org/apache/cassandra/cql3/CFName.java
+++ /dev/null
@@ -1,39 +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;
-
-public class CFName extends KeyspaceElementName
-{
-    private String cfName;
-
-    public void setColumnFamily(String cf, boolean keepCase)
-    {
-        cfName = toInternalName(cf, keepCase);
-    }
-
-    public String getColumnFamily()
-    {
-        return cfName;
-    }
-
-    @Override
-    public String toString()
-    {
-        return super.toString() + cfName;
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/cql3/CQL3Type.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/CQL3Type.java b/src/java/org/apache/cassandra/cql3/CQL3Type.java
index d387a25..340a992 100644
--- a/src/java/org/apache/cassandra/cql3/CQL3Type.java
+++ b/src/java/org/apache/cassandra/cql3/CQL3Type.java
@@ -37,6 +37,8 @@ import org.apache.cassandra.serializers.MarshalException;
 import org.apache.cassandra.transport.ProtocolVersion;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
+import static java.util.stream.Collectors.toList;
+
 public interface CQL3Type
 {
     static final Logger logger = LoggerFactory.getLogger(CQL3Type.class);
@@ -88,7 +90,7 @@ public interface CQL3Type
 
         private final AbstractType<?> type;
 
-        private Native(AbstractType<?> type)
+        Native(AbstractType<?> type)
         {
             this.type = type;
         }
@@ -482,7 +484,12 @@ public interface CQL3Type
     // actual type used, so Raw is a "not yet prepared" CQL3Type.
     public abstract class Raw
     {
-        protected boolean frozen = false;
+        protected final boolean frozen;
+
+        protected Raw(boolean frozen)
+        {
+            this.frozen = frozen;
+        }
 
         public abstract boolean supportsFreezing();
 
@@ -491,11 +498,6 @@ public interface CQL3Type
             return this.frozen;
         }
 
-        public boolean canBeNonFrozen()
-        {
-            return true;
-        }
-
         public boolean isDuration()
         {
             return false;
@@ -516,7 +518,7 @@ public interface CQL3Type
             return null;
         }
 
-        public void freeze() throws InvalidRequestException
+        public Raw freeze()
         {
             String message = String.format("frozen<> is only allowed on collections, tuples, and user-defined types (got %s)", this);
             throw new InvalidRequestException(message);
@@ -544,46 +546,41 @@ public interface CQL3Type
 
         public static Raw from(CQL3Type type)
         {
-            return new RawType(type);
+            return new RawType(type, false);
         }
 
         public static Raw userType(UTName name)
         {
-            return new RawUT(name);
+            return new RawUT(name, false);
         }
 
         public static Raw map(CQL3Type.Raw t1, CQL3Type.Raw t2)
         {
-            return new RawCollection(CollectionType.Kind.MAP, t1, t2);
+            return new RawCollection(CollectionType.Kind.MAP, t1, t2, false);
         }
 
         public static Raw list(CQL3Type.Raw t)
         {
-            return new RawCollection(CollectionType.Kind.LIST, null, t);
+            return new RawCollection(CollectionType.Kind.LIST, null, t, false);
         }
 
         public static Raw set(CQL3Type.Raw t)
         {
-            return new RawCollection(CollectionType.Kind.SET, null, t);
+            return new RawCollection(CollectionType.Kind.SET, null, t, false);
         }
 
         public static Raw tuple(List<CQL3Type.Raw> ts)
         {
-            return new RawTuple(ts);
-        }
-
-        public static Raw frozen(CQL3Type.Raw t) throws InvalidRequestException
-        {
-            t.freeze();
-            return t;
+            return new RawTuple(ts, false);
         }
 
         private static class RawType extends Raw
         {
             private final CQL3Type type;
 
-            private RawType(CQL3Type type)
+            private RawType(CQL3Type type, boolean frozen)
             {
+                super(frozen);
                 this.type = type;
             }
 
@@ -620,20 +617,28 @@ public interface CQL3Type
             private final CQL3Type.Raw keys;
             private final CQL3Type.Raw values;
 
-            private RawCollection(CollectionType.Kind kind, CQL3Type.Raw keys, CQL3Type.Raw values)
+            private RawCollection(CollectionType.Kind kind, CQL3Type.Raw keys, CQL3Type.Raw values, boolean frozen)
             {
+                super(frozen);
                 this.kind = kind;
                 this.keys = keys;
                 this.values = values;
             }
 
-            public void freeze() throws InvalidRequestException
+            @Override
+            public RawCollection freeze()
             {
-                if (keys != null && keys.supportsFreezing())
-                    keys.freeze();
-                if (values != null && values.supportsFreezing())
-                    values.freeze();
-                frozen = true;
+                CQL3Type.Raw frozenKeys =
+                    null != keys && keys.supportsFreezing()
+                  ? keys.freeze()
+                  : keys;
+
+                CQL3Type.Raw frozenValues =
+                    null != values && values.supportsFreezing()
+                  ? values.freeze()
+                  : values;
+
+                return new RawCollection(kind, frozenKeys, frozenValues, true);
             }
 
             public boolean supportsFreezing()
@@ -727,8 +732,9 @@ public interface CQL3Type
         {
             private final UTName name;
 
-            private RawUT(UTName name)
+            private RawUT(UTName name, boolean frozen)
             {
+                super(frozen);
                 this.name = name;
             }
 
@@ -737,14 +743,10 @@ public interface CQL3Type
                 return name.getKeyspace();
             }
 
-            public void freeze()
-            {
-                frozen = true;
-            }
-
-            public boolean canBeNonFrozen()
+            @Override
+            public RawUT freeze()
             {
-                return true;
+                return new RawUT(name, true);
             }
 
             public CQL3Type prepare(String keyspace, Types udts) throws InvalidRequestException
@@ -801,8 +803,9 @@ public interface CQL3Type
         {
             private final List<CQL3Type.Raw> types;
 
-            private RawTuple(List<CQL3Type.Raw> types)
+            private RawTuple(List<CQL3Type.Raw> types, boolean frozen)
             {
+                super(frozen);
                 this.types = types;
             }
 
@@ -811,22 +814,22 @@ public interface CQL3Type
                 return true;
             }
 
-            public void freeze() throws InvalidRequestException
+            @Override
+            public RawTuple freeze()
             {
-                for (CQL3Type.Raw t : types)
-                    if (t.supportsFreezing())
-                        t.freeze();
-
-                frozen = true;
+                List<CQL3Type.Raw> frozenTypes =
+                    types.stream()
+                         .map(t -> t.supportsFreezing() ? t.freeze() : t)
+                         .collect(toList());
+                return new RawTuple(frozenTypes, true);
             }
 
             public CQL3Type prepare(String keyspace, Types udts) throws InvalidRequestException
             {
-                if (!frozen)
-                    freeze();
+                RawTuple raw = frozen ? this : freeze();
 
-                List<AbstractType<?>> ts = new ArrayList<>(types.size());
-                for (CQL3Type.Raw t : types)
+                List<AbstractType<?>> ts = new ArrayList<>(raw.types.size());
+                for (CQL3Type.Raw t : raw.types)
                 {
                     if (t.isCounter())
                         throw new InvalidRequestException("Counters are not allowed inside tuples");

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/cql3/CQLStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/CQLStatement.java b/src/java/org/apache/cassandra/cql3/CQLStatement.java
index 1e4dad3..c34e27f 100644
--- a/src/java/org/apache/cassandra/cql3/CQLStatement.java
+++ b/src/java/org/apache/cassandra/cql3/CQLStatement.java
@@ -17,34 +17,58 @@
  */
 package org.apache.cassandra.cql3;
 
-import org.apache.cassandra.audit.IAuditLogContext;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.cassandra.audit.AuditLogContext;
 import org.apache.cassandra.cql3.functions.Function;
-import org.apache.cassandra.exceptions.*;
 import org.apache.cassandra.service.ClientState;
 import org.apache.cassandra.service.QueryState;
 import org.apache.cassandra.transport.messages.ResultMessage;
 
-public interface CQLStatement extends IAuditLogContext
+public interface CQLStatement
 {
     /**
-     * Returns the number of bound terms in this statement.
+     * Returns all bind variables for the statement
      */
-    public int getBoundTerms();
+    default List<ColumnSpecification> getBindVariables()
+    {
+        return Collections.emptyList();
+    }
+
+    /**
+     * Returns an array with the same length as the number of partition key columns for the table corresponding
+     * to table.  Each short in the array represents the bind index of the marker that holds the value for that
+     * partition key column. If there are no bind markers for any of the partition key columns, null is returned.
+     */
+    default short[] getPartitionKeyBindVariableIndexes()
+    {
+        return null;
+    }
+
+    /**
+     * Return an Iterable over all of the functions (both native and user-defined) used by any component of the statement
+     *
+     * @return functions all functions found (may contain duplicates)
+     */
+    default Iterable<Function> getFunctions()
+    {
+        return Collections.emptyList();
+    }
 
     /**
      * Perform any access verification necessary for the statement.
      *
      * @param state the current client state
      */
-    public void checkAccess(ClientState state) throws UnauthorizedException, InvalidRequestException;
+    public void authorize(ClientState state);
 
     /**
-     * Perform additional validation required by the statment.
-     * To be overriden by subclasses if needed.
+     * Perform additional validation required by the statment. To be overriden by subclasses if needed.
      *
      * @param state the current client state
      */
-    public void validate(ClientState state) throws RequestValidationException;
+    public void validate(ClientState state);
 
     /**
      * Execute the statement and return the resulting result or null if there is no result.
@@ -53,21 +77,19 @@ public interface CQLStatement extends IAuditLogContext
      * @param options options for this query (consistency, variables, pageSize, ...)
      * @param queryStartNanoTime the timestamp returned by System.nanoTime() when this statement was received
      */
-    public ResultMessage execute(QueryState state, QueryOptions options, long queryStartNanoTime) throws RequestValidationException, RequestExecutionException;
+    public ResultMessage execute(QueryState state, QueryOptions options, long queryStartNanoTime);
 
     /**
      * Variant of execute used for internal query against the system tables, and thus only query the local node.
      *
      * @param state the current query state
      */
-    public ResultMessage executeInternal(QueryState state, QueryOptions options) throws RequestValidationException, RequestExecutionException;
+    public ResultMessage executeLocally(QueryState state, QueryOptions options);
 
     /**
-     * Return an Iterable over all of the functions (both native and user-defined) used by any component
-     * of the statement
-     * @return functions all functions found (may contain duplicates)
+     * Provides the context needed for audit logging statements.
      */
-    public Iterable<Function> getFunctions();
+    AuditLogContext getAuditLogContext();
 
     /**
      * Whether or not this CQL Statement has LWT conditions
@@ -76,4 +98,16 @@ public interface CQLStatement extends IAuditLogContext
     {
         return false;
     }
+
+    public static abstract class Raw
+    {
+        protected VariableSpecifications bindVariables;
+
+        public void setBindVariables(List<ColumnIdentifier> variables)
+        {
+            bindVariables = new VariableSpecifications(variables);
+        }
+
+        public abstract CQLStatement prepare(ClientState state);
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/cql3/CustomPayloadMirroringQueryHandler.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/CustomPayloadMirroringQueryHandler.java b/src/java/org/apache/cassandra/cql3/CustomPayloadMirroringQueryHandler.java
index 32cddba..0906d2a 100644
--- a/src/java/org/apache/cassandra/cql3/CustomPayloadMirroringQueryHandler.java
+++ b/src/java/org/apache/cassandra/cql3/CustomPayloadMirroringQueryHandler.java
@@ -21,7 +21,6 @@ import java.nio.ByteBuffer;
 import java.util.Map;
 
 import org.apache.cassandra.cql3.statements.BatchStatement;
-import org.apache.cassandra.cql3.statements.ParsedStatement;
 import org.apache.cassandra.service.ClientState;
 import org.apache.cassandra.service.QueryState;
 import org.apache.cassandra.transport.messages.ResultMessage;
@@ -54,7 +53,7 @@ public class CustomPayloadMirroringQueryHandler implements QueryHandler
         return prepared;
     }
 
-    public ParsedStatement.Prepared getPrepared(MD5Digest id)
+    public QueryProcessor.Prepared getPrepared(MD5Digest id)
     {
         return queryProcessor.getPrepared(id);
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/cql3/IndexName.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/IndexName.java b/src/java/org/apache/cassandra/cql3/IndexName.java
deleted file mode 100644
index d7ff8ff..0000000
--- a/src/java/org/apache/cassandra/cql3/IndexName.java
+++ /dev/null
@@ -1,47 +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;
-
-public final class IndexName extends KeyspaceElementName
-{
-    private String idxName;
-
-    public void setIndex(String idx, boolean keepCase)
-    {
-        idxName = toInternalName(idx, keepCase);
-    }
-
-    public String getIdx()
-    {
-        return idxName;
-    }
-
-    public CFName getCfName()
-    {
-        CFName cfName = new CFName();
-        if (hasKeyspace())
-            cfName.setKeyspace(getKeyspace(), true);
-    	return cfName;
-    }
-
-    @Override
-    public String toString()
-    {
-        return super.toString() + idxName;
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/cql3/KeyspaceElementName.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/KeyspaceElementName.java b/src/java/org/apache/cassandra/cql3/KeyspaceElementName.java
deleted file mode 100644
index 0a68997..0000000
--- a/src/java/org/apache/cassandra/cql3/KeyspaceElementName.java
+++ /dev/null
@@ -1,74 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.cassandra.cql3;
-
-import java.util.Locale;
-
-/**
- * Base class for the names of the keyspace elements (e.g. table, index ...)
- */
-abstract class KeyspaceElementName
-{
-    /**
-     * The keyspace name as stored internally.
-     */
-    private String ksName;
-
-    /**
-     * Sets the keyspace.
-     *
-     * @param ks the keyspace name
-     * @param keepCase <code>true</code> if the case must be kept, <code>false</code> otherwise.
-     */
-    public final void setKeyspace(String ks, boolean keepCase)
-    {
-        ksName = toInternalName(ks, keepCase);
-    }
-
-    /**
-     * Checks if the keyspace is specified.
-     * @return <code>true</code> if the keyspace is specified, <code>false</code> otherwise.
-     */
-    public final boolean hasKeyspace()
-    {
-        return ksName != null;
-    }
-
-    public final String getKeyspace()
-    {
-        return ksName;
-    }
-
-    /**
-     * Converts the specified name into the name used internally.
-     *
-     * @param name the name
-     * @param keepCase <code>true</code> if the case must be kept, <code>false</code> otherwise.
-     * @return the name used internally.
-     */
-    protected static String toInternalName(String name, boolean keepCase)
-    {
-        return keepCase ? name : name.toLowerCase(Locale.US);
-    }
-
-    @Override
-    public String toString()
-    {
-        return hasKeyspace() ? (getKeyspace() + ".") : "";
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/cql3/MultiColumnRelation.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/MultiColumnRelation.java b/src/java/org/apache/cassandra/cql3/MultiColumnRelation.java
index 411af07..2d239fb 100644
--- a/src/java/org/apache/cassandra/cql3/MultiColumnRelation.java
+++ b/src/java/org/apache/cassandra/cql3/MultiColumnRelation.java
@@ -19,6 +19,7 @@ package org.apache.cassandra.cql3;
 
 import java.util.ArrayList;
 import java.util.List;
+import java.util.Objects;
 import java.util.stream.Collectors;
 
 import org.apache.cassandra.schema.ColumnMetadata;
@@ -240,4 +241,26 @@ public class MultiColumnRelation extends Relation
                       .append(valuesOrMarker)
                       .toString();
     }
+
+    @Override
+    public int hashCode()
+    {
+        return Objects.hash(relationType, entities, valuesOrMarker, inValues, inMarker);
+    }
+
+    @Override
+    public boolean equals(Object o)
+    {
+        if (this == o)
+            return true;
+
+        if (!(o instanceof MultiColumnRelation))
+            return false;
+
+        MultiColumnRelation mcr = (MultiColumnRelation) o;
+        return Objects.equals(entities, mcr.entities)
+            && Objects.equals(valuesOrMarker, mcr.valuesOrMarker)
+            && Objects.equals(inValues, mcr.inValues)
+            && Objects.equals(inMarker, mcr.inMarker);
+    }
 }


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


[04/13] cassandra git commit: Make all DDL statements idempotent and not dependent on global state

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/schema/Schema.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/schema/Schema.java b/src/java/org/apache/cassandra/schema/Schema.java
index 09ec62a..fc09c24 100644
--- a/src/java/org/apache/cassandra/schema/Schema.java
+++ b/src/java/org/apache/cassandra/schema/Schema.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.schema;
 
-import java.nio.ByteBuffer;
 import java.util.*;
 import java.util.concurrent.CopyOnWriteArrayList;
 import java.util.stream.Collectors;
@@ -38,8 +37,13 @@ import org.apache.cassandra.db.virtual.VirtualKeyspaceRegistry;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.exceptions.UnknownTableException;
+import org.apache.cassandra.gms.ApplicationState;
+import org.apache.cassandra.gms.Gossiper;
 import org.apache.cassandra.io.sstable.Descriptor;
 import org.apache.cassandra.locator.LocalStrategy;
+import org.apache.cassandra.schema.KeyspaceMetadata.KeyspaceDiff;
+import org.apache.cassandra.schema.Keyspaces.KeyspacesDiff;
+import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.Pair;
 import org.cliffc.high_scale_lib.NonBlockingHashMap;
 
@@ -79,20 +83,6 @@ public final class Schema
     }
 
     /**
-     * Validates that the provided keyspace is not one of the system keyspace.
-     *
-     * @param keyspace the keyspace name to validate.
-     *
-     * @throws InvalidRequestException if {@code keyspace} is the name of a
-     * system keyspace.
-     */
-    public static void validateKeyspaceNotSystem(String keyspace)
-    {
-        if (SchemaConstants.isLocalSystemKeyspace(keyspace))
-            throw new InvalidRequestException(format("%s keyspace is not user-modifiable", keyspace));
-    }
-
-    /**
      * load keyspace (keyspace) definitions, but do not initialize the keyspace instances.
      * Schema version may be updated as the result.
      */
@@ -108,22 +98,12 @@ public final class Schema
      */
     public void loadFromDisk(boolean updateVersion)
     {
-        load(SchemaKeyspace.fetchNonSystemKeyspaces());
+        SchemaKeyspace.fetchNonSystemKeyspaces().forEach(this::load);
         if (updateVersion)
             updateVersion();
     }
 
     /**
-     * Load up non-system keyspaces
-     *
-     * @param keyspaceDefs The non-system keyspace definitions
-     */
-    private void load(Iterable<KeyspaceMetadata> keyspaceDefs)
-    {
-        keyspaceDefs.forEach(this::load);
-    }
-
-    /**
      * Update (or insert) new keyspace definition
      *
      * @param ksm The metadata about keyspace
@@ -153,50 +133,33 @@ public final class Schema
     private void reload(KeyspaceMetadata previous, KeyspaceMetadata updated)
     {
         Keyspace keyspace = getKeyspaceInstance(updated.name);
-        if (keyspace != null)
+        if (null != keyspace)
             keyspace.setMetadata(updated);
 
-        MapDifference<TableId, TableMetadata> tablesDiff = previous.tables.diff(updated.tables);
-        MapDifference<TableId, ViewMetadata> viewsDiff = previous.views.diff(updated.views);
+        Tables.TablesDiff tablesDiff = Tables.diff(previous.tables, updated.tables);
+        Views.ViewsDiff viewsDiff = Views.diff(previous.views, updated.views);
+
         MapDifference<String, TableMetadata> indexesDiff = previous.tables.indexesDiff(updated.tables);
 
         // clean up after removed entries
-
-        tablesDiff.entriesOnlyOnLeft()
-                  .values()
-                  .forEach(table -> metadataRefs.remove(table.id));
-
-        viewsDiff.entriesOnlyOnLeft()
-                 .values()
-                 .forEach(view -> metadataRefs.remove(view.metadata.id));
+        tablesDiff.dropped.forEach(table -> metadataRefs.remove(table.id));
+        viewsDiff.dropped.forEach(view -> metadataRefs.remove(view.metadata.id));
 
         indexesDiff.entriesOnlyOnLeft()
                    .values()
                    .forEach(indexTable -> indexMetadataRefs.remove(Pair.create(indexTable.keyspace, indexTable.indexName().get())));
 
         // load up new entries
-
-        tablesDiff.entriesOnlyOnRight()
-                  .values()
-                  .forEach(table -> metadataRefs.put(table.id, new TableMetadataRef(table)));
-
-        viewsDiff.entriesOnlyOnRight()
-                 .values()
-                 .forEach(view -> metadataRefs.put(view.metadata.id, new TableMetadataRef(view.metadata)));
+        tablesDiff.created.forEach(table -> metadataRefs.put(table.id, new TableMetadataRef(table)));
+        viewsDiff.created.forEach(view -> metadataRefs.put(view.metadata.id, new TableMetadataRef(view.metadata)));
 
         indexesDiff.entriesOnlyOnRight()
                    .values()
                    .forEach(indexTable -> indexMetadataRefs.put(Pair.create(indexTable.keyspace, indexTable.indexName().get()), new TableMetadataRef(indexTable)));
 
         // refresh refs to updated ones
-
-        tablesDiff.entriesDiffering()
-                  .values()
-                  .forEach(diff -> metadataRefs.get(diff.rightValue().id).set(diff.rightValue()));
-
-        viewsDiff.entriesDiffering()
-                 .values()
-                 .forEach(diff -> metadataRefs.get(diff.rightValue().metadata.id).set(diff.rightValue().metadata));
+        tablesDiff.altered.forEach(diff -> metadataRefs.get(diff.after.id).set(diff.after));
+        viewsDiff.altered.forEach(diff -> metadataRefs.get(diff.after.metadata.id).set(diff.after.metadata));
 
         indexesDiff.entriesDiffering()
                    .values()
@@ -556,7 +519,16 @@ public final class Schema
     public void updateVersionAndAnnounce()
     {
         updateVersion();
-        MigrationManager.passiveAnnounce(version);
+        passiveAnnounceVersion();
+    }
+
+    /**
+     * Announce my version passively over gossip.
+     * Used to notify nodes as they arrive in the cluster.
+     */
+    private void passiveAnnounceVersion()
+    {
+        Gossiper.instance.addLocalApplicationState(ApplicationState.SCHEMA, StorageService.instance.valueFactory.schema(version));
     }
 
     /**
@@ -576,7 +548,7 @@ public final class Schema
     {
         Keyspaces before = keyspaces.filter(k -> !SchemaConstants.isLocalSystemKeyspace(k.name));
         Keyspaces after = SchemaKeyspace.fetchNonSystemKeyspaces();
-        merge(before, after);
+        merge(Keyspaces.diff(before, after));
         updateVersionAndAnnounce();
     }
 
@@ -594,6 +566,60 @@ public final class Schema
         updateVersionAndAnnounce();
     }
 
+    public synchronized TransformationResult transform(SchemaTransformation transformation, boolean locally, long now)
+    {
+        KeyspacesDiff diff;
+        try
+        {
+            Keyspaces before = keyspaces;
+            Keyspaces after = transformation.apply(before);
+            diff = Keyspaces.diff(before, after);
+        }
+        catch (RuntimeException e)
+        {
+            return new TransformationResult(e);
+        }
+
+        if (diff.isEmpty())
+            return new TransformationResult(diff, Collections.emptyList());
+
+        Collection<Mutation> mutations = SchemaKeyspace.convertSchemaDiffToMutations(diff, now);
+        SchemaKeyspace.applyChanges(mutations);
+
+        merge(diff);
+        updateVersion();
+        if (!locally)
+            passiveAnnounceVersion();
+
+        return new TransformationResult(diff, mutations);
+    }
+
+    public static final class TransformationResult
+    {
+        public final boolean success;
+        public final RuntimeException exception;
+        public final KeyspacesDiff diff;
+        public final Collection<Mutation> mutations;
+
+        private TransformationResult(boolean success, RuntimeException exception, KeyspacesDiff diff, Collection<Mutation> mutations)
+        {
+            this.success = success;
+            this.exception = exception;
+            this.diff = diff;
+            this.mutations = mutations;
+        }
+
+        TransformationResult(RuntimeException exception)
+        {
+            this(false, exception, null, null);
+        }
+
+        TransformationResult(KeyspacesDiff diff, Collection<Mutation> mutations)
+        {
+            this(true, null, diff, mutations);
+        }
+    }
+
     synchronized void merge(Collection<Mutation> mutations)
     {
         // only compare the keyspaces affected by this set of schema mutations
@@ -608,71 +634,57 @@ public final class Schema
         // apply the schema mutations and fetch the new versions of the altered keyspaces
         Keyspaces after = SchemaKeyspace.fetchKeyspaces(affectedKeyspaces);
 
-        merge(before, after);
+        merge(Keyspaces.diff(before, after));
     }
 
-    private synchronized void merge(Keyspaces before, Keyspaces after)
+    private void merge(KeyspacesDiff diff)
     {
-        MapDifference<String, KeyspaceMetadata> keyspacesDiff = before.diff(after);
-
-        // dropped keyspaces
-        keyspacesDiff.entriesOnlyOnLeft().values().forEach(this::dropKeyspace);
-
-        // new keyspaces
-        keyspacesDiff.entriesOnlyOnRight().values().forEach(this::createKeyspace);
-
-        // updated keyspaces
-        keyspacesDiff.entriesDiffering().entrySet().forEach(diff -> alterKeyspace(diff.getValue().leftValue(), diff.getValue().rightValue()));
+        diff.dropped.forEach(this::dropKeyspace);
+        diff.created.forEach(this::createKeyspace);
+        diff.altered.forEach(this::alterKeyspace);
     }
 
-    private void alterKeyspace(KeyspaceMetadata before, KeyspaceMetadata after)
+    private void alterKeyspace(KeyspaceDiff delta)
     {
-        // calculate the deltas
-        MapDifference<TableId, TableMetadata> tablesDiff = before.tables.diff(after.tables);
-        MapDifference<TableId, ViewMetadata> viewsDiff = before.views.diff(after.views);
-        MapDifference<ByteBuffer, UserType> typesDiff = before.types.diff(after.types);
-        MapDifference<Pair<FunctionName, List<String>>, UDFunction> udfsDiff = before.functions.udfsDiff(after.functions);
-        MapDifference<Pair<FunctionName, List<String>>, UDAggregate> udasDiff = before.functions.udasDiff(after.functions);
-
         // drop tables and views
-        viewsDiff.entriesOnlyOnLeft().values().forEach(this::dropView);
-        tablesDiff.entriesOnlyOnLeft().values().forEach(this::dropTable);
+        delta.views.dropped.forEach(this::dropView);
+        delta.tables.dropped.forEach(this::dropTable);
 
-        load(after);
+        load(delta.after);
 
         // add tables and views
-        tablesDiff.entriesOnlyOnRight().values().forEach(this::createTable);
-        viewsDiff.entriesOnlyOnRight().values().forEach(this::createView);
+        delta.tables.created.forEach(this::createTable);
+        delta.views.created.forEach(this::createView);
 
         // update tables and views
-        tablesDiff.entriesDiffering().values().forEach(diff -> alterTable(diff.rightValue()));
-        viewsDiff.entriesDiffering().values().forEach(diff -> alterView(diff.rightValue()));
+        delta.tables.altered.forEach(diff -> alterTable(diff.after));
+        delta.views.altered.forEach(diff -> alterView(diff.after));
 
         // deal with all removed, added, and altered views
-        Keyspace.open(before.name).viewManager.reload(true);
+        Keyspace.open(delta.after.name).viewManager.reload(true);
 
         // notify on everything dropped
-        udasDiff.entriesOnlyOnLeft().values().forEach(this::notifyDropAggregate);
-        udfsDiff.entriesOnlyOnLeft().values().forEach(this::notifyDropFunction);
-        viewsDiff.entriesOnlyOnLeft().values().forEach(this::notifyDropView);
-        tablesDiff.entriesOnlyOnLeft().values().forEach(this::notifyDropTable);
-        typesDiff.entriesOnlyOnLeft().values().forEach(this::notifyDropType);
+        delta.udas.dropped.forEach(uda -> notifyDropAggregate((UDAggregate) uda));
+        delta.udfs.dropped.forEach(udf -> notifyDropFunction((UDFunction) udf));
+        delta.views.dropped.forEach(this::notifyDropView);
+        delta.tables.dropped.forEach(this::notifyDropTable);
+        delta.types.dropped.forEach(this::notifyDropType);
 
         // notify on everything created
-        typesDiff.entriesOnlyOnRight().values().forEach(this::notifyCreateType);
-        tablesDiff.entriesOnlyOnRight().values().forEach(this::notifyCreateTable);
-        viewsDiff.entriesOnlyOnRight().values().forEach(this::notifyCreateView);
-        udfsDiff.entriesOnlyOnRight().values().forEach(this::notifyCreateFunction);
-        udasDiff.entriesOnlyOnRight().values().forEach(this::notifyCreateAggregate);
+        delta.types.created.forEach(this::notifyCreateType);
+        delta.tables.created.forEach(this::notifyCreateTable);
+        delta.views.created.forEach(this::notifyCreateView);
+        delta.udfs.created.forEach(udf -> notifyCreateFunction((UDFunction) udf));
+        delta.udas.created.forEach(uda -> notifyCreateAggregate((UDAggregate) uda));
 
         // notify on everything altered
-        if (!before.params.equals(after.params))
-            notifyAlterKeyspace(after);
-        typesDiff.entriesDiffering().values().forEach(diff -> notifyAlterType(diff.rightValue()));
-        tablesDiff.entriesDiffering().values().forEach(diff -> notifyAlterTable(diff.leftValue(), diff.rightValue()));
-        viewsDiff.entriesDiffering().values().forEach(diff -> notifyAlterView(diff.leftValue(), diff.rightValue()));
-        udfsDiff.entriesDiffering().values().forEach(diff -> notifyAlterFunction(diff.rightValue()));
-        udasDiff.entriesDiffering().values().forEach(diff -> notifyAlterAggregate(diff.rightValue()));
+        if (!delta.before.params.equals(delta.after.params))
+            notifyAlterKeyspace(delta.before, delta.after);
+        delta.types.altered.forEach(diff -> notifyAlterType(diff.before, diff.after));
+        delta.tables.altered.forEach(diff -> notifyAlterTable(diff.before, diff.after));
+        delta.views.altered.forEach(diff -> notifyAlterView(diff.before, diff.after));
+        delta.udfs.altered.forEach(diff -> notifyAlterFunction(diff.before, diff.after));
+        delta.udas.altered.forEach(diff -> notifyAlterAggregate(diff.before, diff.after));
     }
 
     private void createKeyspace(KeyspaceMetadata keyspace)
@@ -708,7 +720,7 @@ public final class Schema
 
     private void dropView(ViewMetadata metadata)
     {
-        Keyspace.open(metadata.keyspace).viewManager.stopBuild(metadata.name);
+        Keyspace.open(metadata.keyspace()).viewManager.stopBuild(metadata.name());
         dropTable(metadata.metadata);
     }
 
@@ -732,7 +744,7 @@ public final class Schema
 
     private void createView(ViewMetadata view)
     {
-        Keyspace.open(view.keyspace).initCf(metadataRefs.get(view.metadata.id), true);
+        Keyspace.open(view.keyspace()).initCf(metadataRefs.get(view.metadata.id), true);
     }
 
     private void alterTable(TableMetadata updated)
@@ -742,7 +754,7 @@ public final class Schema
 
     private void alterView(ViewMetadata updated)
     {
-        Keyspace.open(updated.keyspace).getColumnFamilyStore(updated.name).reload();
+        Keyspace.open(updated.keyspace()).getColumnFamilyStore(updated.name()).reload();
     }
 
     private void notifyCreateKeyspace(KeyspaceMetadata ksm)
@@ -757,7 +769,7 @@ public final class Schema
 
     private void notifyCreateView(ViewMetadata view)
     {
-        changeListeners.forEach(l -> l.onCreateView(view.keyspace, view.name));
+        changeListeners.forEach(l -> l.onCreateView(view.keyspace(), view.name()));
     }
 
     private void notifyCreateType(UserType ut)
@@ -775,36 +787,36 @@ public final class Schema
         changeListeners.forEach(l -> l.onCreateAggregate(udf.name().keyspace, udf.name().name, udf.argTypes()));
     }
 
-    private void notifyAlterKeyspace(KeyspaceMetadata ksm)
+    private void notifyAlterKeyspace(KeyspaceMetadata before, KeyspaceMetadata after)
     {
-        changeListeners.forEach(l -> l.onAlterKeyspace(ksm.name));
+        changeListeners.forEach(l -> l.onAlterKeyspace(after.name));
     }
 
-    private void notifyAlterTable(TableMetadata current, TableMetadata updated)
+    private void notifyAlterTable(TableMetadata before, TableMetadata after)
     {
-        boolean changeAffectedPreparedStatements = current.changeAffectsPreparedStatements(updated);
-        changeListeners.forEach(l -> l.onAlterTable(updated.keyspace, updated.name, changeAffectedPreparedStatements));
+        boolean changeAffectedPreparedStatements = before.changeAffectsPreparedStatements(after);
+        changeListeners.forEach(l -> l.onAlterTable(after.keyspace, after.name, changeAffectedPreparedStatements));
     }
 
-    private void notifyAlterView(ViewMetadata current, ViewMetadata updated)
+    private void notifyAlterView(ViewMetadata before, ViewMetadata after)
     {
-        boolean changeAffectedPreparedStatements = current.metadata.changeAffectsPreparedStatements(updated.metadata);
-        changeListeners.forEach(l ->l.onAlterView(updated.keyspace, updated.name, changeAffectedPreparedStatements));
+        boolean changeAffectedPreparedStatements = before.metadata.changeAffectsPreparedStatements(after.metadata);
+        changeListeners.forEach(l ->l.onAlterView(after.keyspace(), after.name(), changeAffectedPreparedStatements));
     }
 
-    private void notifyAlterType(UserType ut)
+    private void notifyAlterType(UserType before, UserType after)
     {
-        changeListeners.forEach(l -> l.onAlterType(ut.keyspace, ut.getNameAsString()));
+        changeListeners.forEach(l -> l.onAlterType(after.keyspace, after.getNameAsString()));
     }
 
-    private void notifyAlterFunction(UDFunction udf)
+    private void notifyAlterFunction(UDFunction before, UDFunction after)
     {
-        changeListeners.forEach(l -> l.onAlterFunction(udf.name().keyspace, udf.name().name, udf.argTypes()));
+        changeListeners.forEach(l -> l.onAlterFunction(after.name().keyspace, after.name().name, after.argTypes()));
     }
 
-    private void notifyAlterAggregate(UDAggregate udf)
+    private void notifyAlterAggregate(UDAggregate before, UDAggregate after)
     {
-        changeListeners.forEach(l -> l.onAlterAggregate(udf.name().keyspace, udf.name().name, udf.argTypes()));
+        changeListeners.forEach(l -> l.onAlterAggregate(after.name().keyspace, after.name().name, after.argTypes()));
     }
 
     private void notifyDropKeyspace(KeyspaceMetadata ksm)
@@ -819,7 +831,7 @@ public final class Schema
 
     private void notifyDropView(ViewMetadata view)
     {
-        changeListeners.forEach(l -> l.onDropView(view.keyspace, view.name));
+        changeListeners.forEach(l -> l.onDropView(view.keyspace(), view.name()));
     }
 
     private void notifyDropType(UserType ut)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/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 4945fc2..553ccdc 100644
--- a/src/java/org/apache/cassandra/schema/SchemaKeyspace.java
+++ b/src/java/org/apache/cassandra/schema/SchemaKeyspace.java
@@ -30,20 +30,20 @@ import com.google.common.hash.Hasher;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.antlr.runtime.RecognitionException;
 import org.apache.cassandra.config.*;
-import org.apache.cassandra.cql3.statements.CreateTableStatement;
-import org.apache.cassandra.schema.ColumnMetadata.ClusteringOrder;
 import org.apache.cassandra.cql3.*;
 import org.apache.cassandra.cql3.functions.*;
-import org.apache.cassandra.cql3.statements.SelectStatement;
+import org.apache.cassandra.cql3.statements.schema.CreateTableStatement;
 import org.apache.cassandra.db.*;
+import org.apache.cassandra.db.filter.ColumnFilter;
 import org.apache.cassandra.db.marshal.*;
 import org.apache.cassandra.db.partitions.*;
 import org.apache.cassandra.db.rows.*;
-import org.apache.cassandra.db.filter.ColumnFilter;
-import org.apache.cassandra.db.view.View;
 import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.service.reads.SpeculativeRetryPolicy;
+import org.apache.cassandra.schema.ColumnMetadata.ClusteringOrder;
+import org.apache.cassandra.schema.Keyspaces.KeyspacesDiff;
 import org.apache.cassandra.transport.ProtocolVersion;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
@@ -53,6 +53,7 @@ import static java.lang.String.format;
 
 import static java.util.stream.Collectors.toList;
 import static java.util.stream.Collectors.toSet;
+
 import static org.apache.cassandra.cql3.QueryProcessor.executeInternal;
 import static org.apache.cassandra.cql3.QueryProcessor.executeOnceInternal;
 
@@ -270,6 +271,44 @@ public final class SchemaKeyspace
         return KeyspaceMetadata.create(SchemaConstants.SCHEMA_KEYSPACE_NAME, KeyspaceParams.local(), org.apache.cassandra.schema.Tables.of(ALL_TABLE_METADATA));
     }
 
+    static Collection<Mutation> convertSchemaDiffToMutations(KeyspacesDiff diff, long timestamp)
+    {
+        Map<String, Mutation> mutations = new HashMap<>();
+
+        diff.created.forEach(k -> mutations.put(k.name, makeCreateKeyspaceMutation(k, timestamp).build()));
+        diff.dropped.forEach(k -> mutations.put(k.name, makeDropKeyspaceMutation(k, timestamp).build()));
+        diff.altered.forEach(kd ->
+        {
+            KeyspaceMetadata ks = kd.after;
+
+            Mutation.SimpleBuilder builder = makeCreateKeyspaceMutation(ks.name, ks.params, timestamp);
+
+            kd.types.created.forEach(t -> addTypeToSchemaMutation(t, builder));
+            kd.types.dropped.forEach(t -> addDropTypeToSchemaMutation(t, builder));
+            kd.types.altered(Difference.SHALLOW).forEach(td -> addTypeToSchemaMutation(td.after, builder));
+
+            kd.tables.created.forEach(t -> addTableToSchemaMutation(t, true, builder));
+            kd.tables.dropped.forEach(t -> addDropTableToSchemaMutation(t, builder));
+            kd.tables.altered(Difference.SHALLOW).forEach(td -> addAlterTableToSchemaMutation(td.before, td.after, builder));
+
+            kd.views.created.forEach(v -> addViewToSchemaMutation(v, true, builder));
+            kd.views.dropped.forEach(v -> addDropViewToSchemaMutation(v, builder));
+            kd.views.altered(Difference.SHALLOW).forEach(vd -> addAlterViewToSchemaMutation(vd.before, vd.after, builder));
+
+            kd.udfs.created.forEach(f -> addFunctionToSchemaMutation((UDFunction) f, builder));
+            kd.udfs.dropped.forEach(f -> addDropFunctionToSchemaMutation((UDFunction) f, builder));
+            kd.udfs.altered(Difference.SHALLOW).forEach(fd -> addFunctionToSchemaMutation(fd.after, builder));
+
+            kd.udas.created.forEach(a -> addAggregateToSchemaMutation((UDAggregate) a, builder));
+            kd.udas.dropped.forEach(a -> addDropAggregateToSchemaMutation((UDAggregate) a, builder));
+            kd.udas.altered(Difference.SHALLOW).forEach(ad -> addAggregateToSchemaMutation(ad.after, builder));
+
+            mutations.put(ks.name, builder.build());
+        });
+
+        return mutations.values();
+    }
+
     /**
      * Add entries to system_schema.* for the hardcoded system keyspaces
      */
@@ -298,7 +337,7 @@ public final class SchemaKeyspace
         ALL.reverse().forEach(table -> getSchemaCFS(table).truncateBlocking());
     }
 
-    static void flush()
+    private static void flush()
     {
         if (!DatabaseDescriptor.isUnsafeSystem())
             ALL.forEach(table -> FBUtilities.waitOnFuture(getSchemaCFS(table).forceFlush()));
@@ -463,15 +502,7 @@ public final class SchemaKeyspace
         return builder;
     }
 
-    static Mutation.SimpleBuilder makeCreateTypeMutation(KeyspaceMetadata keyspace, UserType type, long timestamp)
-    {
-        // Include the serialized keyspace in case the target node missed a CREATE KEYSPACE migration (see CASSANDRA-5631).
-        Mutation.SimpleBuilder builder = makeCreateKeyspaceMutation(keyspace.name, keyspace.params, timestamp);
-        addTypeToSchemaMutation(type, builder);
-        return builder;
-    }
-
-    static void addTypeToSchemaMutation(UserType type, Mutation.SimpleBuilder mutation)
+    private static void addTypeToSchemaMutation(UserType type, Mutation.SimpleBuilder mutation)
     {
         mutation.update(Types)
                 .row(type.getNameAsString())
@@ -479,12 +510,9 @@ public final class SchemaKeyspace
                 .add("field_types", type.fieldTypes().stream().map(AbstractType::asCQL3Type).map(CQL3Type::toString).collect(toList()));
     }
 
-    static Mutation.SimpleBuilder dropTypeFromSchemaMutation(KeyspaceMetadata keyspace, UserType type, long timestamp)
+    private static void addDropTypeToSchemaMutation(UserType type, Mutation.SimpleBuilder builder)
     {
-        // Include the serialized keyspace in case the target node missed a CREATE KEYSPACE migration (see CASSANDRA-5631).
-        Mutation.SimpleBuilder builder = makeCreateKeyspaceMutation(keyspace.name, keyspace.params, timestamp);
         builder.update(Types).row(type.name).delete();
-        return builder;
     }
 
     static Mutation.SimpleBuilder makeCreateTableMutation(KeyspaceMetadata keyspace, TableMetadata table, long timestamp)
@@ -495,7 +523,7 @@ public final class SchemaKeyspace
         return builder;
     }
 
-    static void addTableToSchemaMutation(TableMetadata table, boolean withColumnsAndTriggers, Mutation.SimpleBuilder builder)
+    private static void addTableToSchemaMutation(TableMetadata table, boolean withColumnsAndTriggers, Mutation.SimpleBuilder builder)
     {
         Row.SimpleBuilder rowBuilder = builder.update(Tables)
                                               .row(table.name)
@@ -544,13 +572,8 @@ public final class SchemaKeyspace
             builder.add("cdc", params.cdc);
     }
 
-    static Mutation.SimpleBuilder makeUpdateTableMutation(KeyspaceMetadata keyspace,
-                                                                 TableMetadata oldTable,
-                                                                 TableMetadata newTable,
-                                                                 long timestamp)
+    private static void addAlterTableToSchemaMutation(TableMetadata oldTable, TableMetadata newTable, Mutation.SimpleBuilder builder)
     {
-        Mutation.SimpleBuilder builder = makeCreateKeyspaceMutation(keyspace.name, keyspace.params, timestamp);
-
         addTableToSchemaMutation(newTable, false, builder);
 
         MapDifference<ByteBuffer, ColumnMetadata> columnDiff = Maps.difference(oldTable.columns, newTable.columns);
@@ -602,7 +625,15 @@ public final class SchemaKeyspace
         // updated indexes need to be updated
         for (MapDifference.ValueDifference<IndexMetadata> diff : indexesDiff.entriesDiffering().values())
             addUpdatedIndexToSchemaMutation(newTable, diff.rightValue(), builder);
+    }
 
+    static Mutation.SimpleBuilder makeUpdateTableMutation(KeyspaceMetadata keyspace,
+                                                          TableMetadata oldTable,
+                                                          TableMetadata newTable,
+                                                          long timestamp)
+    {
+        Mutation.SimpleBuilder builder = makeCreateKeyspaceMutation(keyspace.name, keyspace.params, timestamp);
+        addAlterTableToSchemaMutation(oldTable, newTable, builder);
         return builder;
     }
 
@@ -632,7 +663,12 @@ public final class SchemaKeyspace
     {
         // Include the serialized keyspace in case the target node missed a CREATE KEYSPACE migration (see CASSANDRA-5631).
         Mutation.SimpleBuilder builder = makeCreateKeyspaceMutation(keyspace.name, keyspace.params, timestamp);
+        addDropTableToSchemaMutation(table, builder);
+        return builder;
+    }
 
+    private static void addDropTableToSchemaMutation(TableMetadata table, Mutation.SimpleBuilder builder)
+    {
         builder.update(Tables).row(table.name).delete();
 
         for (ColumnMetadata column : table.columns())
@@ -646,8 +682,6 @@ public final class SchemaKeyspace
 
         for (IndexMetadata index : table.indexes)
             dropIndexFromSchemaMutation(table, index, builder);
-
-        return builder;
     }
 
     private static void addColumnToSchemaMutation(TableMetadata table, ColumnMetadata column, Mutation.SimpleBuilder builder)
@@ -676,7 +710,7 @@ public final class SchemaKeyspace
         builder.update(DroppedColumns)
                .row(table.name, column.column.name.toString())
                .add("dropped_time", new Date(TimeUnit.MICROSECONDS.toMillis(column.droppedTime)))
-               .add("type", expandUserTypes(column.column.type).asCQL3Type().toString())
+               .add("type", column.column.type.asCQL3Type().toString())
                .add("kind", column.column.kind.toString().toLowerCase());
     }
 
@@ -697,23 +731,15 @@ public final class SchemaKeyspace
         builder.update(Triggers).row(table.name, trigger.name).delete();
     }
 
-    static Mutation.SimpleBuilder makeCreateViewMutation(KeyspaceMetadata keyspace, ViewMetadata view, long timestamp)
-    {
-        // Include the serialized keyspace in case the target node missed a CREATE KEYSPACE migration (see CASSANDRA-5631).
-        Mutation.SimpleBuilder builder = makeCreateKeyspaceMutation(keyspace.name, keyspace.params, timestamp);
-        addViewToSchemaMutation(view, true, builder);
-        return builder;
-    }
-
     private static void addViewToSchemaMutation(ViewMetadata view, boolean includeColumns, Mutation.SimpleBuilder builder)
     {
         TableMetadata table = view.metadata;
         Row.SimpleBuilder rowBuilder = builder.update(Views)
-                                              .row(view.name)
+                                              .row(view.name())
                                               .add("include_all_columns", view.includeAllColumns)
                                               .add("base_table_id", view.baseTableId.asUUID())
                                               .add("base_table_name", view.baseTableName)
-                                              .add("where_clause", view.whereClause)
+                                              .add("where_clause", view.whereClause.toString())
                                               .add("id", table.id.asUUID());
 
         addTableParamsToRowBuilder(table.params, rowBuilder);
@@ -728,57 +754,32 @@ public final class SchemaKeyspace
         }
     }
 
-    static Mutation.SimpleBuilder makeDropViewMutation(KeyspaceMetadata keyspace, ViewMetadata view, long timestamp)
+    private static void addDropViewToSchemaMutation(ViewMetadata view, Mutation.SimpleBuilder builder)
     {
-        // Include the serialized keyspace in case the target node missed a CREATE KEYSPACE migration (see CASSANDRA-5631).
-        Mutation.SimpleBuilder builder = makeCreateKeyspaceMutation(keyspace.name, keyspace.params, timestamp);
-
-        builder.update(Views).row(view.name).delete();
+        builder.update(Views).row(view.name()).delete();
 
         TableMetadata table = view.metadata;
         for (ColumnMetadata column : table.columns())
             dropColumnFromSchemaMutation(table, column, builder);
-
-        for (IndexMetadata index : table.indexes)
-            dropIndexFromSchemaMutation(table, index, builder);
-
-        return builder;
     }
 
-    public static Mutation.SimpleBuilder makeUpdateViewMutation(Mutation.SimpleBuilder builder,
-                                                                ViewMetadata oldView,
-                                                                ViewMetadata newView)
+    private static void addAlterViewToSchemaMutation(ViewMetadata before, ViewMetadata after, Mutation.SimpleBuilder builder)
     {
-        addViewToSchemaMutation(newView, false, builder);
+        addViewToSchemaMutation(after, false, builder);
 
-        MapDifference<ByteBuffer, ColumnMetadata> columnDiff = Maps.difference(oldView.metadata.columns,
-                                                                               newView.metadata.columns);
+        MapDifference<ByteBuffer, ColumnMetadata> columnDiff = Maps.difference(before.metadata.columns, after.metadata.columns);
 
         // columns that are no longer needed
         for (ColumnMetadata column : columnDiff.entriesOnlyOnLeft().values())
-            dropColumnFromSchemaMutation(oldView.metadata, column, builder);
+            dropColumnFromSchemaMutation(before.metadata, column, builder);
 
         // newly added columns
         for (ColumnMetadata column : columnDiff.entriesOnlyOnRight().values())
-            addColumnToSchemaMutation(newView.metadata, column, builder);
+            addColumnToSchemaMutation(after.metadata, column, builder);
 
         // old columns with updated attributes
         for (ByteBuffer name : columnDiff.entriesDiffering().keySet())
-            addColumnToSchemaMutation(newView.metadata, newView.metadata.getColumn(name), builder);
-
-        // dropped columns
-        MapDifference<ByteBuffer, DroppedColumn> droppedColumnDiff =
-        Maps.difference(oldView.metadata.droppedColumns, oldView.metadata.droppedColumns);
-
-        // newly dropped columns
-        for (DroppedColumn column : droppedColumnDiff.entriesOnlyOnRight().values())
-            addDroppedColumnToSchemaMutation(oldView.metadata, column, builder);
-
-        // columns added then dropped again
-        for (ByteBuffer name : droppedColumnDiff.entriesDiffering().keySet())
-            addDroppedColumnToSchemaMutation(newView.metadata, newView.metadata.droppedColumns.get(name), builder);
-
-        return builder;
+            addColumnToSchemaMutation(after.metadata, after.metadata.getColumn(name), builder);
     }
 
     private static void addIndexToSchemaMutation(TableMetadata table, IndexMetadata index, Mutation.SimpleBuilder builder)
@@ -801,15 +802,7 @@ public final class SchemaKeyspace
         addIndexToSchemaMutation(table, index, builder);
     }
 
-    static Mutation.SimpleBuilder makeCreateFunctionMutation(KeyspaceMetadata keyspace, UDFunction function, long timestamp)
-    {
-        // Include the serialized keyspace in case the target node missed a CREATE KEYSPACE migration (see CASSANDRA-5631).
-        Mutation.SimpleBuilder builder = makeCreateKeyspaceMutation(keyspace.name, keyspace.params, timestamp);
-        addFunctionToSchemaMutation(function, builder);
-        return builder;
-    }
-
-    static void addFunctionToSchemaMutation(UDFunction function, Mutation.SimpleBuilder builder)
+    private static void addFunctionToSchemaMutation(UDFunction function, Mutation.SimpleBuilder builder)
     {
         builder.update(Functions)
                .row(function.name().name, function.argumentsList())
@@ -832,23 +825,12 @@ public final class SchemaKeyspace
         }
     }
 
-    static Mutation.SimpleBuilder makeDropFunctionMutation(KeyspaceMetadata keyspace, UDFunction function, long timestamp)
+    private static void addDropFunctionToSchemaMutation(UDFunction function, Mutation.SimpleBuilder builder)
     {
-        // Include the serialized keyspace in case the target node missed a CREATE KEYSPACE migration (see CASSANDRA-5631).
-        Mutation.SimpleBuilder builder = makeCreateKeyspaceMutation(keyspace.name, keyspace.params, timestamp);
         builder.update(Functions).row(function.name().name, function.argumentsList()).delete();
-        return builder;
     }
 
-    static Mutation.SimpleBuilder makeCreateAggregateMutation(KeyspaceMetadata keyspace, UDAggregate aggregate, long timestamp)
-    {
-        // Include the serialized keyspace in case the target node missed a CREATE KEYSPACE migration (see CASSANDRA-5631).
-        Mutation.SimpleBuilder builder = makeCreateKeyspaceMutation(keyspace.name, keyspace.params, timestamp);
-        addAggregateToSchemaMutation(aggregate, builder);
-        return builder;
-    }
-
-    static void addAggregateToSchemaMutation(UDAggregate aggregate, Mutation.SimpleBuilder builder)
+    private static void addAggregateToSchemaMutation(UDAggregate aggregate, Mutation.SimpleBuilder builder)
     {
         builder.update(Aggregates)
                .row(aggregate.name().name, aggregate.argumentsList())
@@ -862,12 +844,9 @@ public final class SchemaKeyspace
                                 : null);
     }
 
-    static Mutation.SimpleBuilder makeDropAggregateMutation(KeyspaceMetadata keyspace, UDAggregate aggregate, long timestamp)
+    private static void addDropAggregateToSchemaMutation(UDAggregate aggregate, Mutation.SimpleBuilder builder)
     {
-        // Include the serialized keyspace in case the target node missed a CREATE KEYSPACE migration (see CASSANDRA-5631).
-        Mutation.SimpleBuilder builder = makeCreateKeyspaceMutation(keyspace.name, keyspace.params, timestamp);
         builder.update(Aggregates).row(aggregate.name().name, aggregate.argumentsList()).delete();
-        return builder;
     }
 
     /*
@@ -1017,7 +996,7 @@ public final class SchemaKeyspace
         String query = format("SELECT * FROM %s.%s WHERE keyspace_name = ? AND table_name = ?", SchemaConstants.SCHEMA_KEYSPACE_NAME, COLUMNS);
         UntypedResultSet columnRows = query(query, keyspace, table);
         if (columnRows.isEmpty())
-            throw new MissingColumns("Columns not found in schema table for " + keyspace + "." + table);
+            throw new MissingColumns("Columns not found in schema table for " + keyspace + '.' + table);
 
         List<ColumnMetadata> columns = new ArrayList<>();
         columnRows.forEach(row -> columns.add(createColumnFromRow(row, types)));
@@ -1120,7 +1099,7 @@ public final class SchemaKeyspace
 
         Views.Builder views = org.apache.cassandra.schema.Views.builder();
         for (UntypedResultSet.Row row : query(query, keyspaceName))
-            views.add(fetchView(keyspaceName, row.getString("view_name"), types));
+            views.put(fetchView(keyspaceName, row.getString("view_name"), types));
         return views.build();
     }
 
@@ -1135,7 +1114,7 @@ public final class SchemaKeyspace
         TableId baseTableId = TableId.fromUUID(row.getUUID("base_table_id"));
         String baseTableName = row.getString("base_table_name");
         boolean includeAll = row.getBoolean("include_all_columns");
-        String whereClause = row.getString("where_clause");
+        String whereClauseString = row.getString("where_clause");
 
         List<ColumnMetadata> columns = fetchColumns(keyspaceName, viewName, types);
 
@@ -1147,31 +1126,36 @@ public final class SchemaKeyspace
                          .params(createTableParamsFromRow(row))
                          .build();
 
-        String rawSelect = View.buildSelectStatement(baseTableName, columns, whereClause);
-        SelectStatement.RawStatement rawStatement = (SelectStatement.RawStatement) QueryProcessor.parseStatement(rawSelect);
+        WhereClause whereClause;
 
-        return new ViewMetadata(keyspaceName, viewName, baseTableId, baseTableName, includeAll, rawStatement, whereClause, metadata);
+        try
+        {
+            whereClause = WhereClause.parse(whereClauseString);
+        }
+        catch (RecognitionException e)
+        {
+            throw new RuntimeException(format("Unexpected error while parsing materialized view's where clause for '%s' (got %s)", viewName, whereClauseString));
+        }
+
+        return new ViewMetadata(baseTableId, baseTableName, includeAll, whereClause, metadata);
     }
 
     private static Functions fetchFunctions(String keyspaceName, Types types)
     {
-        Functions udfs = fetchUDFs(keyspaceName, types);
-        Functions udas = fetchUDAs(keyspaceName, udfs, types);
+        Collection<UDFunction> udfs = fetchUDFs(keyspaceName, types);
+        Collection<UDAggregate> udas = fetchUDAs(keyspaceName, udfs, types);
 
-        return org.apache.cassandra.schema.Functions.builder()
-                                                    .add(udfs)
-                                                    .add(udas)
-                                                    .build();
+        return org.apache.cassandra.schema.Functions.builder().add(udfs).add(udas).build();
     }
 
-    private static Functions fetchUDFs(String keyspaceName, Types types)
+    private static Collection<UDFunction> fetchUDFs(String keyspaceName, Types types)
     {
         String query = format("SELECT * FROM %s.%s WHERE keyspace_name = ?", SchemaConstants.SCHEMA_KEYSPACE_NAME, FUNCTIONS);
 
-        Functions.Builder functions = org.apache.cassandra.schema.Functions.builder();
+        Collection<UDFunction> functions = new ArrayList<>();
         for (UntypedResultSet.Row row : query(query, keyspaceName))
             functions.add(createUDFFromRow(row, types));
-        return functions.build();
+        return functions;
     }
 
     private static UDFunction createUDFFromRow(UntypedResultSet.Row row, Types types)
@@ -1230,17 +1214,16 @@ public final class SchemaKeyspace
         }
     }
 
-    private static Functions fetchUDAs(String keyspaceName, Functions udfs, Types types)
+    private static Collection<UDAggregate> fetchUDAs(String keyspaceName, Collection<UDFunction> udfs, Types types)
     {
         String query = format("SELECT * FROM %s.%s WHERE keyspace_name = ?", SchemaConstants.SCHEMA_KEYSPACE_NAME, AGGREGATES);
 
-        Functions.Builder aggregates = org.apache.cassandra.schema.Functions.builder();
-        for (UntypedResultSet.Row row : query(query, keyspaceName))
-            aggregates.add(createUDAFromRow(row, udfs, types));
-        return aggregates.build();
+        Collection<UDAggregate> aggregates = new ArrayList<>();
+        query(query, keyspaceName).forEach(row -> aggregates.add(createUDAFromRow(row, udfs, types)));
+        return aggregates;
     }
 
-    private static UDAggregate createUDAFromRow(UntypedResultSet.Row row, Functions functions, Types types)
+    private static UDAggregate createUDAFromRow(UntypedResultSet.Row row, Collection<UDFunction> functions, Types types)
     {
         String ksName = row.getString("keyspace_name");
         String functionName = row.getString("aggregate_name");
@@ -1255,18 +1238,12 @@ public final class SchemaKeyspace
         AbstractType<?> returnType = CQLTypeParser.parse(ksName, row.getString("return_type"), types);
 
         FunctionName stateFunc = new FunctionName(ksName, (row.getString("state_func")));
+
         FunctionName finalFunc = row.has("final_func") ? new FunctionName(ksName, row.getString("final_func")) : null;
         AbstractType<?> stateType = row.has("state_type") ? CQLTypeParser.parse(ksName, row.getString("state_type"), types) : null;
         ByteBuffer initcond = row.has("initcond") ? Terms.asBytes(ksName, row.getString("initcond"), stateType) : null;
 
-        try
-        {
-            return UDAggregate.create(functions, name, argTypes, returnType, stateFunc, finalFunc, stateType, initcond);
-        }
-        catch (InvalidRequestException reason)
-        {
-            return UDAggregate.createBroken(name, argTypes, returnType, initcond, reason);
-        }
+        return UDAggregate.create(functions, name, argTypes, returnType, stateFunc, finalFunc, stateType, initcond);
     }
 
     private static UntypedResultSet query(String query, Object... variables)
@@ -1310,52 +1287,6 @@ public final class SchemaKeyspace
         return keyspaces.build();
     }
 
-    /*
-     * Type parsing and transformation
-     */
-
-    /*
-     * Recursively replaces any instances of UserType with an equivalent TupleType.
-     * We do it for dropped_columns, to allow safely dropping unused user types without retaining any references
-     * in dropped_columns.
-     */
-    private static AbstractType<?> expandUserTypes(AbstractType<?> original)
-    {
-        if (original instanceof UserType)
-            return new TupleType(expandUserTypes(((UserType) original).fieldTypes()));
-
-        if (original instanceof TupleType)
-            return new TupleType(expandUserTypes(((TupleType) original).allTypes()));
-
-        if (original instanceof ListType<?>)
-            return ListType.getInstance(expandUserTypes(((ListType<?>) original).getElementsType()), original.isMultiCell());
-
-        if (original instanceof MapType<?,?>)
-        {
-            MapType<?, ?> mt = (MapType<?, ?>) original;
-            return MapType.getInstance(expandUserTypes(mt.getKeysType()), expandUserTypes(mt.getValuesType()), mt.isMultiCell());
-        }
-
-        if (original instanceof SetType<?>)
-            return SetType.getInstance(expandUserTypes(((SetType<?>) original).getElementsType()), original.isMultiCell());
-
-        // this is very unlikely to ever happen, but it's better to be safe than sorry
-        if (original instanceof ReversedType<?>)
-            return ReversedType.getInstance(expandUserTypes(((ReversedType) original).baseType));
-
-        if (original instanceof CompositeType)
-            return CompositeType.getInstance(expandUserTypes(original.getComponents()));
-
-        return original;
-    }
-
-    private static List<AbstractType<?>> expandUserTypes(List<AbstractType<?>> types)
-    {
-        return types.stream()
-                    .map(SchemaKeyspace::expandUserTypes)
-                    .collect(toList());
-    }
-
     @VisibleForTesting
     static class MissingColumns extends RuntimeException
     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/schema/SchemaTransformation.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/schema/SchemaTransformation.java b/src/java/org/apache/cassandra/schema/SchemaTransformation.java
new file mode 100644
index 0000000..c19ac7c
--- /dev/null
+++ b/src/java/org/apache/cassandra/schema/SchemaTransformation.java
@@ -0,0 +1,31 @@
+/*
+ * 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.schema;
+
+public interface SchemaTransformation
+{
+    /**
+     * Apply a statement transformation to a schema snapshot.
+     *
+     * Implementing methods should be side-effect free.
+     *
+     * @param schema Keyspaces to base the transformation on
+     * @return Keyspaces transformed by the statement
+     */
+    Keyspaces apply(Keyspaces schema);
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/schema/TableId.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/schema/TableId.java b/src/java/org/apache/cassandra/schema/TableId.java
index 95256fe..695147f 100644
--- a/src/java/org/apache/cassandra/schema/TableId.java
+++ b/src/java/org/apache/cassandra/schema/TableId.java
@@ -30,7 +30,7 @@ import org.apache.cassandra.utils.UUIDGen;
 /**
  * The unique identifier of a table.
  * <p>
- * This is essentially a UUID, but we wrap it as it's used quite a bit in the code and having a nicely name class make
+ * This is essentially a UUID, but we wrap it as it's used quite a bit in the code and having a nicely named class make
  * the code more readable.
  */
 public class TableId

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/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 47e5b47..6466e2e 100644
--- a/src/java/org/apache/cassandra/schema/TableMetadata.java
+++ b/src/java/org/apache/cassandra/schema/TableMetadata.java
@@ -44,6 +44,7 @@ import static java.lang.String.format;
 import static java.util.stream.Collectors.toList;
 import static java.util.stream.Collectors.toSet;
 
+import static com.google.common.collect.Iterables.any;
 import static com.google.common.collect.Iterables.transform;
 import static org.apache.cassandra.schema.IndexMetadata.isNameValid;
 
@@ -204,6 +205,21 @@ public final class TableMetadata
         return kind == Kind.INDEX;
     }
 
+    public TableMetadata withSwapped(TableParams params)
+    {
+        return unbuild().params(params).build();
+    }
+
+    public TableMetadata withSwapped(Triggers triggers)
+    {
+        return unbuild().triggers(triggers).build();
+    }
+
+    public TableMetadata withSwapped(Indexes indexes)
+    {
+        return unbuild().indexes(indexes).build();
+    }
+
     public boolean isView()
     {
         return kind == Kind.VIEW;
@@ -418,42 +434,62 @@ public final class TableMetadata
         indexes.validate(this);
     }
 
-    void validateCompatibility(TableMetadata other)
+    void validateCompatibility(TableMetadata previous)
     {
         if (isIndex())
             return;
 
-        if (!other.keyspace.equals(keyspace))
-            except("Keyspace mismatch (found %s; expected %s)", other.keyspace, keyspace);
+        if (!previous.keyspace.equals(keyspace))
+            except("Keyspace mismatch (found %s; expected %s)", keyspace, previous.keyspace);
 
-        if (!other.name.equals(name))
-            except("Table mismatch (found %s; expected %s)", other.name, name);
+        if (!previous.name.equals(name))
+            except("Table mismatch (found %s; expected %s)", name, previous.name);
 
-        if (!other.id.equals(id))
-            except("Table ID mismatch (found %s; expected %s)", other.id, id);
+        if (!previous.id.equals(id))
+            except("Table ID mismatch (found %s; expected %s)", id, previous.id);
 
-        if (!other.flags.equals(flags))
-            except("Table type mismatch (found %s; expected %s)", other.flags, flags);
+        if (!previous.flags.equals(flags))
+            except("Table type mismatch (found %s; expected %s)", flags, previous.flags);
 
-        if (other.partitionKeyColumns.size() != partitionKeyColumns.size())
-            except("Partition keys of different length (found %s; expected %s)", other.partitionKeyColumns.size(), partitionKeyColumns.size());
+        if (previous.partitionKeyColumns.size() != partitionKeyColumns.size())
+        {
+            except("Partition keys of different length (found %s; expected %s)",
+                   partitionKeyColumns.size(),
+                   previous.partitionKeyColumns.size());
+        }
 
         for (int i = 0; i < partitionKeyColumns.size(); i++)
-            if (!other.partitionKeyColumns.get(i).type.isCompatibleWith(partitionKeyColumns.get(i).type))
-                except("Partition key column mismatch (found %s; expected %s)", other.partitionKeyColumns.get(i).type, partitionKeyColumns.get(i).type);
+        {
+            if (!partitionKeyColumns.get(i).type.isCompatibleWith(previous.partitionKeyColumns.get(i).type))
+            {
+                except("Partition key column mismatch (found %s; expected %s)",
+                       partitionKeyColumns.get(i).type,
+                       previous.partitionKeyColumns.get(i).type);
+            }
+        }
 
-        if (other.clusteringColumns.size() != clusteringColumns.size())
-            except("Clustering columns of different length (found %s; expected %s)", other.clusteringColumns.size(), clusteringColumns.size());
+        if (previous.clusteringColumns.size() != clusteringColumns.size())
+        {
+            except("Clustering columns of different length (found %s; expected %s)",
+                   clusteringColumns.size(),
+                   previous.clusteringColumns.size());
+        }
 
         for (int i = 0; i < clusteringColumns.size(); i++)
-            if (!other.clusteringColumns.get(i).type.isCompatibleWith(clusteringColumns.get(i).type))
-                except("Clustering column mismatch (found %s; expected %s)", other.clusteringColumns.get(i).type, clusteringColumns.get(i).type);
+        {
+            if (!clusteringColumns.get(i).type.isCompatibleWith(previous.clusteringColumns.get(i).type))
+            {
+                except("Clustering column mismatch (found %s; expected %s)",
+                       clusteringColumns.get(i).type,
+                       previous.clusteringColumns.get(i).type);
+            }
+        }
 
-        for (ColumnMetadata otherColumn : other.regularAndStaticColumns)
+        for (ColumnMetadata previousColumn : previous.regularAndStaticColumns)
         {
-            ColumnMetadata column = getColumn(otherColumn.name);
-            if (column != null && !otherColumn.type.isCompatibleWith(column.type))
-                except("Column mismatch (found %s; expected %s", otherColumn, column);
+            ColumnMetadata column = getColumn(previousColumn.name);
+            if (column != null && !column.type.isCompatibleWith(previousColumn.type))
+                except("Column mismatch (found %s; expected %s)", column, previousColumn);
         }
     }
 
@@ -472,7 +508,7 @@ public final class TableMetadata
      * This method should only be called for superColumn tables and "static
      * compact" ones. For any other table, all column names are UTF8.
      */
-    public AbstractType<?> staticCompactOrSuperTableColumnNameType()
+    AbstractType<?> staticCompactOrSuperTableColumnNameType()
     {
         if (isSuper())
         {
@@ -543,6 +579,22 @@ public final class TableMetadata
         return unbuild().params(builder.build()).build();
     }
 
+    boolean referencesUserType(ByteBuffer name)
+    {
+        return any(columns(), c -> c.type.referencesUserType(name));
+    }
+
+    public TableMetadata withUpdatedUserType(UserType udt)
+    {
+        if (!referencesUserType(udt.name))
+            return this;
+
+        Builder builder = unbuild();
+        columns().forEach(c -> builder.alterColumnType(c.name, c.type.withUpdatedUserType(udt)));
+
+        return builder.build();
+    }
+
     private void except(String format, Object... args)
     {
         throw new ConfigurationException(keyspace + "." + name + ": " + format(format, args));
@@ -559,6 +611,11 @@ public final class TableMetadata
 
         TableMetadata tm = (TableMetadata) o;
 
+        return equalsWithoutColumns(tm) && columns.equals(tm.columns);
+    }
+
+    private boolean equalsWithoutColumns(TableMetadata tm)
+    {
         return keyspace.equals(tm.keyspace)
             && name.equals(tm.name)
             && id.equals(tm.id)
@@ -566,12 +623,46 @@ public final class TableMetadata
             && kind == tm.kind
             && params.equals(tm.params)
             && flags.equals(tm.flags)
-            && columns.equals(tm.columns)
             && droppedColumns.equals(tm.droppedColumns)
             && indexes.equals(tm.indexes)
             && triggers.equals(tm.triggers);
     }
 
+    Optional<Difference> compare(TableMetadata other)
+    {
+        return equalsWithoutColumns(other)
+             ? compareColumns(other.columns)
+             : Optional.of(Difference.SHALLOW);
+    }
+
+    private Optional<Difference> compareColumns(Map<ByteBuffer, ColumnMetadata> other)
+    {
+        if (!columns.keySet().equals(other.keySet()))
+            return Optional.of(Difference.SHALLOW);
+
+        boolean differsDeeply = false;
+
+        for (Map.Entry<ByteBuffer, ColumnMetadata> entry : columns.entrySet())
+        {
+            ColumnMetadata thisColumn = entry.getValue();
+            ColumnMetadata thatColumn = other.get(entry.getKey());
+
+            Optional<Difference> difference = thisColumn.compare(thatColumn);
+            if (difference.isPresent())
+            {
+                switch (difference.get())
+                {
+                    case SHALLOW:
+                        return difference;
+                    case DEEP:
+                        differsDeeply = true;
+                }
+            }
+        }
+
+        return differsDeeply ? Optional.of(Difference.DEEP) : Optional.empty();
+    }
+
     @Override
     public int hashCode()
     {
@@ -858,7 +949,7 @@ public final class TableMetadata
             return this;
         }
 
-        public Builder addColumns(Iterable<ColumnMetadata> columns)
+        Builder addColumns(Iterable<ColumnMetadata> columns)
         {
             columns.forEach(this::addColumn);
             return this;
@@ -884,7 +975,7 @@ public final class TableMetadata
 
         public Builder recordColumnDrop(ColumnMetadata column, long timeMicros)
         {
-            droppedColumns.put(column.name.bytes, new DroppedColumn(column, timeMicros));
+            droppedColumns.put(column.name.bytes, new DroppedColumn(column.withNewType(column.type.expandUserTypes()), timeMicros));
             return this;
         }
 
@@ -950,7 +1041,7 @@ public final class TableMetadata
             return this;
         }
 
-        public Builder alterColumnType(ColumnIdentifier name, AbstractType<?> type)
+        Builder alterColumnType(ColumnIdentifier name, AbstractType<?> type)
         {
             ColumnMetadata column = columns.get(name.bytes);
             if (column == null)
@@ -986,6 +1077,8 @@ public final class TableMetadata
      * Currently this is only used by views with normal base column as PK column
      * so updates to other columns do not make the row live when the base column
      * is not live. See CASSANDRA-11500.
+     *
+     * TODO: does not belong here, should be gone
      */
     public boolean enforceStrictLiveness()
     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/schema/TableMetadataRef.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/schema/TableMetadataRef.java b/src/java/org/apache/cassandra/schema/TableMetadataRef.java
index 5ff9d5b..3c45594 100644
--- a/src/java/org/apache/cassandra/schema/TableMetadataRef.java
+++ b/src/java/org/apache/cassandra/schema/TableMetadataRef.java
@@ -66,7 +66,7 @@ public final class TableMetadataRef
      */
     void set(TableMetadata metadata)
     {
-        get().validateCompatibility(metadata);
+        metadata.validateCompatibility(get());
         this.metadata = metadata;
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/schema/TableParams.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/schema/TableParams.java b/src/java/org/apache/cassandra/schema/TableParams.java
index 78dc894..f5b3c89 100644
--- a/src/java/org/apache/cassandra/schema/TableParams.java
+++ b/src/java/org/apache/cassandra/schema/TableParams.java
@@ -245,7 +245,7 @@ public final class TableParams
     {
         private String comment = "";
         private Double bloomFilterFpChance;
-        public Double crcCheckChance = 1.0;
+        private double crcCheckChance = 1.0;
         private int gcGraceSeconds = 864000; // 10 days
         private int defaultTimeToLive = 0;
         private int memtableFlushPeriodInMs = 0;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/schema/Tables.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/schema/Tables.java b/src/java/org/apache/cassandra/schema/Tables.java
index a83c061..0320440 100644
--- a/src/java/org/apache/cassandra/schema/Tables.java
+++ b/src/java/org/apache/cassandra/schema/Tables.java
@@ -17,32 +17,38 @@
  */
 package org.apache.cassandra.schema;
 
+import java.nio.ByteBuffer;
 import java.util.HashMap;
 import java.util.Iterator;
 import java.util.Map;
 import java.util.Optional;
+import java.util.function.Predicate;
 
 import javax.annotation.Nullable;
 
-import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.MapDifference;
-import com.google.common.collect.Maps;
+import com.google.common.collect.*;
 
+import org.apache.cassandra.db.marshal.UserType;
 import org.apache.cassandra.index.internal.CassandraIndex;
 
-import static com.google.common.collect.Iterables.filter;
+import static com.google.common.collect.Iterables.any;
+import static com.google.common.collect.Iterables.transform;
 
 /**
  * An immutable container for a keyspace's Tables.
  */
 public final class Tables implements Iterable<TableMetadata>
 {
+    private static final Tables NONE = builder().build();
+
     private final ImmutableMap<String, TableMetadata> tables;
+    private final ImmutableMap<TableId, TableMetadata> tablesById;
     private final ImmutableMap<String, TableMetadata> indexTables;
 
     private Tables(Builder builder)
     {
         tables = builder.tables.build();
+        tablesById = builder.tablesById.build();
         indexTables = builder.indexTables.build();
     }
 
@@ -53,7 +59,7 @@ public final class Tables implements Iterable<TableMetadata>
 
     public static Tables none()
     {
-        return builder().build();
+        return NONE;
     }
 
     public static Tables of(TableMetadata... tables)
@@ -71,6 +77,11 @@ public final class Tables implements Iterable<TableMetadata>
         return tables.values().iterator();
     }
 
+    public Iterable<TableMetadata> referencingUserType(ByteBuffer name)
+    {
+        return Iterables.filter(tables.values(), t -> t.referencesUserType(name));
+    }
+
     ImmutableMap<String, TableMetadata> indexTables()
     {
         return indexTables;
@@ -105,9 +116,21 @@ public final class Tables implements Iterable<TableMetadata>
     }
 
     @Nullable
-    public TableMetadata getIndexTableNullable(String name)
+    TableMetadata getNullable(TableId id)
     {
-        return indexTables.get(name);
+        return tablesById.get(id);
+    }
+
+    boolean containsTable(TableId id)
+    {
+        return tablesById.containsKey(id);
+    }
+
+    public Tables filter(Predicate<TableMetadata> predicate)
+    {
+        Builder builder = builder();
+        tables.values().stream().filter(predicate).forEach(builder::add);
+        return builder.build();
     }
 
     /**
@@ -134,18 +157,19 @@ public final class Tables implements Iterable<TableMetadata>
         TableMetadata table =
             get(name).orElseThrow(() -> new IllegalStateException(String.format("Table %s doesn't exists", name)));
 
-        return builder().add(filter(this, t -> t != table)).build();
+        return without(table);
     }
 
-    MapDifference<TableId, TableMetadata> diff(Tables other)
+    public Tables without(TableMetadata table)
     {
-        Map<TableId, TableMetadata> thisTables = new HashMap<>();
-        this.forEach(t -> thisTables.put(t.id, t));
-
-        Map<TableId, TableMetadata> otherTables = new HashMap<>();
-        other.forEach(t -> otherTables.put(t.id, t));
+        return filter(t -> t != table);
+    }
 
-        return Maps.difference(thisTables, otherTables);
+    public Tables withUpdatedUserType(UserType udt)
+    {
+        return any(this, t -> t.referencesUserType(udt.name))
+             ? builder().add(transform(this, t -> t.withUpdatedUserType(udt))).build()
+             : this;
     }
 
     MapDifference<String, TableMetadata> indexesDiff(Tables other)
@@ -180,6 +204,7 @@ public final class Tables implements Iterable<TableMetadata>
     public static final class Builder
     {
         final ImmutableMap.Builder<String, TableMetadata> tables = new ImmutableMap.Builder<>();
+        final ImmutableMap.Builder<TableId, TableMetadata> tablesById = new ImmutableMap.Builder<>();
         final ImmutableMap.Builder<String, TableMetadata> indexTables = new ImmutableMap.Builder<>();
 
         private Builder()
@@ -195,6 +220,8 @@ public final class Tables implements Iterable<TableMetadata>
         {
             tables.put(table.name, table);
 
+            tablesById.put(table.id, table);
+
             table.indexes
                  .stream()
                  .filter(i -> !i.isCustom())
@@ -217,4 +244,38 @@ public final class Tables implements Iterable<TableMetadata>
             return this;
         }
     }
+
+    static TablesDiff diff(Tables before, Tables after)
+    {
+        return TablesDiff.diff(before, after);
+    }
+
+    public static final class TablesDiff extends Diff<Tables, TableMetadata>
+    {
+        private final static TablesDiff NONE = new TablesDiff(Tables.none(), Tables.none(), ImmutableList.of());
+
+        private TablesDiff(Tables created, Tables dropped, ImmutableCollection<Altered<TableMetadata>> altered)
+        {
+            super(created, dropped, altered);
+        }
+
+        private static TablesDiff diff(Tables before, Tables after)
+        {
+            if (before == after)
+                return NONE;
+
+            Tables created = after.filter(t -> !before.containsTable(t.id));
+            Tables dropped = before.filter(t -> !after.containsTable(t.id));
+
+            ImmutableList.Builder<Altered<TableMetadata>> altered = ImmutableList.builder();
+            before.forEach(tableBefore ->
+            {
+                TableMetadata tableAfter = after.getNullable(tableBefore.id);
+                if (null != tableAfter)
+                    tableBefore.compare(tableAfter).ifPresent(kind -> altered.add(new Altered<>(tableBefore, tableAfter, kind)));
+            });
+
+            return new TablesDiff(created, dropped, altered.build());
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/schema/Types.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/schema/Types.java b/src/java/org/apache/cassandra/schema/Types.java
index 0bdf7cf..64aeead 100644
--- a/src/java/org/apache/cassandra/schema/Types.java
+++ b/src/java/org/apache/cassandra/schema/Types.java
@@ -19,6 +19,7 @@ package org.apache.cassandra.schema;
 
 import java.nio.ByteBuffer;
 import java.util.*;
+import java.util.function.Predicate;
 
 import javax.annotation.Nullable;
 
@@ -31,8 +32,11 @@ import org.apache.cassandra.db.marshal.UserType;
 import org.apache.cassandra.exceptions.ConfigurationException;
 
 import static java.lang.String.format;
-import static com.google.common.collect.Iterables.filter;
 import static java.util.stream.Collectors.toList;
+
+import static com.google.common.collect.Iterables.any;
+import static com.google.common.collect.Iterables.transform;
+
 import static org.apache.cassandra.utils.ByteBufferUtil.bytes;
 
 /**
@@ -82,6 +86,11 @@ public final class Types implements Iterable<UserType>
         return types.values().iterator();
     }
 
+    public Iterable<UserType> referencingUserType(ByteBuffer name)
+    {
+        return Iterables.filter(types.values(), t -> t.referencesUserType(name) && !t.name.equals(name));
+    }
+
     /**
      * Get the type with the specified name
      *
@@ -105,6 +114,18 @@ public final class Types implements Iterable<UserType>
         return types.get(name);
     }
 
+    boolean containsType(ByteBuffer name)
+    {
+        return types.containsKey(name);
+    }
+
+    Types filter(Predicate<UserType> predicate)
+    {
+        Builder builder = builder();
+        types.values().stream().filter(predicate).forEach(builder::add);
+        return builder.build();
+    }
+
     /**
      * Create a Types instance with the provided type added
      */
@@ -124,12 +145,19 @@ public final class Types implements Iterable<UserType>
         UserType type =
             get(name).orElseThrow(() -> new IllegalStateException(format("Type %s doesn't exists", name)));
 
-        return builder().add(filter(this, t -> t != type)).build();
+        return without(type);
     }
 
-    MapDifference<ByteBuffer, UserType> diff(Types other)
+    public Types without(UserType type)
     {
-        return Maps.difference(types, other.types);
+        return filter(t -> t != type);
+    }
+
+    public Types withUpdatedUserType(UserType udt)
+    {
+        return any(this, t -> t.referencesUserType(udt.name))
+             ? builder().add(transform(this, t -> t.withUpdatedUserType(udt))).build()
+             : this;
     }
 
     @Override
@@ -155,7 +183,7 @@ public final class Types implements Iterable<UserType>
             if (!thisNext.getKey().equals(otherNext.getKey()))
                 return false;
 
-            if (!thisNext.getValue().equals(otherNext.getValue(), true))  // ignore freezing
+            if (!thisNext.getValue().equals(otherNext.getValue()))
                 return false;
         }
         return true;
@@ -305,7 +333,7 @@ public final class Types implements Iterable<UserType>
             {
                 List<FieldIdentifier> preparedFieldNames =
                     fieldNames.stream()
-                              .map(t -> FieldIdentifier.forInternalString(t))
+                              .map(FieldIdentifier::forInternalString)
                               .collect(toList());
 
                 List<AbstractType<?>> preparedFieldTypes =
@@ -329,4 +357,38 @@ public final class Types implements Iterable<UserType>
             }
         }
     }
+
+    static TypesDiff diff(Types before, Types after)
+    {
+        return TypesDiff.diff(before, after);
+    }
+
+    static final class TypesDiff extends Diff<Types, UserType>
+    {
+        private static final TypesDiff NONE = new TypesDiff(Types.none(), Types.none(), ImmutableList.of());
+
+        private TypesDiff(Types created, Types dropped, ImmutableCollection<Altered<UserType>> altered)
+        {
+            super(created, dropped, altered);
+        }
+
+        private static TypesDiff diff(Types before, Types after)
+        {
+            if (before == after)
+                return NONE;
+
+            Types created = after.filter(t -> !before.containsType(t.name));
+            Types dropped = before.filter(t -> !after.containsType(t.name));
+
+            ImmutableList.Builder<Altered<UserType>> altered = ImmutableList.builder();
+            before.forEach(typeBefore ->
+            {
+                UserType typeAfter = after.getNullable(typeBefore.name);
+                if (null != typeAfter)
+                    typeBefore.compare(typeAfter).ifPresent(kind -> altered.add(new Altered<>(typeBefore, typeAfter, kind)));
+            });
+
+            return new TypesDiff(created, dropped, altered.build());
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/schema/ViewMetadata.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/schema/ViewMetadata.java b/src/java/org/apache/cassandra/schema/ViewMetadata.java
index 57f4092..66360bf 100644
--- a/src/java/org/apache/cassandra/schema/ViewMetadata.java
+++ b/src/java/org/apache/cassandra/schema/ViewMetadata.java
@@ -17,56 +17,52 @@
  */
 package org.apache.cassandra.schema;
 
-import java.util.List;
-import java.util.Objects;
-import java.util.stream.Collectors;
-
-import org.antlr.runtime.*;
-import org.apache.cassandra.cql3.*;
-import org.apache.cassandra.cql3.statements.SelectStatement;
-import org.apache.cassandra.db.marshal.AbstractType;
-import org.apache.cassandra.db.view.View;
-import org.apache.cassandra.exceptions.SyntaxException;
+import java.nio.ByteBuffer;
+import java.util.Optional;
 
 import org.apache.commons.lang3.builder.HashCodeBuilder;
 import org.apache.commons.lang3.builder.ToStringBuilder;
 
+import org.apache.cassandra.cql3.*;
+import org.apache.cassandra.db.marshal.UserType;
+
 public final class ViewMetadata
 {
-    public final String keyspace;
-    public final String name;
     public final TableId baseTableId;
     public final String baseTableName;
+
     public final boolean includeAllColumns;
     public final TableMetadata metadata;
 
-    public final SelectStatement.RawStatement select;
-    public final String whereClause;
+    public final WhereClause whereClause;
 
     /**
-     * @param name              Name of the view
      * @param baseTableId       Internal ID of the table which this view is based off of
      * @param includeAllColumns Whether to include all columns or not
      */
-    public ViewMetadata(String keyspace,
-                        String name,
-                        TableId baseTableId,
+    public ViewMetadata(TableId baseTableId,
                         String baseTableName,
                         boolean includeAllColumns,
-                        SelectStatement.RawStatement select,
-                        String whereClause,
+                        WhereClause whereClause,
                         TableMetadata metadata)
     {
-        this.keyspace = keyspace;
-        this.name = name;
         this.baseTableId = baseTableId;
         this.baseTableName = baseTableName;
         this.includeAllColumns = includeAllColumns;
-        this.select = select;
         this.whereClause = whereClause;
         this.metadata = metadata;
     }
 
+    public String keyspace()
+    {
+        return metadata.keyspace;
+    }
+
+    public String name()
+    {
+        return metadata.name;
+    }
+
     /**
      * @return true if the view specified by this definition will include the column, false otherwise
      */
@@ -77,7 +73,7 @@ public final class ViewMetadata
 
     public ViewMetadata copy(TableMetadata newMetadata)
     {
-        return new ViewMetadata(keyspace, name, baseTableId, baseTableName, includeAllColumns, select, whereClause, newMetadata);
+        return new ViewMetadata(baseTableId, baseTableName, includeAllColumns, whereClause, newMetadata);
     }
 
     public TableMetadata baseTableMetadata()
@@ -95,20 +91,24 @@ public final class ViewMetadata
             return false;
 
         ViewMetadata other = (ViewMetadata) o;
-        return Objects.equals(keyspace, other.keyspace)
-               && Objects.equals(name, other.name)
-               && Objects.equals(baseTableId, other.baseTableId)
-               && Objects.equals(includeAllColumns, other.includeAllColumns)
-               && Objects.equals(whereClause, other.whereClause)
-               && Objects.equals(metadata, other.metadata);
+        return baseTableId.equals(other.baseTableId)
+            && includeAllColumns == other.includeAllColumns
+            && whereClause.equals(other.whereClause)
+            && metadata.equals(other.metadata);
+    }
+
+    Optional<Difference> compare(ViewMetadata other)
+    {
+        if (!baseTableId.equals(other.baseTableId) || includeAllColumns != other.includeAllColumns || !whereClause.equals(other.whereClause))
+            return Optional.of(Difference.SHALLOW);
+
+        return metadata.compare(other.metadata);
     }
 
     @Override
     public int hashCode()
     {
         return new HashCodeBuilder(29, 1597)
-               .append(keyspace)
-               .append(name)
                .append(baseTableId)
                .append(includeAllColumns)
                .append(whereClause)
@@ -120,8 +120,6 @@ public final class ViewMetadata
     public String toString()
     {
         return new ToStringBuilder(this)
-               .append("keyspace", keyspace)
-               .append("name", name)
                .append("baseTableId", baseTableId)
                .append("baseTableName", baseTableName)
                .append("includeAllColumns", includeAllColumns)
@@ -130,68 +128,37 @@ public final class ViewMetadata
                .toString();
     }
 
-    /**
-     * Replace the column 'from' with 'to' in this materialized view definition's partition,
-     * clustering, or included columns.
-     * @param from the existing column
-     * @param to the new column
-     */
-    public ViewMetadata renamePrimaryKeyColumn(ColumnIdentifier from, ColumnIdentifier to)
+    public boolean referencesUserType(ByteBuffer name)
+    {
+        return metadata.referencesUserType(name);
+    }
+
+    public ViewMetadata withUpdatedUserType(UserType udt)
+    {
+        return referencesUserType(udt.name)
+             ? copy(metadata.withUpdatedUserType(udt))
+             : this;
+    }
+
+    public ViewMetadata withRenamedPrimaryKeyColumn(ColumnIdentifier from, ColumnIdentifier to)
     {
         // convert whereClause to Relations, rename ids in Relations, then convert back to whereClause
-        List<Relation> relations = whereClauseToRelations(whereClause);
-        ColumnMetadata.Raw fromRaw = ColumnMetadata.Raw.forQuoted(from.toString());
-        ColumnMetadata.Raw toRaw = ColumnMetadata.Raw.forQuoted(to.toString());
-        List<Relation> newRelations =
-            relations.stream()
-                     .map(r -> r.renameIdentifier(fromRaw, toRaw))
-                     .collect(Collectors.toList());
-
-        String rawSelect = View.buildSelectStatement(baseTableName, metadata.columns(), whereClause);
-
-        return new ViewMetadata(keyspace,
-                                name,
-                                baseTableId,
+        ColumnMetadata.Raw rawFrom = ColumnMetadata.Raw.forQuoted(from.toString());
+        ColumnMetadata.Raw rawTo = ColumnMetadata.Raw.forQuoted(to.toString());
+
+        return new ViewMetadata(baseTableId,
                                 baseTableName,
                                 includeAllColumns,
-                                (SelectStatement.RawStatement) QueryProcessor.parseStatement(rawSelect),
-                                View.relationsToWhereClause(newRelations),
+                                whereClause.renameIdentifier(rawFrom, rawTo),
                                 metadata.unbuild().renamePrimaryKeyColumn(from, to).build());
     }
 
     public ViewMetadata withAddedRegularColumn(ColumnMetadata column)
     {
-        return new ViewMetadata(keyspace,
-                                name,
-                                baseTableId,
+        return new ViewMetadata(baseTableId,
                                 baseTableName,
                                 includeAllColumns,
-                                select,
                                 whereClause,
                                 metadata.unbuild().addColumn(column).build());
     }
-
-    public ViewMetadata withAlteredColumnType(ColumnIdentifier name, AbstractType<?> type)
-    {
-        return new ViewMetadata(keyspace,
-                                this.name,
-                                baseTableId,
-                                baseTableName,
-                                includeAllColumns,
-                                select,
-                                whereClause,
-                                metadata.unbuild().alterColumnType(name, type).build());
-    }
-
-    private static List<Relation> whereClauseToRelations(String whereClause)
-    {
-        try
-        {
-            return CQLFragmentParser.parseAnyUnhandled(CqlParser::whereClause, whereClause).build().relations;
-        }
-        catch (RecognitionException | SyntaxException exc)
-        {
-            throw new RuntimeException("Unexpected error parsing materialized view's where clause while handling column rename: ", exc);
-        }
-    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/schema/Views.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/schema/Views.java b/src/java/org/apache/cassandra/schema/Views.java
index 5765433..07cd8f2 100644
--- a/src/java/org/apache/cassandra/schema/Views.java
+++ b/src/java/org/apache/cassandra/schema/Views.java
@@ -21,24 +21,26 @@ import java.util.HashMap;
 import java.util.Iterator;
 import java.util.Map;
 import java.util.Optional;
-import java.util.UUID;
+import java.util.function.Predicate;
 
 import javax.annotation.Nullable;
 
-import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.Iterables;
-import com.google.common.collect.MapDifference;
-import com.google.common.collect.Maps;
+import com.google.common.collect.*;
 
-import static com.google.common.collect.Iterables.filter;
+import org.apache.cassandra.db.marshal.UserType;
+
+import static com.google.common.collect.Iterables.any;
+import static com.google.common.collect.Iterables.transform;
 
 public final class Views implements Iterable<ViewMetadata>
 {
+    private static final Views NONE = builder().build();
+
     private final ImmutableMap<String, ViewMetadata> views;
 
     private Views(Builder builder)
     {
-        views = builder.views.build();
+        views = ImmutableMap.copyOf(builder.views);
     }
 
     public static Builder builder()
@@ -46,9 +48,14 @@ public final class Views implements Iterable<ViewMetadata>
         return new Builder();
     }
 
+    public Builder unbuild()
+    {
+        return builder().put(this);
+    }
+
     public static Views none()
     {
-        return builder().build();
+        return NONE;
     }
 
     public Iterator<ViewMetadata> iterator()
@@ -56,7 +63,7 @@ public final class Views implements Iterable<ViewMetadata>
         return views.values().iterator();
     }
 
-    public Iterable<TableMetadata> metadatas()
+    Iterable<TableMetadata> allTableMetadata()
     {
         return Iterables.transform(views.values(), view -> view.metadata);
     }
@@ -71,9 +78,9 @@ public final class Views implements Iterable<ViewMetadata>
         return views.isEmpty();
     }
 
-    public Iterable<ViewMetadata> forTable(UUID tableId)
+    public Iterable<ViewMetadata> forTable(TableId tableId)
     {
-        return Iterables.filter(this, v -> v.baseTableId.asUUID().equals(tableId));
+        return Iterables.filter(this, v -> v.baseTableId.equals(tableId));
     }
 
     /**
@@ -99,20 +106,32 @@ public final class Views implements Iterable<ViewMetadata>
         return views.get(name);
     }
 
+    boolean containsView(String name)
+    {
+        return views.containsKey(name);
+    }
+
+    Views filter(Predicate<ViewMetadata> predicate)
+    {
+        Builder builder = builder();
+        views.values().stream().filter(predicate).forEach(builder::put);
+        return builder.build();
+    }
+
     /**
      * Create a MaterializedViews instance with the provided materialized view added
      */
     public Views with(ViewMetadata view)
     {
-        if (get(view.name).isPresent())
-            throw new IllegalStateException(String.format("Materialized View %s already exists", view.name));
+        if (get(view.name()).isPresent())
+            throw new IllegalStateException(String.format("Materialized View %s already exists", view.name()));
 
-        return builder().add(this).add(view).build();
+        return builder().put(this).put(view).build();
     }
 
     public Views withSwapped(ViewMetadata view)
     {
-        return without(view.name).with(view);
+        return without(view.name()).with(view);
     }
 
     /**
@@ -123,18 +142,14 @@ public final class Views implements Iterable<ViewMetadata>
         ViewMetadata materializedView =
             get(name).orElseThrow(() -> new IllegalStateException(String.format("Materialized View %s doesn't exists", name)));
 
-        return builder().add(filter(this, v -> v != materializedView)).build();
+        return filter(v -> v != materializedView);
     }
 
-    MapDifference<TableId, ViewMetadata> diff(Views other)
+    Views withUpdatedUserTypes(UserType udt)
     {
-        Map<TableId, ViewMetadata> thisViews = new HashMap<>();
-        this.forEach(v -> thisViews.put(v.metadata.id, v));
-
-        Map<TableId, ViewMetadata> otherViews = new HashMap<>();
-        other.forEach(v -> otherViews.put(v.metadata.id, v));
-
-        return Maps.difference(thisViews, otherViews);
+        return any(this, v -> v.referencesUserType(udt.name))
+             ? builder().put(transform(this, v -> v.withUpdatedUserType(udt))).build()
+             : this;
     }
 
     @Override
@@ -157,7 +172,7 @@ public final class Views implements Iterable<ViewMetadata>
 
     public static final class Builder
     {
-        final ImmutableMap.Builder<String, ViewMetadata> views = new ImmutableMap.Builder<>();
+        final Map<String, ViewMetadata> views = new HashMap<>();
 
         private Builder()
         {
@@ -168,17 +183,61 @@ public final class Views implements Iterable<ViewMetadata>
             return new Views(this);
         }
 
+        public ViewMetadata get(String name)
+        {
+            return views.get(name);
+        }
+
+        public Builder put(ViewMetadata view)
+        {
+            views.put(view.name(), view);
+            return this;
+        }
 
-        public Builder add(ViewMetadata view)
+        public Builder remove(String name)
         {
-            views.put(view.name, view);
+            views.remove(name);
             return this;
         }
 
-        public Builder add(Iterable<ViewMetadata> views)
+        public Builder put(Iterable<ViewMetadata> views)
         {
-            views.forEach(this::add);
+            views.forEach(this::put);
             return this;
         }
     }
+
+    static ViewsDiff diff(Views before, Views after)
+    {
+        return ViewsDiff.diff(before, after);
+    }
+
+    static final class ViewsDiff extends Diff<Views, ViewMetadata>
+    {
+        private static final ViewsDiff NONE = new ViewsDiff(Views.none(), Views.none(), ImmutableList.of());
+
+        private ViewsDiff(Views created, Views dropped, ImmutableCollection<Altered<ViewMetadata>> altered)
+        {
+            super(created, dropped, altered);
+        }
+
+        private static ViewsDiff diff(Views before, Views after)
+        {
+            if (before == after)
+                return NONE;
+
+            Views created = after.filter(v -> !before.containsView(v.name()));
+            Views dropped = before.filter(v -> !after.containsView(v.name()));
+
+            ImmutableList.Builder<Altered<ViewMetadata>> altered = ImmutableList.builder();
+            before.forEach(viewBefore ->
+            {
+                ViewMetadata viewAfter = after.getNullable(viewBefore.name());
+                if (null != viewAfter)
+                    viewBefore.compare(viewAfter).ifPresent(kind -> altered.add(new Altered<>(viewBefore, viewAfter, kind)));
+            });
+
+            return new ViewsDiff(created, dropped, altered.build());
+        }
+    }
 }
\ No newline at end of file


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


[08/13] cassandra git commit: Make all DDL statements idempotent and not dependent on global state

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/cql3/statements/TableAttributes.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/TableAttributes.java b/src/java/org/apache/cassandra/cql3/statements/TableAttributes.java
deleted file mode 100644
index 572362b..0000000
--- a/src/java/org/apache/cassandra/cql3/statements/TableAttributes.java
+++ /dev/null
@@ -1,195 +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.statements;
-
-import java.util.Map;
-import java.util.Set;
-
-import com.google.common.collect.ImmutableSet;
-
-import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.exceptions.SyntaxException;
-import org.apache.cassandra.schema.*;
-import org.apache.cassandra.schema.TableParams.Option;
-import org.apache.cassandra.service.reads.SpeculativeRetryPolicy;
-
-import static java.lang.String.format;
-
-public final class TableAttributes extends PropertyDefinitions
-{
-    private static final String KW_ID = "id";
-    private static final Set<String> validKeywords;
-    private static final Set<String> obsoleteKeywords;
-
-    static
-    {
-        ImmutableSet.Builder<String> validBuilder = ImmutableSet.builder();
-        for (Option option : Option.values())
-            validBuilder.add(option.toString());
-        validBuilder.add(KW_ID);
-        validKeywords = validBuilder.build();
-        obsoleteKeywords = ImmutableSet.of();
-    }
-
-    public void validate()
-    {
-        validate(validKeywords, obsoleteKeywords);
-        build(TableParams.builder()).validate();
-    }
-
-    public TableParams asNewTableParams()
-    {
-        return build(TableParams.builder());
-    }
-
-    public TableParams asAlteredTableParams(TableParams previous)
-    {
-        if (getId() != null)
-            throw new ConfigurationException("Cannot alter table id.");
-        return build(previous.unbuild());
-    }
-
-    public TableId getId() throws ConfigurationException
-    {
-        String id = getSimple(KW_ID);
-        try
-        {
-            return id != null ? TableId.fromString(id) : null;
-        }
-        catch (IllegalArgumentException e)
-        {
-            throw new ConfigurationException("Invalid table id", e);
-        }
-    }
-
-    private TableParams build(TableParams.Builder builder)
-    {
-        if (hasOption(Option.BLOOM_FILTER_FP_CHANCE))
-            builder.bloomFilterFpChance(getDouble(Option.BLOOM_FILTER_FP_CHANCE));
-
-        if (hasOption(Option.CACHING))
-            builder.caching(CachingParams.fromMap(getMap(Option.CACHING)));
-
-        if (hasOption(Option.COMMENT))
-            builder.comment(getString(Option.COMMENT));
-
-        if (hasOption(Option.COMPACTION))
-            builder.compaction(CompactionParams.fromMap(getMap(Option.COMPACTION)));
-
-        if (hasOption(Option.COMPRESSION))
-        {
-            //crc_check_chance was "promoted" from a compression property to a top-level-property after #9839
-            //so we temporarily accept it to be defined as a compression option, to maintain backwards compatibility
-            Map<String, String> compressionOpts = getMap(Option.COMPRESSION);
-            if (compressionOpts.containsKey(Option.CRC_CHECK_CHANCE.toString().toLowerCase()))
-            {
-                Double crcCheckChance = getDeprecatedCrcCheckChance(compressionOpts);
-                builder.crcCheckChance(crcCheckChance);
-            }
-            builder.compression(CompressionParams.fromMap(getMap(Option.COMPRESSION)));
-        }
-
-        if (hasOption(Option.DEFAULT_TIME_TO_LIVE))
-            builder.defaultTimeToLive(getInt(Option.DEFAULT_TIME_TO_LIVE));
-
-        if (hasOption(Option.GC_GRACE_SECONDS))
-            builder.gcGraceSeconds(getInt(Option.GC_GRACE_SECONDS));
-
-        if (hasOption(Option.MAX_INDEX_INTERVAL))
-            builder.maxIndexInterval(getInt(Option.MAX_INDEX_INTERVAL));
-
-        if (hasOption(Option.MEMTABLE_FLUSH_PERIOD_IN_MS))
-            builder.memtableFlushPeriodInMs(getInt(Option.MEMTABLE_FLUSH_PERIOD_IN_MS));
-
-        if (hasOption(Option.MIN_INDEX_INTERVAL))
-            builder.minIndexInterval(getInt(Option.MIN_INDEX_INTERVAL));
-
-        if (hasOption(Option.SPECULATIVE_RETRY))
-            builder.speculativeRetry(SpeculativeRetryPolicy.fromString(getString(Option.SPECULATIVE_RETRY)));
-
-        if (hasOption(Option.CRC_CHECK_CHANCE))
-            builder.crcCheckChance(getDouble(Option.CRC_CHECK_CHANCE));
-
-        if (hasOption(Option.CDC))
-            builder.cdc(getBoolean(Option.CDC.toString(), false));
-
-        return builder.build();
-    }
-
-    private Double getDeprecatedCrcCheckChance(Map<String, String> compressionOpts)
-    {
-        String value = compressionOpts.get(Option.CRC_CHECK_CHANCE.toString().toLowerCase());
-        try
-        {
-            return Double.valueOf(value);
-        }
-        catch (NumberFormatException e)
-        {
-            throw new SyntaxException(String.format("Invalid double value %s for crc_check_chance.'", value));
-        }
-    }
-
-    private double getDouble(Option option)
-    {
-        String value = getString(option);
-
-        try
-        {
-            return Double.parseDouble(value);
-        }
-        catch (NumberFormatException e)
-        {
-            throw new SyntaxException(format("Invalid double value %s for '%s'", value, option));
-        }
-    }
-
-    private int getInt(Option option)
-    {
-        String value = getString(option);
-
-        try
-        {
-            return Integer.parseInt(value);
-        }
-        catch (NumberFormatException e)
-        {
-            throw new SyntaxException(String.format("Invalid integer value %s for '%s'", value, option));
-        }
-    }
-
-    private String getString(Option option)
-    {
-        String value = getSimple(option.toString());
-        if (value == null)
-            throw new IllegalStateException(format("Option '%s' is absent", option));
-        return value;
-    }
-
-    private Map<String, String> getMap(Option option)
-    {
-        Map<String, String> value = getMap(option.toString());
-        if (value == null)
-            throw new IllegalStateException(format("Option '%s' is absent", option));
-        return value;
-    }
-
-    private boolean hasOption(Option option)
-    {
-        return hasProperty(option.toString());
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/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 d41a814..206d116 100644
--- a/src/java/org/apache/cassandra/cql3/statements/TruncateStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/TruncateStatement.java
@@ -19,6 +19,7 @@ package org.apache.cassandra.cql3.statements;
 
 import java.util.concurrent.TimeoutException;
 
+import org.apache.cassandra.audit.AuditLogContext;
 import org.apache.cassandra.audit.AuditLogEntryType;
 import org.apache.cassandra.auth.Permission;
 import org.apache.cassandra.cql3.*;
@@ -34,45 +35,40 @@ import org.apache.cassandra.transport.messages.ResultMessage;
 import org.apache.commons.lang3.builder.ToStringBuilder;
 import org.apache.commons.lang3.builder.ToStringStyle;
 
-public class TruncateStatement extends CFStatement implements CQLStatement
+public class TruncateStatement extends QualifiedStatement implements CQLStatement
 {
-    public TruncateStatement(CFName name)
+    public TruncateStatement(QualifiedName name)
     {
         super(name);
     }
 
-    public int getBoundTerms()
+    public TruncateStatement prepare(ClientState state)
     {
-        return 0;
+        return this;
     }
 
-    public Prepared prepare() throws InvalidRequestException
+    public void authorize(ClientState state) throws InvalidRequestException, UnauthorizedException
     {
-        return new Prepared(this);
-    }
-
-    public void checkAccess(ClientState state) throws InvalidRequestException, UnauthorizedException
-    {
-        state.hasColumnFamilyAccess(keyspace(), columnFamily(), Permission.MODIFY);
+        state.ensureTablePermission(keyspace(), name(), Permission.MODIFY);
     }
 
     public void validate(ClientState state) throws InvalidRequestException
     {
-        Schema.instance.validateTable(keyspace(), columnFamily());
+        Schema.instance.validateTable(keyspace(), name());
     }
 
     public ResultMessage execute(QueryState state, QueryOptions options, long queryStartNanoTime) throws InvalidRequestException, TruncateException
     {
         try
         {
-            TableMetadata metaData = Schema.instance.getTableMetadata(keyspace(), columnFamily());
+            TableMetadata metaData = Schema.instance.getTableMetadata(keyspace(), name());
             if (metaData.isView())
                 throw new InvalidRequestException("Cannot TRUNCATE materialized view directly; must truncate base table instead");
 
             if (metaData.isVirtual())
                 throw new InvalidRequestException("Cannot truncate virtual tables");
 
-            StorageProxy.truncateBlocking(keyspace(), columnFamily());
+            StorageProxy.truncateBlocking(keyspace(), name());
         }
         catch (UnavailableException | TimeoutException e)
         {
@@ -81,18 +77,18 @@ public class TruncateStatement extends CFStatement implements CQLStatement
         return null;
     }
 
-    public ResultMessage executeInternal(QueryState state, QueryOptions options)
+    public ResultMessage executeLocally(QueryState state, QueryOptions options)
     {
         try
         {
-            TableMetadata metaData = Schema.instance.getTableMetadata(keyspace(), columnFamily());
+            TableMetadata metaData = Schema.instance.getTableMetadata(keyspace(), name());
             if (metaData.isView())
                 throw new InvalidRequestException("Cannot TRUNCATE materialized view directly; must truncate base table instead");
 
             if (metaData.isVirtual())
                 throw new InvalidRequestException("Cannot truncate virtual tables");
 
-            ColumnFamilyStore cfs = Keyspace.open(keyspace()).getColumnFamilyStore(columnFamily());
+            ColumnFamilyStore cfs = Keyspace.open(keyspace()).getColumnFamilyStore(name());
             cfs.truncateBlocking();
         }
         catch (Exception e)
@@ -111,6 +107,6 @@ public class TruncateStatement extends CFStatement implements CQLStatement
     @Override
     public AuditLogContext getAuditLogContext()
     {
-        return new AuditLogContext(AuditLogEntryType.TRUNCATE, keyspace(), cfName.getColumnFamily());
+        return new AuditLogContext(AuditLogEntryType.TRUNCATE, keyspace(), name());
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/cql3/statements/UpdateStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/UpdateStatement.java b/src/java/org/apache/cassandra/cql3/statements/UpdateStatement.java
index 66addab..21323d2 100644
--- a/src/java/org/apache/cassandra/cql3/statements/UpdateStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/UpdateStatement.java
@@ -21,6 +21,7 @@ import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
 
+import org.apache.cassandra.audit.AuditLogContext;
 import org.apache.cassandra.audit.AuditLogEntryType;
 import org.apache.cassandra.cql3.*;
 import org.apache.cassandra.cql3.conditions.ColumnCondition;
@@ -50,19 +51,14 @@ public class UpdateStatement extends ModificationStatement
     private static final Constants.Value EMPTY = new Constants.Value(ByteBufferUtil.EMPTY_BYTE_BUFFER);
 
     private UpdateStatement(StatementType type,
-                            int boundTerms,
+                            VariableSpecifications bindVariables,
                             TableMetadata metadata,
                             Operations operations,
                             StatementRestrictions restrictions,
                             Conditions conditions,
                             Attributes attrs)
     {
-        super(type, boundTerms, metadata, operations, restrictions, conditions, attrs);
-    }
-
-    public boolean requireFullClusteringKey()
-    {
-        return true;
+        super(type, bindVariables, metadata, operations, restrictions, conditions, attrs);
     }
 
     @Override
@@ -127,7 +123,7 @@ public class UpdateStatement extends ModificationStatement
          * @param columnValues list of column values (corresponds to names)
          * @param ifNotExists true if an IF NOT EXISTS condition was specified, false otherwise
          */
-        public ParsedInsert(CFName name,
+        public ParsedInsert(QualifiedName name,
                             Attributes.Raw attrs,
                             List<ColumnMetadata.Raw> columnNames,
                             List<Term.Raw> columnValues,
@@ -140,7 +136,7 @@ public class UpdateStatement extends ModificationStatement
 
         @Override
         protected ModificationStatement prepareInternal(TableMetadata metadata,
-                                                        VariableSpecifications boundNames,
+                                                        VariableSpecifications bindVariables,
                                                         Conditions conditions,
                                                         Attributes attrs)
         {
@@ -173,7 +169,7 @@ public class UpdateStatement extends ModificationStatement
                 else
                 {
                     Operation operation = new Operation.SetValue(value).prepare(metadata, def);
-                    operation.collectMarkerSpecification(boundNames);
+                    operation.collectMarkerSpecification(bindVariables);
                     operations.add(operation);
                 }
             }
@@ -183,13 +179,13 @@ public class UpdateStatement extends ModificationStatement
             StatementRestrictions restrictions = new StatementRestrictions(type,
                                                                            metadata,
                                                                            whereClause.build(),
-                                                                           boundNames,
+                                                                           bindVariables,
                                                                            applyOnlyToStaticColumns,
                                                                            false,
                                                                            false);
 
             return new UpdateStatement(type,
-                                       boundNames.size(),
+                                       bindVariables,
                                        metadata,
                                        operations,
                                        restrictions,
@@ -206,7 +202,7 @@ public class UpdateStatement extends ModificationStatement
         private final Json.Raw jsonValue;
         private final boolean defaultUnset;
 
-        public ParsedInsertJson(CFName name, Attributes.Raw attrs, Json.Raw jsonValue, boolean defaultUnset, boolean ifNotExists)
+        public ParsedInsertJson(QualifiedName name, Attributes.Raw attrs, Json.Raw jsonValue, boolean defaultUnset, boolean ifNotExists)
         {
             super(name, StatementType.INSERT, attrs, null, ifNotExists, false);
             this.jsonValue = jsonValue;
@@ -215,14 +211,14 @@ public class UpdateStatement extends ModificationStatement
 
         @Override
         protected ModificationStatement prepareInternal(TableMetadata metadata,
-                                                        VariableSpecifications boundNames,
+                                                        VariableSpecifications bindVariables,
                                                         Conditions conditions,
                                                         Attributes attrs)
         {
             checkFalse(metadata.isCounter(), "INSERT statements are not allowed on counter tables, use UPDATE instead");
 
             Collection<ColumnMetadata> defs = metadata.columns();
-            Json.Prepared prepared = jsonValue.prepareAndCollectMarkers(metadata, defs, boundNames);
+            Json.Prepared prepared = jsonValue.prepareAndCollectMarkers(metadata, defs, bindVariables);
 
             WhereClause.Builder whereClause = new WhereClause.Builder();
             Operations operations = new Operations(type);
@@ -241,7 +237,7 @@ public class UpdateStatement extends ModificationStatement
                 else
                 {
                     Operation operation = new Operation.SetValue(raw).prepare(metadata, def);
-                    operation.collectMarkerSpecification(boundNames);
+                    operation.collectMarkerSpecification(bindVariables);
                     operations.add(operation);
                 }
             }
@@ -251,13 +247,13 @@ public class UpdateStatement extends ModificationStatement
             StatementRestrictions restrictions = new StatementRestrictions(type,
                                                                            metadata,
                                                                            whereClause.build(),
-                                                                           boundNames,
+                                                                           bindVariables,
                                                                            applyOnlyToStaticColumns,
                                                                            false,
                                                                            false);
 
             return new UpdateStatement(type,
-                                       boundNames.size(),
+                                       bindVariables,
                                        metadata,
                                        operations,
                                        restrictions,
@@ -282,7 +278,7 @@ public class UpdateStatement extends ModificationStatement
          * @param whereClause the where clause
          * @param ifExists flag to check if row exists
          * */
-        public ParsedUpdate(CFName name,
+        public ParsedUpdate(QualifiedName name,
                             Attributes.Raw attrs,
                             List<Pair<ColumnMetadata.Raw, Operation.RawUpdate>> updates,
                             WhereClause whereClause,
@@ -296,7 +292,7 @@ public class UpdateStatement extends ModificationStatement
 
         @Override
         protected ModificationStatement prepareInternal(TableMetadata metadata,
-                                                        VariableSpecifications boundNames,
+                                                        VariableSpecifications bindVariables,
                                                         Conditions conditions,
                                                         Attributes attrs)
         {
@@ -309,18 +305,18 @@ public class UpdateStatement extends ModificationStatement
                 checkFalse(def.isPrimaryKeyColumn(), "PRIMARY KEY part %s found in SET part", def.name);
 
                 Operation operation = entry.right.prepare(metadata, def);
-                operation.collectMarkerSpecification(boundNames);
+                operation.collectMarkerSpecification(bindVariables);
                 operations.add(operation);
             }
 
             StatementRestrictions restrictions = newRestrictions(metadata,
-                                                                 boundNames,
+                                                                 bindVariables,
                                                                  operations,
                                                                  whereClause,
                                                                  conditions);
 
             return new UpdateStatement(type,
-                                       boundNames.size(),
+                                       bindVariables,
                                        metadata,
                                        operations,
                                        restrictions,

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/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 381ed3a..3013d9f 100644
--- a/src/java/org/apache/cassandra/cql3/statements/UseStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/UseStatement.java
@@ -17,6 +17,7 @@
  */
 package org.apache.cassandra.cql3.statements;
 
+import org.apache.cassandra.audit.AuditLogContext;
 import org.apache.cassandra.audit.AuditLogEntryType;
 import org.apache.cassandra.cql3.CQLStatement;
 import org.apache.cassandra.cql3.QueryOptions;
@@ -28,7 +29,7 @@ import org.apache.cassandra.service.QueryState;
 import org.apache.commons.lang3.builder.ToStringBuilder;
 import org.apache.commons.lang3.builder.ToStringStyle;
 
-public class UseStatement extends ParsedStatement implements CQLStatement
+public class UseStatement extends CQLStatement.Raw implements CQLStatement
 {
     private final String keyspace;
 
@@ -37,17 +38,12 @@ public class UseStatement extends ParsedStatement implements CQLStatement
         this.keyspace = keyspace;
     }
 
-    public int getBoundTerms()
+    public UseStatement prepare(ClientState state)
     {
-        return 0;
+        return this;
     }
 
-    public Prepared prepare() throws InvalidRequestException
-    {
-        return new Prepared(this);
-    }
-
-    public void checkAccess(ClientState state) throws UnauthorizedException
+    public void authorize(ClientState state) throws UnauthorizedException
     {
         state.validateLogin();
     }
@@ -62,7 +58,7 @@ public class UseStatement extends ParsedStatement implements CQLStatement
         return new ResultMessage.SetKeyspace(keyspace);
     }
 
-    public ResultMessage executeInternal(QueryState state, QueryOptions options) throws InvalidRequestException
+    public ResultMessage executeLocally(QueryState state, QueryOptions options) throws InvalidRequestException
     {
         // In production, internal queries are exclusively on the system keyspace and 'use' is thus useless
         // but for some unit tests we need to set the keyspace (e.g. for tests with DROP INDEX)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/cql3/statements/schema/AlterKeyspaceStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/schema/AlterKeyspaceStatement.java b/src/java/org/apache/cassandra/cql3/statements/schema/AlterKeyspaceStatement.java
new file mode 100644
index 0000000..c2d0e4c
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/statements/schema/AlterKeyspaceStatement.java
@@ -0,0 +1,114 @@
+/*
+ * 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.statements.schema;
+
+import java.util.Set;
+
+import com.google.common.collect.ImmutableSet;
+
+import org.apache.cassandra.audit.AuditLogContext;
+import org.apache.cassandra.audit.AuditLogEntryType;
+import org.apache.cassandra.auth.Permission;
+import org.apache.cassandra.cql3.CQLStatement;
+import org.apache.cassandra.locator.AbstractReplicationStrategy;
+import org.apache.cassandra.locator.LocalStrategy;
+import org.apache.cassandra.schema.KeyspaceMetadata;
+import org.apache.cassandra.schema.KeyspaceMetadata.KeyspaceDiff;
+import org.apache.cassandra.schema.Keyspaces;
+import org.apache.cassandra.schema.Keyspaces.KeyspacesDiff;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.transport.Event.SchemaChange;
+import org.apache.cassandra.transport.Event.SchemaChange.Change;
+
+public final class AlterKeyspaceStatement extends AlterSchemaStatement
+{
+    private final KeyspaceAttributes attrs;
+
+    public AlterKeyspaceStatement(String keyspaceName, KeyspaceAttributes attrs)
+    {
+        super(keyspaceName);
+        this.attrs = attrs;
+    }
+
+    public Keyspaces apply(Keyspaces schema)
+    {
+        attrs.validate();
+
+        KeyspaceMetadata keyspace = schema.getNullable(keyspaceName);
+        if (null == keyspace)
+            throw ire("Keyspace '%s' doesn't exist", keyspaceName);
+
+        KeyspaceMetadata newKeyspace = keyspace.withSwapped(attrs.asAlteredKeyspaceParams(keyspace.params));
+
+        if (newKeyspace.params.replication.klass.equals(LocalStrategy.class))
+            throw ire("Unable to use given strategy class: LocalStrategy is reserved for internal use.");
+
+        newKeyspace.params.validate(keyspaceName);
+
+        return schema.withAddedOrUpdated(newKeyspace);
+    }
+
+    SchemaChange schemaChangeEvent(KeyspacesDiff diff)
+    {
+        return new SchemaChange(Change.UPDATED, keyspaceName);
+    }
+
+    public void authorize(ClientState client)
+    {
+        client.ensureKeyspacePermission(keyspaceName, Permission.ALTER);
+    }
+
+    @Override
+    Set<String> clientWarnings(KeyspacesDiff diff)
+    {
+        if (diff.isEmpty())
+            return ImmutableSet.of();
+
+        KeyspaceDiff keyspaceDiff = diff.altered.get(0);
+
+        AbstractReplicationStrategy before = keyspaceDiff.before.createReplicationStrategy();
+        AbstractReplicationStrategy after = keyspaceDiff.after.createReplicationStrategy();
+
+        return before.getReplicationFactor() < after.getReplicationFactor()
+             ? ImmutableSet.of("When increasing replication factor you need to run a full (-full) repair to distribute the data.")
+             : ImmutableSet.of();
+    }
+
+    @Override
+    public AuditLogContext getAuditLogContext()
+    {
+        return new AuditLogContext(AuditLogEntryType.ALTER_KEYSPACE, keyspaceName);
+    }
+
+    public static final class Raw extends CQLStatement.Raw
+    {
+        private final String keyspaceName;
+        private final KeyspaceAttributes attrs;
+
+        public Raw(String keyspaceName, KeyspaceAttributes attrs)
+        {
+            this.keyspaceName = keyspaceName;
+            this.attrs = attrs;
+        }
+
+        public AlterKeyspaceStatement prepare(ClientState state)
+        {
+            return new AlterKeyspaceStatement(keyspaceName, attrs);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/cql3/statements/schema/AlterSchemaStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/schema/AlterSchemaStatement.java b/src/java/org/apache/cassandra/cql3/statements/schema/AlterSchemaStatement.java
new file mode 100644
index 0000000..161c9c4
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/statements/schema/AlterSchemaStatement.java
@@ -0,0 +1,153 @@
+/*
+ * 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.statements.schema;
+
+import java.util.Set;
+
+import com.google.common.collect.ImmutableSet;
+
+import org.apache.cassandra.auth.AuthenticatedUser;
+import org.apache.cassandra.auth.IResource;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.cql3.CQLStatement;
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.schema.*;
+import org.apache.cassandra.schema.Keyspaces.KeyspacesDiff;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.service.ClientWarn;
+import org.apache.cassandra.service.QueryState;
+import org.apache.cassandra.transport.Event.SchemaChange;
+import org.apache.cassandra.transport.messages.ResultMessage;
+
+abstract class AlterSchemaStatement implements CQLStatement, SchemaTransformation
+{
+    protected final String keyspaceName; // name of the keyspace affected by the statement
+
+    protected AlterSchemaStatement(String keyspaceName)
+    {
+        this.keyspaceName = keyspaceName;
+    }
+
+    public final void validate(ClientState state)
+    {
+        // no-op; validation is performed while executing the statement, in apply()
+    }
+
+    public ResultMessage execute(QueryState state, QueryOptions options, long queryStartNanoTime)
+    {
+        return execute(state, false);
+    }
+
+    public ResultMessage executeLocally(QueryState state, QueryOptions options)
+    {
+        return execute(state, true);
+    }
+
+    /**
+     * TODO: document
+     */
+    abstract SchemaChange schemaChangeEvent(KeyspacesDiff diff);
+
+    /**
+     * Schema alteration may result in a new database object (keyspace, table, role, function) being created capable of
+     * having permissions GRANTed on it. The creator of the object (the primary role assigned to the AuthenticatedUser
+     * performing the operation) is automatically granted ALL applicable permissions on the object. This is a hook for
+     * subclasses to override in order indicate which resources to to perform that grant on when the statement is executed.
+     *
+     * Only called if the transformation resulted in a non-empty diff.
+     */
+    Set<IResource> createdResources(KeyspacesDiff diff)
+    {
+        return ImmutableSet.of();
+    }
+
+    /**
+     * Schema alteration might produce a client warning (e.g. a warning to run full repair when increading RF of a keyspace).
+     * This method should be used to generate them instead of calling warn() in transformation code.
+     *
+     * Only called if the transformation resulted in a non-empty diff.
+     */
+    Set<String> clientWarnings(KeyspacesDiff diff)
+    {
+        return ImmutableSet.of();
+    }
+
+    public ResultMessage execute(QueryState state, boolean locally)
+    {
+        if (SchemaConstants.isLocalSystemKeyspace(keyspaceName))
+            throw ire("System keyspace '%s' is not user-modifiable", keyspaceName);
+
+        KeyspaceMetadata keyspace = Schema.instance.getKeyspaceMetadata(keyspaceName);
+        if (null != keyspace && keyspace.isVirtual())
+            throw ire("Virtual keyspace '%s' is not user-modifiable", keyspaceName);
+
+        validateKeyspaceName();
+
+        KeyspacesDiff diff = MigrationManager.announce(this, locally);
+
+        clientWarnings(diff).forEach(ClientWarn.instance::warn);
+
+        if (diff.isEmpty())
+            return new ResultMessage.Void();
+
+        /*
+         * When a schema alteration results in a new db object being created, we grant permissions on the new
+         * object to the user performing the request if:
+         * - the user is not anonymous
+         * - the configured IAuthorizer supports granting of permissions (not all do, AllowAllAuthorizer doesn't and
+         *   custom external implementations may not)
+         */
+        AuthenticatedUser user = state.getClientState().getUser();
+        if (null != user && !user.isAnonymous())
+            createdResources(diff).forEach(r -> grantPermissionsOnResource(r, user));
+
+        return new ResultMessage.SchemaChange(schemaChangeEvent(diff));
+    }
+
+    private void validateKeyspaceName()
+    {
+        if (!SchemaConstants.isValidName(keyspaceName))
+        {
+            throw ire("Keyspace name must not be empty, more than %d characters long, " +
+                      "or contain non-alphanumeric-underscore characters (got '%s')",
+                      SchemaConstants.NAME_LENGTH, keyspaceName);
+        }
+    }
+
+    private void grantPermissionsOnResource(IResource resource, AuthenticatedUser user)
+    {
+        try
+        {
+            DatabaseDescriptor.getAuthorizer()
+                              .grant(AuthenticatedUser.SYSTEM_USER,
+                                     resource.applicablePermissions(),
+                                     resource,
+                                     user.getPrimaryRole());
+        }
+        catch (UnsupportedOperationException e)
+        {
+            // not a problem - grant is an optional method on IAuthorizer
+        }
+    }
+
+    static InvalidRequestException ire(String format, Object... args)
+    {
+        return new InvalidRequestException(String.format(format, args));
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/cql3/statements/schema/AlterTableStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/schema/AlterTableStatement.java b/src/java/org/apache/cassandra/cql3/statements/schema/AlterTableStatement.java
new file mode 100644
index 0000000..a081a2c
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/statements/schema/AlterTableStatement.java
@@ -0,0 +1,441 @@
+/*
+ * 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.statements.schema;
+
+import java.util.*;
+
+import org.apache.cassandra.audit.AuditLogContext;
+import org.apache.cassandra.audit.AuditLogEntryType;
+import org.apache.cassandra.auth.Permission;
+import org.apache.cassandra.cql3.*;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.schema.*;
+import org.apache.cassandra.schema.Keyspaces.KeyspacesDiff;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.transport.Event.SchemaChange;
+import org.apache.cassandra.transport.Event.SchemaChange.Change;
+import org.apache.cassandra.transport.Event.SchemaChange.Target;
+import org.apache.cassandra.utils.FBUtilities;
+
+import static java.lang.String.join;
+
+import static com.google.common.collect.Iterables.isEmpty;
+import static com.google.common.collect.Iterables.transform;
+
+public abstract class AlterTableStatement extends AlterSchemaStatement
+{
+    protected final String tableName;
+
+    public AlterTableStatement(String keyspaceName, String tableName)
+    {
+        super(keyspaceName);
+        this.tableName = tableName;
+    }
+
+    public Keyspaces apply(Keyspaces schema)
+    {
+        KeyspaceMetadata keyspace = schema.getNullable(keyspaceName);
+
+        TableMetadata table = null == keyspace
+                            ? null
+                            : keyspace.getTableOrViewNullable(tableName);
+
+        if (null == table)
+            throw ire("Table '%s.%s' doesn't exist", keyspaceName, tableName);
+
+        if (table.isView())
+            throw ire("Cannot use ALTER TABLE on a materialized view; use ALTER MATERIALIZED VIEW instead");
+
+        return schema.withAddedOrUpdated(apply(keyspace, table));
+    }
+
+    SchemaChange schemaChangeEvent(KeyspacesDiff diff)
+    {
+        return new SchemaChange(Change.UPDATED, Target.TABLE, keyspaceName, tableName);
+    }
+
+    public void authorize(ClientState client)
+    {
+        client.ensureTablePermission(keyspaceName, tableName, Permission.ALTER);
+    }
+
+    @Override
+    public AuditLogContext getAuditLogContext()
+    {
+        return new AuditLogContext(AuditLogEntryType.ALTER_TABLE, keyspaceName, tableName);
+    }
+
+    abstract KeyspaceMetadata apply(KeyspaceMetadata keyspace, TableMetadata table);
+
+    /**
+     * ALTER TABLE <table> ALTER <column> TYPE <newtype>;
+     *
+     * No longer supported.
+     */
+    public static class AlterColumn extends AlterTableStatement
+    {
+        AlterColumn(String keyspaceName, String tableName)
+        {
+            super(keyspaceName, tableName);
+        }
+
+        public KeyspaceMetadata apply(KeyspaceMetadata keyspace, TableMetadata table)
+        {
+            throw ire("Altering column types is no longer supported");
+        }
+    }
+
+    /**
+     * ALTER TABLE <table> ADD <column> <newtype>
+     * ALTER TABLE <table> ADD (<column> <newtype>, <column1> <newtype1>, ... <columnn> <newtypen>)
+     */
+    private static class AddColumns extends AlterTableStatement
+    {
+        private static class Column
+        {
+            private final ColumnMetadata.Raw name;
+            private final CQL3Type.Raw type;
+            private final boolean isStatic;
+
+            Column(ColumnMetadata.Raw name, CQL3Type.Raw type, boolean isStatic)
+            {
+                this.name = name;
+                this.type = type;
+                this.isStatic = isStatic;
+            }
+        }
+
+        private final Collection<Column> newColumns;
+
+        private AddColumns(String keyspaceName, String tableName, Collection<Column> newColumns)
+        {
+            super(keyspaceName, tableName);
+            this.newColumns = newColumns;
+        }
+
+        public KeyspaceMetadata apply(KeyspaceMetadata keyspace, TableMetadata table)
+        {
+            TableMetadata.Builder tableBuilder = table.unbuild();
+            Views.Builder viewsBuilder = keyspace.views.unbuild();
+            newColumns.forEach(c -> addColumn(keyspace, table, c, tableBuilder, viewsBuilder));
+
+            return keyspace.withSwapped(keyspace.tables.withSwapped(tableBuilder.build()))
+                           .withSwapped(viewsBuilder.build());
+        }
+
+        private void addColumn(KeyspaceMetadata keyspace,
+                               TableMetadata table,
+                               Column column,
+                               TableMetadata.Builder tableBuilder,
+                               Views.Builder viewsBuilder)
+        {
+            ColumnIdentifier name = column.name.getIdentifier(table);
+            AbstractType<?> type = column.type.prepare(keyspaceName, keyspace.types).getType();
+            boolean isStatic = column.isStatic;
+
+            if (null != tableBuilder.getColumn(name))
+                throw ire("Column with name '%s' already exists", name);
+
+            if (isStatic && table.clusteringColumns().isEmpty())
+                throw ire("Static columns are only useful (and thus allowed) if the table has at least one clustering column");
+
+            ColumnMetadata droppedColumn = table.getDroppedColumn(name.bytes);
+            if (null != droppedColumn)
+            {
+                // After #8099, not safe to re-add columns of incompatible types - until *maybe* deser logic with dropped
+                // columns is pushed deeper down the line. The latter would still be problematic in cases of schema races.
+                if (!droppedColumn.type.isValueCompatibleWith(type))
+                {
+                    throw ire("Cannot re-add a previously dropped column '%s' of type %s, incompatible with previous type %s",
+                              name,
+                              type.asCQL3Type(),
+                              droppedColumn.type.asCQL3Type());
+                }
+
+                // Cannot re-add a dropped counter column. See #7831.
+                if (table.isCounter())
+                    throw ire("Cannot re-add previously dropped counter column %s", name);
+            }
+
+            if (isStatic)
+                tableBuilder.addStaticColumn(name, type);
+            else
+                tableBuilder.addRegularColumn(name, type);
+
+            if (!isStatic)
+            {
+                for (ViewMetadata view : keyspace.views.forTable(table.id))
+                {
+                    if (view.includeAllColumns)
+                    {
+                        ColumnMetadata viewColumn = ColumnMetadata.regularColumn(view.metadata, name.bytes, type);
+                        viewsBuilder.put(viewsBuilder.get(view.name()).withAddedRegularColumn(viewColumn));
+                    }
+                }
+            }
+        }
+    }
+
+    /**
+     * ALTER TABLE <table> DROP <column>
+     * ALTER TABLE <table> DROP ( <column>, <column1>, ... <columnn>)
+     */
+    // TODO: swap UDT refs with expanded tuples on drop
+    private static class DropColumns extends AlterTableStatement
+    {
+        private final Collection<ColumnMetadata.Raw> removedColumns;
+        private final long timestamp;
+
+        private DropColumns(String keyspaceName, String tableName, Collection<ColumnMetadata.Raw> removedColumns, long timestamp)
+        {
+            super(keyspaceName, tableName);
+            this.removedColumns = removedColumns;
+            this.timestamp = timestamp;
+        }
+
+        public KeyspaceMetadata apply(KeyspaceMetadata keyspace, TableMetadata table)
+        {
+            TableMetadata.Builder builder = table.unbuild();
+            removedColumns.forEach(c -> dropColumn(keyspace, table, c, builder));
+            return keyspace.withSwapped(keyspace.tables.withSwapped(builder.build()));
+        }
+
+        private void dropColumn(KeyspaceMetadata keyspace, TableMetadata table, ColumnMetadata.Raw column, TableMetadata.Builder builder)
+        {
+            ColumnIdentifier name = column.getIdentifier(table);
+
+            ColumnMetadata currentColumn = table.getColumn(name);
+            if (null == currentColumn)
+                throw ire("Column %s was not found in table '%s'", name, table);
+
+            if (currentColumn.isPrimaryKeyColumn())
+                throw ire("Cannot drop PRIMARY KEY column %s", name);
+
+            /*
+             * Cannot allow dropping top-level columns of user defined types that aren't frozen because we cannot convert
+             * the type into an equivalent tuple: we only support frozen tuples currently. And as such we cannot persist
+             * the correct type in system_schema.dropped_columns.
+             */
+            if (currentColumn.type.isUDT() && currentColumn.type.isMultiCell())
+                throw ire("Cannot drop non-frozen column %s of user type %s", name, currentColumn.type.asCQL3Type());
+
+            // TODO: some day try and find a way to not rely on Keyspace/IndexManager/Index to find dependent indexes
+            Set<IndexMetadata> dependentIndexes = Keyspace.openAndGetStore(table).indexManager.getDependentIndexes(currentColumn);
+            if (!dependentIndexes.isEmpty())
+            {
+                throw ire("Cannot drop column %s because it has dependent secondary indexes (%s)",
+                          currentColumn,
+                          join(", ", transform(dependentIndexes, i -> i.name)));
+            }
+
+            if (!isEmpty(keyspace.views.forTable(table.id)))
+                throw ire("Cannot drop column %s on base table %s with materialized views", currentColumn, table.name);
+
+            builder.removeRegularOrStaticColumn(name);
+            builder.recordColumnDrop(currentColumn, timestamp);
+        }
+    }
+
+    /**
+     * ALTER TABLE <table> RENAME <column> TO <column>;
+     */
+    private static class RenameColumns extends AlterTableStatement
+    {
+        private final Map<ColumnMetadata.Raw, ColumnMetadata.Raw> renamedColumns;
+
+        private RenameColumns(String keyspaceName, String tableName, Map<ColumnMetadata.Raw, ColumnMetadata.Raw> renamedColumns)
+        {
+            super(keyspaceName, tableName);
+            this.renamedColumns = renamedColumns;
+        }
+
+        public KeyspaceMetadata apply(KeyspaceMetadata keyspace, TableMetadata table)
+        {
+            TableMetadata.Builder tableBuilder = table.unbuild();
+            Views.Builder viewsBuilder = keyspace.views.unbuild();
+            renamedColumns.forEach((o, n) -> renameColumn(keyspace, table, o, n, tableBuilder, viewsBuilder));
+
+            return keyspace.withSwapped(keyspace.tables.withSwapped(tableBuilder.build()))
+                           .withSwapped(viewsBuilder.build());
+        }
+
+        private void renameColumn(KeyspaceMetadata keyspace,
+                                  TableMetadata table,
+                                  ColumnMetadata.Raw oldName,
+                                  ColumnMetadata.Raw newName,
+                                  TableMetadata.Builder tableBuilder,
+                                  Views.Builder viewsBuilder)
+        {
+            ColumnIdentifier oldColumnName = oldName.getIdentifier(table);
+            ColumnIdentifier newColumnName = newName.getIdentifier(table);
+
+            ColumnMetadata column = table.getColumn(oldColumnName);
+            if (null == column)
+                throw ire("Column %s was not found in table %s", oldColumnName, table);
+
+            if (!column.isPrimaryKeyColumn())
+                throw ire("Cannot rename non PRIMARY KEY column %s", oldColumnName);
+
+            if (null != table.getColumn(newColumnName))
+            {
+                throw ire("Cannot rename column %s to %s in table '%s'; another column with that name already exists",
+                          oldColumnName,
+                          newColumnName,
+                          table);
+            }
+
+            // TODO: some day try and find a way to not rely on Keyspace/IndexManager/Index to find dependent indexes
+            Set<IndexMetadata> dependentIndexes = Keyspace.openAndGetStore(table).indexManager.getDependentIndexes(column);
+            if (!dependentIndexes.isEmpty())
+            {
+                throw ire("Can't rename column %s because it has dependent secondary indexes (%s)",
+                          oldColumnName,
+                          join(", ", transform(dependentIndexes, i -> i.name)));
+            }
+
+            for (ViewMetadata view : keyspace.views.forTable(table.id))
+            {
+                if (view.includes(oldColumnName))
+                {
+                    ColumnIdentifier oldViewColumn = oldName.getIdentifier(view.metadata);
+                    ColumnIdentifier newViewColumn = newName.getIdentifier(view.metadata);
+
+                    viewsBuilder.put(viewsBuilder.get(view.name()).withRenamedPrimaryKeyColumn(oldViewColumn, newViewColumn));
+                }
+            }
+
+            tableBuilder.renamePrimaryKeyColumn(oldColumnName, newColumnName);
+        }
+    }
+
+    /**
+     * ALTER TABLE <table> WITH <property> = <value>
+     */
+    private static class AlterOptions extends AlterTableStatement
+    {
+        private final TableAttributes attrs;
+
+        private AlterOptions(String keyspaceName, String tableName, TableAttributes attrs)
+        {
+            super(keyspaceName, tableName);
+            this.attrs = attrs;
+        }
+
+        public KeyspaceMetadata apply(KeyspaceMetadata keyspace, TableMetadata table)
+        {
+            attrs.validate();
+
+            TableParams params = attrs.asAlteredTableParams(table.params);
+
+            if (table.isCounter() && params.defaultTimeToLive > 0)
+                throw ire("Cannot set default_time_to_live on a table with counters");
+
+            if (!isEmpty(keyspace.views.forTable(table.id)) && params.gcGraceSeconds == 0)
+            {
+                throw ire("Cannot alter gc_grace_seconds of the base table of a " +
+                          "materialized view to 0, since this value is used to TTL " +
+                          "undelivered updates. Setting gc_grace_seconds too low might " +
+                          "cause undelivered updates to expire " +
+                          "before being replayed.");
+            }
+
+            return keyspace.withSwapped(keyspace.tables.withSwapped(table.withSwapped(params)));
+        }
+    }
+
+    public static final class Raw extends CQLStatement.Raw
+    {
+        private enum Kind
+        {
+            ALTER_COLUMN, ADD_COLUMNS, DROP_COLUMNS, RENAME_COLUMNS, ALTER_OPTIONS
+        }
+
+        private final QualifiedName name;
+
+        private Kind kind;
+
+        // ADD
+        private final List<AddColumns.Column> addedColumns = new ArrayList<>();
+
+        // DROP
+        private final List<ColumnMetadata.Raw> droppedColumns = new ArrayList<>();
+        private long timestamp = FBUtilities.timestampMicros();
+
+        // RENAME
+        private final Map<ColumnMetadata.Raw, ColumnMetadata.Raw> renamedColumns = new HashMap<>();
+
+        // OPTIONS
+        public final TableAttributes attrs = new TableAttributes();
+
+        public Raw(QualifiedName name)
+        {
+            this.name = name;
+        }
+
+        public AlterTableStatement prepare(ClientState state)
+        {
+            String keyspaceName = name.hasKeyspace() ? name.getKeyspace() : state.getKeyspace();
+            String tableName = name.getName();
+
+            switch (kind)
+            {
+                case   ALTER_COLUMN: return new AlterColumn(keyspaceName, tableName);
+                case    ADD_COLUMNS: return new AddColumns(keyspaceName, tableName, addedColumns);
+                case   DROP_COLUMNS: return new DropColumns(keyspaceName, tableName, droppedColumns, timestamp);
+                case RENAME_COLUMNS: return new RenameColumns(keyspaceName, tableName, renamedColumns);
+                case  ALTER_OPTIONS: return new AlterOptions(keyspaceName, tableName, attrs);
+            }
+
+            throw new AssertionError();
+        }
+
+        public void alter(ColumnMetadata.Raw name, CQL3Type.Raw type)
+        {
+            kind = Kind.ALTER_COLUMN;
+        }
+
+        public void add(ColumnMetadata.Raw name, CQL3Type.Raw type, boolean isStatic)
+        {
+            kind = Kind.ADD_COLUMNS;
+            addedColumns.add(new AddColumns.Column(name, type, isStatic));
+        }
+
+        public void drop(ColumnMetadata.Raw name)
+        {
+            kind = Kind.DROP_COLUMNS;
+            droppedColumns.add(name);
+        }
+
+        public void timestamp(long timestamp)
+        {
+            this.timestamp = timestamp;
+        }
+
+        public void rename(ColumnMetadata.Raw from, ColumnMetadata.Raw to)
+        {
+            kind = Kind.RENAME_COLUMNS;
+            renamedColumns.put(from, to);
+        }
+
+        public void attrs()
+        {
+            this.kind = Kind.ALTER_OPTIONS;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/cql3/statements/schema/AlterTypeStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/schema/AlterTypeStatement.java b/src/java/org/apache/cassandra/cql3/statements/schema/AlterTypeStatement.java
new file mode 100644
index 0000000..50f09a0
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/statements/schema/AlterTypeStatement.java
@@ -0,0 +1,229 @@
+/*
+ * 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.statements.schema;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.cassandra.audit.AuditLogContext;
+import org.apache.cassandra.audit.AuditLogEntryType;
+import org.apache.cassandra.auth.Permission;
+import org.apache.cassandra.cql3.*;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.UserType;
+import org.apache.cassandra.schema.KeyspaceMetadata;
+import org.apache.cassandra.schema.Keyspaces;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.transport.Event.SchemaChange;
+import org.apache.cassandra.transport.Event.SchemaChange.Change;
+import org.apache.cassandra.transport.Event.SchemaChange.Target;
+
+import static java.lang.String.join;
+import static java.util.function.Predicate.isEqual;
+import static java.util.stream.Collectors.toList;
+
+import static org.apache.cassandra.utils.ByteBufferUtil.bytes;
+
+public abstract class AlterTypeStatement extends AlterSchemaStatement
+{
+    protected final String typeName;
+
+    public AlterTypeStatement(String keyspaceName, String typeName)
+    {
+        super(keyspaceName);
+        this.typeName = typeName;
+    }
+
+    public void authorize(ClientState client)
+    {
+        client.ensureKeyspacePermission(keyspaceName, Permission.ALTER);
+    }
+
+    SchemaChange schemaChangeEvent(Keyspaces.KeyspacesDiff diff)
+    {
+        return new SchemaChange(Change.UPDATED, Target.TYPE, keyspaceName, typeName);
+    }
+
+    public Keyspaces apply(Keyspaces schema)
+    {
+        KeyspaceMetadata keyspace = schema.getNullable(keyspaceName);
+
+        UserType type = null == keyspace
+                      ? null
+                      : keyspace.types.getNullable(bytes(typeName));
+
+        if (null == type)
+            throw ire("Type %s.%s doesn't exist", keyspaceName, typeName);
+
+        return schema.withAddedOrUpdated(keyspace.withUpdatedUserType(apply(keyspace, type)));
+    }
+
+    abstract UserType apply(KeyspaceMetadata keyspace, UserType type);
+
+    @Override
+    public AuditLogContext getAuditLogContext()
+    {
+        return new AuditLogContext(AuditLogEntryType.ALTER_TYPE, keyspaceName, typeName);
+    }
+
+    private static final class AddField extends AlterTypeStatement
+    {
+        private final FieldIdentifier fieldName;
+        private final CQL3Type.Raw type;
+
+        private AddField(String keyspaceName, String typeName, FieldIdentifier fieldName, CQL3Type.Raw type)
+        {
+            super(keyspaceName, typeName);
+            this.fieldName = fieldName;
+            this.type = type;
+        }
+
+        UserType apply(KeyspaceMetadata keyspace, UserType userType)
+        {
+            if (userType.fieldPosition(fieldName) >= 0)
+                throw ire("Cannot add field %s to type %s: a field with name %s already exists", fieldName, userType.toCQLString(), fieldName);
+
+            AbstractType<?> fieldType = type.prepare(keyspaceName, keyspace.types).getType();
+            if (fieldType.referencesUserType(userType.name))
+                throw ire("Cannot add new field %s of type %s to user type %s as it would create a circular reference", fieldName, type, userType.toCQLString());
+
+            List<FieldIdentifier> fieldNames = new ArrayList<>(userType.fieldNames()); fieldNames.add(fieldName);
+            List<AbstractType<?>> fieldTypes = new ArrayList<>(userType.fieldTypes()); fieldTypes.add(fieldType);
+
+            return new UserType(keyspaceName, userType.name, fieldNames, fieldTypes, true);
+        }
+    }
+
+    private static final class RenameFields extends AlterTypeStatement
+    {
+        private final Map<FieldIdentifier, FieldIdentifier> renamedFields;
+
+        private RenameFields(String keyspaceName, String typeName, Map<FieldIdentifier, FieldIdentifier> renamedFields)
+        {
+            super(keyspaceName, typeName);
+            this.renamedFields = renamedFields;
+        }
+
+        UserType apply(KeyspaceMetadata keyspace, UserType userType)
+        {
+            List<String> dependentAggregates =
+                keyspace.functions
+                        .udas()
+                        .filter(uda -> null != uda.initialCondition() && uda.stateType().referencesUserType(userType.name))
+                        .map(uda -> uda.name().toString())
+                        .collect(toList());
+
+            if (!dependentAggregates.isEmpty())
+            {
+                throw ire("Cannot alter user type %s as it is still used in INITCOND by aggregates %s",
+                          userType.toCQLString(),
+                          join(", ", dependentAggregates));
+            }
+
+            List<FieldIdentifier> fieldNames = new ArrayList<>(userType.fieldNames());
+
+            renamedFields.forEach((oldName, newName) ->
+            {
+                int idx = userType.fieldPosition(oldName);
+                if (idx < 0)
+                    throw ire("Unkown field %s in user type %s", oldName, keyspaceName, userType.toCQLString());
+                fieldNames.set(idx, newName);
+            });
+
+            fieldNames.forEach(name ->
+            {
+                if (fieldNames.stream().filter(isEqual(name)).count() > 1)
+                    throw ire("Duplicate field name %s in type %s", name, keyspaceName, userType.toCQLString());
+            });
+
+            return new UserType(keyspaceName, userType.name, fieldNames, userType.fieldTypes(), true);
+        }
+    }
+
+    private static final class AlterField extends AlterTypeStatement
+    {
+        private AlterField(String keyspaceName, String typeName)
+        {
+            super(keyspaceName, typeName);
+        }
+
+        UserType apply(KeyspaceMetadata keyspace, UserType userType)
+        {
+            throw ire("Alterting field types is no longer supported");
+        }
+    }
+
+    public static final class Raw extends CQLStatement.Raw
+    {
+        private enum Kind
+        {
+            ADD_FIELD, RENAME_FIELDS, ALTER_FIELD
+        }
+
+        private final UTName name;
+
+        private Kind kind;
+
+        // ADD
+        private FieldIdentifier newFieldName;
+        private CQL3Type.Raw newFieldType;
+
+        // RENAME
+        private final Map<FieldIdentifier, FieldIdentifier> renamedFields = new HashMap<>();
+
+        public Raw(UTName name)
+        {
+            this.name = name;
+        }
+
+        public AlterTypeStatement prepare(ClientState state)
+        {
+            String keyspaceName = name.hasKeyspace() ? name.getKeyspace() : state.getKeyspace();
+            String typeName = name.getStringTypeName();
+
+            switch (kind)
+            {
+                case     ADD_FIELD: return new AddField(keyspaceName, typeName, newFieldName, newFieldType);
+                case RENAME_FIELDS: return new RenameFields(keyspaceName, typeName, renamedFields);
+                case   ALTER_FIELD: return new AlterField(keyspaceName, typeName);
+            }
+
+            throw new AssertionError();
+        }
+
+        public void add(FieldIdentifier name, CQL3Type.Raw type)
+        {
+            kind = Kind.ADD_FIELD;
+            newFieldName = name;
+            newFieldType = type;
+        }
+
+        public void rename(FieldIdentifier from, FieldIdentifier to)
+        {
+            kind = Kind.RENAME_FIELDS;
+            renamedFields.put(from, to);
+        }
+
+        public void alter(FieldIdentifier name, CQL3Type.Raw type)
+        {
+            kind = Kind.ALTER_FIELD;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/cql3/statements/schema/AlterViewStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/schema/AlterViewStatement.java b/src/java/org/apache/cassandra/cql3/statements/schema/AlterViewStatement.java
new file mode 100644
index 0000000..2ecc095
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/statements/schema/AlterViewStatement.java
@@ -0,0 +1,112 @@
+/*
+ * 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.statements.schema;
+
+import org.apache.cassandra.audit.AuditLogContext;
+import org.apache.cassandra.audit.AuditLogEntryType;
+import org.apache.cassandra.auth.Permission;
+import org.apache.cassandra.cql3.CQLStatement;
+import org.apache.cassandra.cql3.QualifiedName;
+import org.apache.cassandra.schema.*;
+import org.apache.cassandra.schema.Keyspaces.KeyspacesDiff;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.transport.Event.SchemaChange;
+import org.apache.cassandra.transport.Event.SchemaChange.Change;
+import org.apache.cassandra.transport.Event.SchemaChange.Target;
+
+public final class AlterViewStatement extends AlterSchemaStatement
+{
+    private final String viewName;
+    private final TableAttributes attrs;
+
+    public AlterViewStatement(String keyspaceName, String viewName, TableAttributes attrs)
+    {
+        super(keyspaceName);
+        this.viewName = viewName;
+        this.attrs = attrs;
+    }
+
+    public Keyspaces apply(Keyspaces schema)
+    {
+        KeyspaceMetadata keyspace = schema.getNullable(keyspaceName);
+
+        ViewMetadata view = null == keyspace
+                          ? null
+                          : keyspace.views.getNullable(viewName);
+
+        if (null == view)
+            throw ire("Materialized view '%s.%s' doesn't exist", keyspaceName, viewName);
+
+        attrs.validate();
+
+        TableParams params = attrs.asAlteredTableParams(view.metadata.params);
+
+        if (params.gcGraceSeconds == 0)
+        {
+            throw ire("Cannot alter gc_grace_seconds of a materialized view to 0, since this " +
+                      "value is used to TTL undelivered updates. Setting gc_grace_seconds too " +
+                      "low might cause undelivered updates to expire before being replayed.");
+        }
+
+        if (params.defaultTimeToLive > 0)
+        {
+            throw ire("Cannot set or alter default_time_to_live for a materialized view. " +
+                      "Data in a materialized view always expire at the same time than " +
+                      "the corresponding data in the parent table.");
+        }
+
+        ViewMetadata newView = view.copy(view.metadata.withSwapped(params));
+        return schema.withAddedOrUpdated(keyspace.withSwapped(keyspace.views.withSwapped(newView)));
+    }
+
+    SchemaChange schemaChangeEvent(KeyspacesDiff diff)
+    {
+        return new SchemaChange(Change.UPDATED, Target.TABLE, keyspaceName, viewName);
+    }
+
+    public void authorize(ClientState client)
+    {
+        ViewMetadata view = Schema.instance.getView(keyspaceName, viewName);
+        if (null != view)
+            client.ensureTablePermission(keyspaceName, view.baseTableName, Permission.ALTER);
+    }
+
+    @Override
+    public AuditLogContext getAuditLogContext()
+    {
+        return new AuditLogContext(AuditLogEntryType.ALTER_VIEW, keyspaceName, viewName);
+    }
+
+    public static final class Raw extends CQLStatement.Raw
+    {
+        private final QualifiedName name;
+        private final TableAttributes attrs;
+
+        public Raw(QualifiedName name, TableAttributes attrs)
+        {
+            this.name = name;
+            this.attrs = attrs;
+        }
+
+        public AlterViewStatement prepare(ClientState state)
+        {
+            String keyspaceName = name.hasKeyspace() ? name.getKeyspace() : state.getKeyspace();
+            return new AlterViewStatement(keyspaceName, name.getName(), attrs);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/cql3/statements/schema/CreateAggregateStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/schema/CreateAggregateStatement.java b/src/java/org/apache/cassandra/cql3/statements/schema/CreateAggregateStatement.java
new file mode 100644
index 0000000..cd9808a
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/statements/schema/CreateAggregateStatement.java
@@ -0,0 +1,329 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.cql3.statements.schema;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.Set;
+
+import com.google.common.base.Objects;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Lists;
+
+import org.apache.cassandra.audit.AuditLogContext;
+import org.apache.cassandra.audit.AuditLogEntryType;
+import org.apache.cassandra.auth.FunctionResource;
+import org.apache.cassandra.auth.IResource;
+import org.apache.cassandra.auth.Permission;
+import org.apache.cassandra.cql3.*;
+import org.apache.cassandra.cql3.functions.*;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.schema.Functions.FunctionsDiff;
+import org.apache.cassandra.schema.KeyspaceMetadata;
+import org.apache.cassandra.schema.Keyspaces;
+import org.apache.cassandra.schema.Keyspaces.KeyspacesDiff;
+import org.apache.cassandra.schema.Schema;
+import org.apache.cassandra.serializers.MarshalException;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.transport.Event.SchemaChange;
+import org.apache.cassandra.transport.Event.SchemaChange.Change;
+import org.apache.cassandra.transport.Event.SchemaChange.Target;
+import org.apache.cassandra.transport.ProtocolVersion;
+
+import static java.lang.String.format;
+import static java.lang.String.join;
+import static java.util.Collections.singleton;
+import static java.util.Collections.singletonList;
+import static java.util.stream.Collectors.toList;
+
+import static com.google.common.collect.Iterables.concat;
+import static com.google.common.collect.Iterables.transform;
+
+public final class CreateAggregateStatement extends AlterSchemaStatement
+{
+    private final String aggregateName;
+    private final List<CQL3Type.Raw> rawArgumentTypes;
+    private final CQL3Type.Raw rawStateType;
+    private final FunctionName stateFunctionName;
+    private final FunctionName finalFunctionName;
+    private final Term.Raw rawInitialValue;
+    private final boolean orReplace;
+    private final boolean ifNotExists;
+
+    public CreateAggregateStatement(String keyspaceName,
+                                    String aggregateName,
+                                    List<CQL3Type.Raw> rawArgumentTypes,
+                                    CQL3Type.Raw rawStateType,
+                                    FunctionName stateFunctionName,
+                                    FunctionName finalFunctionName,
+                                    Term.Raw rawInitialValue,
+                                    boolean orReplace,
+                                    boolean ifNotExists)
+    {
+        super(keyspaceName);
+        this.aggregateName = aggregateName;
+        this.rawArgumentTypes = rawArgumentTypes;
+        this.rawStateType = rawStateType;
+        this.stateFunctionName = stateFunctionName;
+        this.finalFunctionName = finalFunctionName;
+        this.rawInitialValue = rawInitialValue;
+        this.orReplace = orReplace;
+        this.ifNotExists = ifNotExists;
+    }
+
+    public Keyspaces apply(Keyspaces schema)
+    {
+        if (ifNotExists && orReplace)
+            throw ire("Cannot use both 'OR REPLACE' and 'IF NOT EXISTS' directives");
+
+        rawArgumentTypes.stream()
+                        .filter(CQL3Type.Raw::isFrozen)
+                        .findFirst()
+                        .ifPresent(t -> { throw ire("Argument '%s' cannot be frozen; remove frozen<> modifier from '%s'", t, t); });
+
+        if (rawStateType.isFrozen())
+            throw ire("State type '%s' cannot be frozen; remove frozen<> modifier from '%s'", rawStateType, rawStateType);
+
+        KeyspaceMetadata keyspace = schema.getNullable(keyspaceName);
+        if (null == keyspace)
+            throw ire("Keyspace '%s' doesn't exist", keyspaceName);
+
+        /*
+         * Resolve the state function
+         */
+
+        List<AbstractType<?>> argumentTypes =
+            rawArgumentTypes.stream()
+                            .map(t -> t.prepare(keyspaceName, keyspace.types).getType())
+                            .collect(toList());
+        AbstractType<?> stateType = rawStateType.prepare(keyspaceName, keyspace.types).getType();
+        List<AbstractType<?>> stateFunctionArguments = Lists.newArrayList(concat(singleton(stateType), argumentTypes));
+
+        Function stateFunction =
+            keyspace.functions
+                    .find(stateFunctionName, stateFunctionArguments)
+                    .orElseThrow(() -> ire("State function %s doesn't exist", stateFunctionString()));
+
+        if (stateFunction.isAggregate())
+            throw ire("State function %s isn't a scalar function", stateFunctionString());
+
+        if (!stateFunction.returnType().equals(stateType))
+        {
+            throw ire("State function %s return type must be the same as the first argument type - check STYPE, argument and return types",
+                      stateFunctionString());
+        }
+
+        /*
+         * Resolve the final function and return type
+         */
+
+        Function finalFunction = null;
+        AbstractType<?> returnType = stateFunction.returnType();
+
+        if (null != finalFunctionName)
+        {
+            finalFunction = keyspace.functions.find(finalFunctionName, singletonList(stateType)).orElse(null);
+            if (null == finalFunction)
+                throw ire("Final function %s doesn't exist", finalFunctionString());
+
+            if (finalFunction.isAggregate())
+                throw ire("Final function %s isn't a scalar function", finalFunctionString());
+
+            // override return type with that of the final function
+            returnType = finalFunction.returnType();
+        }
+
+        /*
+         * Validate initial condition
+         */
+
+        ByteBuffer initialValue = null;
+        if (null != rawInitialValue)
+        {
+            initialValue = Terms.asBytes(keyspaceName, rawInitialValue.toString(), stateType);
+
+            if (null != initialValue)
+            {
+                try
+                {
+                    stateType.validate(initialValue);
+                }
+                catch (MarshalException e)
+                {
+                    throw ire("Invalid value for INITCOND of type %s", stateType.asCQL3Type());
+                }
+            }
+
+            // Converts initcond to a CQL literal and parse it back to avoid another CASSANDRA-11064
+            String initialValueString = stateType.asCQL3Type().toCQLLiteral(initialValue, ProtocolVersion.CURRENT);
+            assert Objects.equal(initialValue, Terms.asBytes(keyspaceName, initialValueString, stateType));
+
+            if (Constants.NULL_LITERAL != rawInitialValue && UDHelper.isNullOrEmpty(stateType, initialValue))
+                throw ire("INITCOND must not be empty for all types except TEXT, ASCII, BLOB");
+        }
+
+        if (!((UDFunction) stateFunction).isCalledOnNullInput() && null == initialValue)
+        {
+            throw ire("Cannot create aggregate '%s' without INITCOND because state function %s does not accept 'null' arguments",
+                      aggregateName,
+                      stateFunctionName);
+        }
+
+        /*
+         * Create or replace
+         */
+
+        UDAggregate aggregate =
+            new UDAggregate(new FunctionName(keyspaceName, aggregateName),
+                            argumentTypes,
+                            returnType,
+                            (ScalarFunction) stateFunction,
+                            (ScalarFunction) finalFunction,
+                            initialValue);
+
+        Function existingAggregate = keyspace.functions.find(aggregate.name(), argumentTypes).orElse(null);
+        if (null != existingAggregate)
+        {
+            if (!existingAggregate.isAggregate())
+                throw ire("Aggregate '%s' cannot replace a function", aggregateName);
+
+            if (ifNotExists)
+                return schema;
+
+            if (!orReplace)
+                throw ire("Aggregate '%s' already exists", aggregateName);
+
+            if (!returnType.isCompatibleWith(existingAggregate.returnType()))
+            {
+                throw ire("Cannot replace aggregate '%s', the new return type %s isn't compatible with the return type %s of existing function",
+                          aggregateName,
+                          returnType.asCQL3Type(),
+                          existingAggregate.returnType().asCQL3Type());
+            }
+        }
+
+        return schema.withAddedOrUpdated(keyspace.withSwapped(keyspace.functions.withAddedOrUpdated(aggregate)));
+    }
+
+    SchemaChange schemaChangeEvent(KeyspacesDiff diff)
+    {
+        assert diff.altered.size() == 1;
+        FunctionsDiff<UDAggregate> udasDiff = diff.altered.get(0).udas;
+
+        assert udasDiff.created.size() + udasDiff.altered.size() == 1;
+        boolean created = !udasDiff.created.isEmpty();
+
+        return new SchemaChange(created ? Change.CREATED : Change.UPDATED,
+                                Target.AGGREGATE,
+                                keyspaceName,
+                                aggregateName,
+                                rawArgumentTypes.stream().map(CQL3Type.Raw::toString).collect(toList()));
+    }
+
+    public void authorize(ClientState client)
+    {
+        FunctionName name = new FunctionName(keyspaceName, aggregateName);
+
+        if (Schema.instance.findFunction(name, Lists.transform(rawArgumentTypes, t -> t.prepare(keyspaceName).getType())).isPresent() && orReplace)
+            client.ensurePermission(Permission.ALTER, FunctionResource.functionFromCql(keyspaceName, aggregateName, rawArgumentTypes));
+        else
+            client.ensurePermission(Permission.CREATE, FunctionResource.keyspace(keyspaceName));
+
+        FunctionResource stateFunction =
+            FunctionResource.functionFromCql(stateFunctionName, Lists.newArrayList(concat(singleton(rawStateType), rawArgumentTypes)));
+        client.ensurePermission(Permission.EXECUTE, stateFunction);
+
+        if (null != finalFunctionName)
+            client.ensurePermission(Permission.EXECUTE, FunctionResource.functionFromCql(finalFunctionName, singletonList(rawStateType)));
+    }
+
+    @Override
+    Set<IResource> createdResources(KeyspacesDiff diff)
+    {
+        assert diff.altered.size() == 1;
+        FunctionsDiff<UDAggregate> udasDiff = diff.altered.get(0).udas;
+
+        assert udasDiff.created.size() + udasDiff.altered.size() == 1;
+
+        return udasDiff.created.isEmpty()
+             ? ImmutableSet.of()
+             : ImmutableSet.of(FunctionResource.functionFromCql(keyspaceName, aggregateName, rawArgumentTypes));
+    }
+
+    @Override
+    public AuditLogContext getAuditLogContext()
+    {
+        return new AuditLogContext(AuditLogEntryType.CREATE_AGGREGATE, keyspaceName, aggregateName);
+    }
+
+    private String stateFunctionString()
+    {
+        return format("%s(%s)", stateFunctionName, join(", ", transform(concat(singleton(rawStateType), rawArgumentTypes), Object::toString)));
+    }
+
+    private String finalFunctionString()
+    {
+        return format("%s(%s)", finalFunctionName, rawStateType);
+    }
+
+    public static final class Raw extends CQLStatement.Raw
+    {
+        private final FunctionName aggregateName;
+        private final List<CQL3Type.Raw> rawArgumentTypes;
+        private final CQL3Type.Raw rawStateType;
+        private final String stateFunctionName;
+        private final String finalFunctionName;
+        private final Term.Raw rawInitialValue;
+        private final boolean orReplace;
+        private final boolean ifNotExists;
+
+        public Raw(FunctionName aggregateName,
+                   List<CQL3Type.Raw> rawArgumentTypes,
+                   CQL3Type.Raw rawStateType,
+                   String stateFunctionName,
+                   String finalFunctionName,
+                   Term.Raw rawInitialValue,
+                   boolean orReplace,
+                   boolean ifNotExists)
+        {
+            this.aggregateName = aggregateName;
+            this.rawArgumentTypes = rawArgumentTypes;
+            this.rawStateType = rawStateType;
+            this.stateFunctionName = stateFunctionName;
+            this.finalFunctionName = finalFunctionName;
+            this.rawInitialValue = rawInitialValue;
+            this.orReplace = orReplace;
+            this.ifNotExists = ifNotExists;
+        }
+
+        public CreateAggregateStatement prepare(ClientState state)
+        {
+            String keyspaceName = aggregateName.hasKeyspace() ? aggregateName.keyspace : state.getKeyspace();
+
+            return new CreateAggregateStatement(keyspaceName,
+                                                aggregateName.name,
+                                                rawArgumentTypes,
+                                                rawStateType,
+                                                new FunctionName(keyspaceName, stateFunctionName),
+                                                null != finalFunctionName ? new FunctionName(keyspaceName, finalFunctionName) : null,
+                                                rawInitialValue,
+                                                orReplace,
+                                                ifNotExists);
+        }
+    }
+}


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


[10/13] cassandra git commit: Make all DDL statements idempotent and not dependent on global state

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/cql3/statements/CreateKeyspaceStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/CreateKeyspaceStatement.java b/src/java/org/apache/cassandra/cql3/statements/CreateKeyspaceStatement.java
deleted file mode 100644
index b452d16..0000000
--- a/src/java/org/apache/cassandra/cql3/statements/CreateKeyspaceStatement.java
+++ /dev/null
@@ -1,152 +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.statements;
-
-import java.util.regex.Pattern;
-
-import org.apache.cassandra.audit.AuditLogEntryType;
-import org.apache.cassandra.auth.*;
-import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.exceptions.*;
-import org.apache.cassandra.locator.LocalStrategy;
-import org.apache.cassandra.schema.KeyspaceMetadata;
-import org.apache.cassandra.schema.KeyspaceParams;
-import org.apache.cassandra.schema.MigrationManager;
-import org.apache.cassandra.schema.Schema;
-import org.apache.cassandra.schema.SchemaConstants;
-import org.apache.cassandra.service.*;
-import org.apache.cassandra.transport.Event;
-import org.apache.commons.lang3.builder.ToStringBuilder;
-import org.apache.commons.lang3.builder.ToStringStyle;
-
-/** A <code>CREATE KEYSPACE</code> statement parsed from a CQL query. */
-public class CreateKeyspaceStatement extends SchemaAlteringStatement
-{
-    private static final Pattern PATTERN_WORD_CHARS = Pattern.compile("\\w+");
-
-    private final String name;
-    private final KeyspaceAttributes attrs;
-    private final boolean ifNotExists;
-
-    /**
-     * Creates a new <code>CreateKeyspaceStatement</code> instance for a given
-     * keyspace name and keyword arguments.
-     *
-     * @param name the name of the keyspace to create
-     * @param attrs map of the raw keyword arguments that followed the <code>WITH</code> keyword.
-     */
-    public CreateKeyspaceStatement(String name, KeyspaceAttributes attrs, boolean ifNotExists)
-    {
-        super();
-        this.name = name;
-        this.attrs = attrs;
-        this.ifNotExists = ifNotExists;
-    }
-
-    @Override
-    public String keyspace()
-    {
-        return name;
-    }
-
-    public void checkAccess(ClientState state) throws UnauthorizedException
-    {
-        state.hasAllKeyspacesAccess(Permission.CREATE);
-    }
-
-    /**
-     * The <code>CqlParser</code> only goes as far as extracting the keyword arguments
-     * from these statements, so this method is responsible for processing and
-     * validating.
-     *
-     * @throws InvalidRequestException if arguments are missing or unacceptable
-     */
-    public void validate(ClientState state) throws RequestValidationException
-    {
-        Schema.validateKeyspaceNotSystem(name);
-
-        // keyspace name
-        if (!PATTERN_WORD_CHARS.matcher(name).matches())
-            throw new InvalidRequestException(String.format("\"%s\" is not a valid keyspace name", name));
-        if (name.length() > SchemaConstants.NAME_LENGTH)
-            throw new InvalidRequestException(String.format("Keyspace names shouldn't be more than %s characters long (got \"%s\")", SchemaConstants.NAME_LENGTH, name));
-
-        attrs.validate();
-
-        if (attrs.getReplicationStrategyClass() == null)
-            throw new ConfigurationException("Missing mandatory replication strategy class");
-
-        // The strategy is validated through KSMetaData.validate() in announceNewKeyspace below.
-        // However, for backward compatibility with thrift, this doesn't validate unexpected options yet,
-        // so doing proper validation here.
-        KeyspaceParams params = attrs.asNewKeyspaceParams();
-        params.validate(name);
-        if (params.replication.klass.equals(LocalStrategy.class))
-            throw new ConfigurationException("Unable to use given strategy class: LocalStrategy is reserved for internal use.");
-    }
-
-    public Event.SchemaChange announceMigration(QueryState queryState, boolean isLocalOnly) throws RequestValidationException
-    {
-        KeyspaceMetadata ksm = KeyspaceMetadata.create(name, attrs.asNewKeyspaceParams());
-        try
-        {
-            MigrationManager.announceNewKeyspace(ksm, isLocalOnly);
-            return new Event.SchemaChange(Event.SchemaChange.Change.CREATED, keyspace());
-        }
-        catch (AlreadyExistsException e)
-        {
-            if (ifNotExists)
-                return null;
-            throw e;
-        }
-    }
-
-    protected void grantPermissionsToCreator(QueryState state)
-    {
-        try
-        {
-            RoleResource role = RoleResource.role(state.getClientState().getUser().getName());
-            DataResource keyspace = DataResource.keyspace(keyspace());
-            DatabaseDescriptor.getAuthorizer().grant(AuthenticatedUser.SYSTEM_USER,
-                                                     keyspace.applicablePermissions(),
-                                                     keyspace,
-                                                     role);
-            FunctionResource functions = FunctionResource.keyspace(keyspace());
-            DatabaseDescriptor.getAuthorizer().grant(AuthenticatedUser.SYSTEM_USER,
-                                                     functions.applicablePermissions(),
-                                                     functions,
-                                                     role);
-        }
-        catch (RequestExecutionException e)
-        {
-            throw new RuntimeException(e);
-        }
-    }
-    
-    @Override
-    public String toString()
-    {
-        return ToStringBuilder.reflectionToString(this, ToStringStyle.SHORT_PREFIX_STYLE);
-    }
-
-    @Override
-    public AuditLogContext getAuditLogContext()
-    {
-        return new AuditLogContext(AuditLogEntryType.CREATE_KEYSPACE, keyspace());
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/cql3/statements/CreateRoleStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/CreateRoleStatement.java b/src/java/org/apache/cassandra/cql3/statements/CreateRoleStatement.java
index f12d7e6..574d661 100644
--- a/src/java/org/apache/cassandra/cql3/statements/CreateRoleStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/CreateRoleStatement.java
@@ -17,6 +17,7 @@
  */
 package org.apache.cassandra.cql3.statements;
 
+import org.apache.cassandra.audit.AuditLogContext;
 import org.apache.cassandra.audit.AuditLogEntryType;
 import org.apache.cassandra.auth.*;
 import org.apache.cassandra.config.DatabaseDescriptor;
@@ -42,7 +43,7 @@ public class CreateRoleStatement extends AuthenticationStatement
         this.ifNotExists = ifNotExists;
     }
 
-    public void checkAccess(ClientState state) throws UnauthorizedException
+    public void authorize(ClientState state) throws UnauthorizedException
     {
         super.checkPermission(state, Permission.CREATE, RoleResource.root());
         if (opts.getSuperuser().isPresent())
@@ -64,7 +65,7 @@ public class CreateRoleStatement extends AuthenticationStatement
         if (role.getRoleName().isEmpty())
             throw new InvalidRequestException("Role name can't be an empty string");
 
-        // validate login here before checkAccess to avoid leaking role existence to anonymous users.
+        // validate login here before authorize to avoid leaking role existence to anonymous users.
         state.ensureNotAnonymous();
 
         if (!ifNotExists && DatabaseDescriptor.getRoleManager().isExistingRole(role))
@@ -88,7 +89,7 @@ public class CreateRoleStatement extends AuthenticationStatement
 
     /**
      * Grant all applicable permissions on the newly created role to the user performing the request
-     * see also: SchemaAlteringStatement#grantPermissionsToCreator and the overridden implementations
+     * see also: AlterTableStatement#createdResources() and the overridden implementations
      * of it in subclasses CreateKeyspaceStatement & CreateTableStatement.
      * @param state
      */

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/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
deleted file mode 100644
index 7c639e2..0000000
--- a/src/java/org/apache/cassandra/cql3/statements/CreateTableStatement.java
+++ /dev/null
@@ -1,431 +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.statements;
-
-import java.nio.ByteBuffer;
-import java.util.*;
-import java.util.regex.Pattern;
-
-import com.google.common.collect.HashMultiset;
-import com.google.common.collect.Multiset;
-import org.apache.commons.lang3.StringUtils;
-
-import org.apache.cassandra.audit.AuditLogEntryType;
-import org.apache.cassandra.auth.*;
-import org.apache.cassandra.config.*;
-import org.apache.cassandra.cql3.*;
-import org.apache.cassandra.db.*;
-import org.apache.cassandra.db.marshal.*;
-import org.apache.cassandra.exceptions.*;
-import org.apache.cassandra.schema.*;
-import org.apache.cassandra.service.ClientState;
-import org.apache.cassandra.service.QueryState;
-import org.apache.cassandra.transport.Event;
-import org.apache.commons.lang3.builder.ToStringBuilder;
-import org.apache.commons.lang3.builder.ToStringStyle;
-
-/** A {@code CREATE TABLE} parsed from a CQL query statement. */
-public class CreateTableStatement extends SchemaAlteringStatement
-{
-    private static final Pattern PATTERN_WORD_CHARS = Pattern.compile("\\w+");
-
-    private List<AbstractType<?>> keyTypes;
-    private List<AbstractType<?>> clusteringTypes;
-
-    private final Map<ByteBuffer, AbstractType> multicellColumns = new HashMap<>();
-
-    private final List<ColumnIdentifier> keyAliases = new ArrayList<>();
-    private final List<ColumnIdentifier> columnAliases = new ArrayList<>();
-
-    private boolean isDense;
-    private boolean isCompound;
-    private boolean hasCounters;
-
-    // use a TreeMap to preserve ordering across JDK versions (see CASSANDRA-9492)
-    private final Map<ColumnIdentifier, AbstractType> columns = new TreeMap<>((o1, o2) -> o1.bytes.compareTo(o2.bytes));
-
-    private final Set<ColumnIdentifier> staticColumns;
-    private final TableParams params;
-    private final boolean ifNotExists;
-    private final TableId id;
-
-    public CreateTableStatement(CFName name, TableParams params, boolean ifNotExists, Set<ColumnIdentifier> staticColumns, TableId id)
-    {
-        super(name);
-        this.params = params;
-        this.ifNotExists = ifNotExists;
-        this.staticColumns = staticColumns;
-        this.id = id;
-    }
-
-    public void checkAccess(ClientState state) throws UnauthorizedException, InvalidRequestException
-    {
-        state.hasKeyspaceAccess(keyspace(), Permission.CREATE);
-    }
-
-    public void validate(ClientState state)
-    {
-        // validated in announceMigration()
-    }
-
-    public Event.SchemaChange announceMigration(QueryState queryState, boolean isLocalOnly) throws RequestValidationException
-    {
-        try
-        {
-            MigrationManager.announceNewTable(toTableMetadata(), isLocalOnly);
-            return new Event.SchemaChange(Event.SchemaChange.Change.CREATED, Event.SchemaChange.Target.TABLE, keyspace(), columnFamily());
-        }
-        catch (AlreadyExistsException e)
-        {
-            if (ifNotExists)
-                return null;
-            throw e;
-        }
-    }
-
-    protected void grantPermissionsToCreator(QueryState state)
-    {
-        try
-        {
-            IResource resource = DataResource.table(keyspace(), columnFamily());
-            DatabaseDescriptor.getAuthorizer().grant(AuthenticatedUser.SYSTEM_USER,
-                                                     resource.applicablePermissions(),
-                                                     resource,
-                                                     RoleResource.role(state.getClientState().getUser().getName()));
-        }
-        catch (RequestExecutionException e)
-        {
-            throw new RuntimeException(e);
-        }
-    }
-
-    /**
-     */
-    public static TableMetadata.Builder parse(String cql, String keyspace)
-    {
-        CreateTableStatement.RawStatement raw = CQLFragmentParser.parseAny(CqlParser::createTableStatement, cql, "CREATE TABLE");
-        raw.prepareKeyspace(keyspace);
-        CreateTableStatement prepared = (CreateTableStatement) raw.prepare(Types.none()).statement;
-        return prepared.builder();
-    }
-
-    public TableMetadata.Builder builder()
-    {
-        TableMetadata.Builder builder = TableMetadata.builder(keyspace(), columnFamily());
-
-        if (id != null)
-            builder.id(id);
-
-        builder.isDense(isDense)
-               .isCompound(isCompound)
-               .isCounter(hasCounters)
-               .isSuper(false)
-               .params(params);
-
-        for (int i = 0; i < keyAliases.size(); i++)
-            builder.addPartitionKeyColumn(keyAliases.get(i), keyTypes.get(i));
-
-        for (int i = 0; i < columnAliases.size(); i++)
-            builder.addClusteringColumn(columnAliases.get(i), clusteringTypes.get(i));
-
-        boolean isStaticCompact = !isDense && !isCompound;
-        for (Map.Entry<ColumnIdentifier, AbstractType> entry : columns.entrySet())
-        {
-            ColumnIdentifier name = entry.getKey();
-            // Note that for "static" no-clustering compact storage we use static for the defined columns
-            if (staticColumns.contains(name) || isStaticCompact)
-                builder.addStaticColumn(name, entry.getValue());
-            else
-                builder.addRegularColumn(name, entry.getValue());
-        }
-
-        boolean isCompactTable = isDense || !isCompound;
-        if (isCompactTable)
-        {
-            CompactTables.DefaultNames names = CompactTables.defaultNameGenerator(builder.columnNames());
-            // Compact tables always have a clustering and a single regular value.
-            if (isStaticCompact)
-            {
-                builder.addClusteringColumn(names.defaultClusteringName(), UTF8Type.instance);
-                builder.addRegularColumn(names.defaultCompactValueName(), hasCounters ? CounterColumnType.instance : BytesType.instance);
-            }
-            else if (isDense && !builder.hasRegularColumns())
-            {
-                // Even for dense, we might not have our regular column if it wasn't part of the declaration. If
-                // that's the case, add it but with a specific EmptyType so we can recognize that case later
-                builder.addRegularColumn(names.defaultCompactValueName(), EmptyType.instance);
-            }
-        }
-
-        return builder;
-    }
-
-    /**
-     * Returns a TableMetadata instance based on the parameters parsed from this
-     * {@code CREATE} statement, or defaults where applicable.
-     *
-     * @return a TableMetadata instance corresponding to the values parsed from this statement
-     */
-    public TableMetadata toTableMetadata()
-    {
-        return builder().build();
-    }
-
-    public static class RawStatement extends CFStatement
-    {
-        private final Map<ColumnIdentifier, CQL3Type.Raw> definitions = new HashMap<>();
-        public final CFProperties properties = new CFProperties();
-
-        private final List<List<ColumnIdentifier>> keyAliases = new ArrayList<>();
-        private final List<ColumnIdentifier> columnAliases = new ArrayList<>();
-        private final Set<ColumnIdentifier> staticColumns = new HashSet<>();
-
-        private final Multiset<ColumnIdentifier> definedNames = HashMultiset.create(1);
-
-        private final boolean ifNotExists;
-
-        public RawStatement(CFName name, boolean ifNotExists)
-        {
-            super(name);
-            this.ifNotExists = ifNotExists;
-        }
-
-        /**
-         * Transform this raw statement into a CreateTableStatement.
-         */
-        public ParsedStatement.Prepared prepare() throws RequestValidationException
-        {
-            KeyspaceMetadata ksm = Schema.instance.getKeyspaceMetadata(keyspace());
-            if (ksm == null)
-                throw new ConfigurationException(String.format("Keyspace %s doesn't exist", keyspace()));
-            if (ksm.isVirtual())
-                throw new InvalidRequestException("Cannot create tables in virtual keyspaces");
-
-            return prepare(ksm.types);
-        }
-
-        public ParsedStatement.Prepared prepare(Types udts) throws RequestValidationException
-        {
-            // Column family name
-            if (!PATTERN_WORD_CHARS.matcher(columnFamily()).matches())
-                throw new InvalidRequestException(String.format("\"%s\" is not a valid table name (must be alphanumeric character or underscore only: [a-zA-Z_0-9]+)", columnFamily()));
-            if (columnFamily().length() > SchemaConstants.NAME_LENGTH)
-                throw new InvalidRequestException(String.format("Table names shouldn't be more than %s characters long (got \"%s\")", SchemaConstants.NAME_LENGTH, columnFamily()));
-
-            for (Multiset.Entry<ColumnIdentifier> entry : definedNames.entrySet())
-                if (entry.getCount() > 1)
-                    throw new InvalidRequestException(String.format("Multiple definition of identifier %s", entry.getElement()));
-
-            properties.validate();
-
-            TableParams params = properties.properties.asNewTableParams();
-
-            CreateTableStatement stmt = new CreateTableStatement(cfName, params, ifNotExists, staticColumns, properties.properties.getId());
-
-            for (Map.Entry<ColumnIdentifier, CQL3Type.Raw> entry : definitions.entrySet())
-            {
-                ColumnIdentifier id = entry.getKey();
-                CQL3Type pt = entry.getValue().prepare(keyspace(), udts);
-                if (pt.getType().isMultiCell())
-                    stmt.multicellColumns.put(id.bytes, pt.getType());
-                if (entry.getValue().isCounter())
-                    stmt.hasCounters = true;
-
-                // check for non-frozen UDTs or collections in a non-frozen UDT
-                if (pt.getType().isUDT() && pt.getType().isMultiCell())
-                {
-                    for (AbstractType<?> innerType : ((UserType) pt.getType()).fieldTypes())
-                    {
-                        if (innerType.isMultiCell())
-                        {
-                            assert innerType.isCollection();  // shouldn't get this far with a nested non-frozen UDT
-                            throw new InvalidRequestException("Non-frozen UDTs with nested non-frozen collections are not supported");
-                        }
-                    }
-                }
-
-                stmt.columns.put(id, pt.getType()); // we'll remove what is not a column below
-            }
-
-            if (keyAliases.isEmpty())
-                throw new InvalidRequestException("No PRIMARY KEY specifed (exactly one required)");
-            if (keyAliases.size() > 1)
-                throw new InvalidRequestException("Multiple PRIMARY KEYs specifed (exactly one required)");
-            if (stmt.hasCounters && params.defaultTimeToLive > 0)
-                throw new InvalidRequestException("Cannot set default_time_to_live on a table with counters");
-
-            List<ColumnIdentifier> kAliases = keyAliases.get(0);
-            stmt.keyTypes = new ArrayList<>(kAliases.size());
-            for (ColumnIdentifier alias : kAliases)
-            {
-                stmt.keyAliases.add(alias);
-                AbstractType<?> t = getTypeAndRemove(stmt.columns, alias);
-                if (t.asCQL3Type().getType() instanceof CounterColumnType)
-                    throw new InvalidRequestException(String.format("counter type is not supported for PRIMARY KEY part %s", alias));
-                if (t.asCQL3Type().getType().referencesDuration())
-                    throw new InvalidRequestException(String.format("duration type is not supported for PRIMARY KEY part %s", alias));
-                if (staticColumns.contains(alias))
-                    throw new InvalidRequestException(String.format("Static column %s cannot be part of the PRIMARY KEY", alias));
-                stmt.keyTypes.add(t);
-            }
-
-            stmt.clusteringTypes = new ArrayList<>(columnAliases.size());
-            // Handle column aliases
-            for (ColumnIdentifier t : columnAliases)
-            {
-                stmt.columnAliases.add(t);
-
-                AbstractType<?> type = getTypeAndRemove(stmt.columns, t);
-                if (type.asCQL3Type().getType() instanceof CounterColumnType)
-                    throw new InvalidRequestException(String.format("counter type is not supported for PRIMARY KEY part %s", t));
-                if (type.asCQL3Type().getType().referencesDuration())
-                    throw new InvalidRequestException(String.format("duration type is not supported for PRIMARY KEY part %s", t));
-                if (staticColumns.contains(t))
-                    throw new InvalidRequestException(String.format("Static column %s cannot be part of the PRIMARY KEY", t));
-                stmt.clusteringTypes.add(type);
-            }
-
-            // We've handled anything that is not a rpimary key so stmt.columns only contains NON-PK columns. So
-            // if it's a counter table, make sure we don't have non-counter types
-            if (stmt.hasCounters)
-            {
-                for (AbstractType<?> type : stmt.columns.values())
-                    if (!type.isCounter())
-                        throw new InvalidRequestException("Cannot mix counter and non counter columns in the same table");
-            }
-
-            boolean useCompactStorage = properties.useCompactStorage;
-            // Dense meant, back with thrift, that no part of the "thrift column name" stores a "CQL/metadata column name".
-            // This means COMPACT STORAGE with at least one clustering type (otherwise it's a "static" CF).
-            stmt.isDense = useCompactStorage && !stmt.clusteringTypes.isEmpty();
-            // Compound meant the "thrift column name" was a composite one. It's the case unless
-            // we use compact storage COMPACT STORAGE and we have either no clustering columns ("static" CF) or
-            // only one of them (if more than one, it's a "dense composite").
-            stmt.isCompound = !(useCompactStorage && stmt.clusteringTypes.size() <= 1);
-
-            // For COMPACT STORAGE, we reject any "feature" that we wouldn't be able to translate back to thrift.
-            if (useCompactStorage)
-            {
-                if (!stmt.multicellColumns.isEmpty())
-                    throw new InvalidRequestException("Non-frozen collections and UDTs are not supported with COMPACT STORAGE");
-                if (!staticColumns.isEmpty())
-                    throw new InvalidRequestException("Static columns are not supported in COMPACT STORAGE tables");
-
-                if (stmt.clusteringTypes.isEmpty())
-                {
-                    // It's a thrift "static CF" so there should be some columns definition
-                    if (stmt.columns.isEmpty())
-                        throw new InvalidRequestException("No definition found that is not part of the PRIMARY KEY");
-                }
-
-                if (stmt.isDense)
-                {
-                    // We can have no columns (only the PK), but we can't have more than one.
-                    if (stmt.columns.size() > 1)
-                        throw new InvalidRequestException(String.format("COMPACT STORAGE with composite PRIMARY KEY allows no more than one column not part of the PRIMARY KEY (got: %s)", StringUtils.join(stmt.columns.keySet(), ", ")));
-                }
-                else
-                {
-                    // we are in the "static" case, so we need at least one column defined. For non-compact however, having
-                    // just the PK is fine.
-                    if (stmt.columns.isEmpty())
-                        throw new InvalidRequestException("COMPACT STORAGE with non-composite PRIMARY KEY require one column not part of the PRIMARY KEY, none given");
-                }
-            }
-            else
-            {
-                if (stmt.clusteringTypes.isEmpty() && !staticColumns.isEmpty())
-                {
-                    // Static columns only make sense if we have at least one clustering column. Otherwise everything is static anyway
-                    if (columnAliases.isEmpty())
-                        throw new InvalidRequestException("Static columns are only useful (and thus allowed) if the table has at least one clustering column");
-                }
-            }
-
-            // If we give a clustering order, we must explicitly do so for all aliases and in the order of the PK
-            if (!properties.definedOrdering.isEmpty())
-            {
-                if (properties.definedOrdering.size() > columnAliases.size())
-                    throw new InvalidRequestException("Only clustering key columns can be defined in CLUSTERING ORDER directive");
-
-                int i = 0;
-                for (ColumnIdentifier id : properties.definedOrdering.keySet())
-                {
-                    ColumnIdentifier c = columnAliases.get(i);
-                    if (!id.equals(c))
-                    {
-                        if (properties.definedOrdering.containsKey(c))
-                            throw new InvalidRequestException(String.format("The order of columns in the CLUSTERING ORDER directive must be the one of the clustering key (%s must appear before %s)", c, id));
-                        else
-                            throw new InvalidRequestException(String.format("Missing CLUSTERING ORDER for column %s", c));
-                    }
-                    ++i;
-                }
-            }
-
-            return new ParsedStatement.Prepared(stmt);
-        }
-
-        private AbstractType<?> getTypeAndRemove(Map<ColumnIdentifier, AbstractType> columns, ColumnIdentifier t) throws InvalidRequestException
-        {
-            AbstractType type = columns.get(t);
-            if (type == null)
-                throw new InvalidRequestException(String.format("Unknown definition %s referenced in PRIMARY KEY", t));
-            if (type.isMultiCell())
-            {
-                if (type.isCollection())
-                    throw new InvalidRequestException(String.format("Invalid non-frozen collection type for PRIMARY KEY component %s", t));
-                else
-                    throw new InvalidRequestException(String.format("Invalid non-frozen user-defined type for PRIMARY KEY component %s", t));
-            }
-
-            columns.remove(t);
-            Boolean isReversed = properties.definedOrdering.get(t);
-            return isReversed != null && isReversed ? ReversedType.getInstance(type) : type;
-        }
-
-        public void addDefinition(ColumnIdentifier def, CQL3Type.Raw type, boolean isStatic)
-        {
-            definedNames.add(def);
-            definitions.put(def, type);
-            if (isStatic)
-                staticColumns.add(def);
-        }
-
-        public void addKeyAliases(List<ColumnIdentifier> aliases)
-        {
-            keyAliases.add(aliases);
-        }
-
-        public void addColumnAlias(ColumnIdentifier alias)
-        {
-            columnAliases.add(alias);
-        }
-    }
-    
-    @Override
-    public String toString()
-    {
-        return ToStringBuilder.reflectionToString(this, ToStringStyle.SHORT_PREFIX_STYLE);
-    }
-    @Override
-    public AuditLogContext getAuditLogContext()
-    {
-        return new AuditLogContext(AuditLogEntryType.CREATE_TABLE, keyspace(), cfName.getColumnFamily());
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/cql3/statements/CreateTriggerStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/CreateTriggerStatement.java b/src/java/org/apache/cassandra/cql3/statements/CreateTriggerStatement.java
deleted file mode 100644
index f2cd217..0000000
--- a/src/java/org/apache/cassandra/cql3/statements/CreateTriggerStatement.java
+++ /dev/null
@@ -1,116 +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.statements;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.cassandra.audit.AuditLogEntryType;
-import org.apache.cassandra.cql3.CFName;
-import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.exceptions.InvalidRequestException;
-import org.apache.cassandra.exceptions.RequestValidationException;
-import org.apache.cassandra.exceptions.UnauthorizedException;
-import org.apache.cassandra.schema.MigrationManager;
-import org.apache.cassandra.schema.Schema;
-import org.apache.cassandra.schema.TableMetadata;
-import org.apache.cassandra.schema.TriggerMetadata;
-import org.apache.cassandra.schema.Triggers;
-import org.apache.cassandra.service.ClientState;
-import org.apache.cassandra.service.QueryState;
-import org.apache.cassandra.transport.Event;
-import org.apache.cassandra.triggers.TriggerExecutor;
-import org.apache.commons.lang3.builder.ToStringBuilder;
-import org.apache.commons.lang3.builder.ToStringStyle;
-
-public class CreateTriggerStatement extends SchemaAlteringStatement
-{
-    private static final Logger logger = LoggerFactory.getLogger(CreateTriggerStatement.class);
-
-    private final String triggerName;
-    private final String triggerClass;
-    private final boolean ifNotExists;
-
-    public CreateTriggerStatement(CFName name, String triggerName, String clazz, boolean ifNotExists)
-    {
-        super(name);
-        this.triggerName = triggerName;
-        this.triggerClass = clazz;
-        this.ifNotExists = ifNotExists;
-    }
-
-    public void checkAccess(ClientState state) throws UnauthorizedException
-    {
-        state.ensureIsSuper("Only superusers are allowed to perform CREATE TRIGGER queries");
-    }
-
-    public void validate(ClientState state) throws RequestValidationException
-    {
-        TableMetadata metadata = Schema.instance.validateTable(keyspace(), columnFamily());
-        if (metadata.isVirtual())
-            throw new InvalidRequestException("Cannot CREATE TRIGGER against a virtual table");
-        if (metadata.isView())
-            throw new InvalidRequestException("Cannot CREATE TRIGGER against a materialized view");
-
-        try
-        {
-            TriggerExecutor.instance.loadTriggerInstance(triggerClass);
-        }
-        catch (Exception e)
-        {
-            throw new ConfigurationException(String.format("Trigger class '%s' doesn't exist", triggerClass));
-        }
-    }
-
-    public Event.SchemaChange announceMigration(QueryState queryState, boolean isLocalOnly) throws ConfigurationException, InvalidRequestException
-    {
-        TableMetadata current = Schema.instance.getTableMetadata(keyspace(), columnFamily());
-        Triggers triggers = current.triggers;
-
-        if (triggers.get(triggerName).isPresent())
-        {
-            if (ifNotExists)
-                return null;
-            else
-                throw new InvalidRequestException(String.format("Trigger %s already exists", triggerName));
-        }
-
-        TableMetadata updated =
-            current.unbuild()
-                   .triggers(triggers.with(TriggerMetadata.create(triggerName, triggerClass)))
-                   .build();
-
-        logger.info("Adding trigger with name {} and class {}", triggerName, triggerClass);
-
-        MigrationManager.announceTableUpdate(updated, isLocalOnly);
-        return new Event.SchemaChange(Event.SchemaChange.Change.UPDATED, Event.SchemaChange.Target.TABLE, keyspace(), columnFamily());
-    }
-    
-    @Override
-    public String toString()
-    {
-        return ToStringBuilder.reflectionToString(this, ToStringStyle.SHORT_PREFIX_STYLE);
-    }
-
-    @Override
-    public AuditLogContext getAuditLogContext()
-    {
-        return new AuditLogContext(AuditLogEntryType.CREATE_TRIGGER, keyspace(), triggerName);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/cql3/statements/CreateTypeStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/CreateTypeStatement.java b/src/java/org/apache/cassandra/cql3/statements/CreateTypeStatement.java
deleted file mode 100644
index 1a0da4c..0000000
--- a/src/java/org/apache/cassandra/cql3/statements/CreateTypeStatement.java
+++ /dev/null
@@ -1,152 +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.statements;
-
-import java.util.*;
-import java.util.stream.Collectors;
-
-import org.apache.cassandra.audit.AuditLogEntryType;
-import org.apache.cassandra.auth.Permission;
-import org.apache.cassandra.cql3.*;
-import org.apache.cassandra.db.marshal.AbstractType;
-import org.apache.cassandra.db.marshal.UserType;
-import org.apache.cassandra.exceptions.*;
-import org.apache.cassandra.schema.KeyspaceMetadata;
-import org.apache.cassandra.schema.MigrationManager;
-import org.apache.cassandra.schema.Schema;
-import org.apache.cassandra.schema.Types;
-import org.apache.cassandra.service.ClientState;
-import org.apache.cassandra.service.QueryState;
-import org.apache.cassandra.transport.Event;
-import org.apache.commons.lang3.builder.ToStringBuilder;
-import org.apache.commons.lang3.builder.ToStringStyle;
-
-public class CreateTypeStatement extends SchemaAlteringStatement
-{
-    private final UTName name;
-    private final List<FieldIdentifier> columnNames = new ArrayList<>();
-    private final List<CQL3Type.Raw> columnTypes = new ArrayList<>();
-    private final boolean ifNotExists;
-
-    public CreateTypeStatement(UTName name, boolean ifNotExists)
-    {
-        super();
-        this.name = name;
-        this.ifNotExists = ifNotExists;
-    }
-
-    @Override
-    public void prepareKeyspace(ClientState state) throws InvalidRequestException
-    {
-        if (!name.hasKeyspace())
-            name.setKeyspace(state.getKeyspace());
-    }
-
-    public void addDefinition(FieldIdentifier name, CQL3Type.Raw type)
-    {
-        columnNames.add(name);
-        columnTypes.add(type);
-    }
-
-    public void checkAccess(ClientState state) throws UnauthorizedException, InvalidRequestException
-    {
-        state.hasKeyspaceAccess(keyspace(), Permission.CREATE);
-    }
-
-    public void validate(ClientState state) throws RequestValidationException
-    {
-        KeyspaceMetadata ksm = Schema.instance.getKeyspaceMetadata(name.getKeyspace());
-        if (ksm == null)
-            throw new InvalidRequestException(String.format("Cannot add type in unknown keyspace %s", name.getKeyspace()));
-        if (ksm.isVirtual())
-            throw new InvalidRequestException("Cannot create types in virtual keyspaces");
-
-        if (ksm.types.get(name.getUserTypeName()).isPresent() && !ifNotExists)
-            throw new InvalidRequestException(String.format("A user type of name %s already exists", name));
-
-        for (CQL3Type.Raw type : columnTypes)
-        {
-            if (type.isCounter())
-                throw new InvalidRequestException("A user type cannot contain counters");
-            if (type.isUDT() && !type.isFrozen())
-                throw new InvalidRequestException("A user type cannot contain non-frozen UDTs");
-        }
-    }
-
-    public static void checkForDuplicateNames(UserType type) throws InvalidRequestException
-    {
-        for (int i = 0; i < type.size() - 1; i++)
-        {
-            FieldIdentifier fieldName = type.fieldName(i);
-            for (int j = i+1; j < type.size(); j++)
-            {
-                if (fieldName.equals(type.fieldName(j)))
-                    throw new InvalidRequestException(String.format("Duplicate field name %s in type %s", fieldName, type.name));
-            }
-        }
-    }
-
-    public void addToRawBuilder(Types.RawBuilder builder) throws InvalidRequestException
-    {
-        builder.add(name.getStringTypeName(),
-                    columnNames.stream().map(FieldIdentifier::toString).collect(Collectors.toList()),
-                    columnTypes.stream().map(CQL3Type.Raw::toString).collect(Collectors.toList()));
-    }
-
-    @Override
-    public String keyspace()
-    {
-        return name.getKeyspace();
-    }
-
-    public UserType createType() throws InvalidRequestException
-    {
-        List<AbstractType<?>> types = new ArrayList<>(columnTypes.size());
-        for (CQL3Type.Raw type : columnTypes)
-            types.add(type.prepare(keyspace()).getType());
-
-        return new UserType(name.getKeyspace(), name.getUserTypeName(), columnNames, types, true);
-    }
-
-    public Event.SchemaChange announceMigration(QueryState queryState, boolean isLocalOnly) throws InvalidRequestException, ConfigurationException
-    {
-        KeyspaceMetadata ksm = Schema.instance.getKeyspaceMetadata(name.getKeyspace());
-        assert ksm != null; // should haven't validate otherwise
-
-        // Can happen with ifNotExists
-        if (ksm.types.get(name.getUserTypeName()).isPresent())
-            return null;
-
-        UserType type = createType();
-        checkForDuplicateNames(type);
-        MigrationManager.announceNewType(type, isLocalOnly);
-        return new Event.SchemaChange(Event.SchemaChange.Change.CREATED, Event.SchemaChange.Target.TYPE, keyspace(), name.getStringTypeName());
-    }
-    
-    @Override
-    public String toString()
-    {
-        return ToStringBuilder.reflectionToString(this, ToStringStyle.SHORT_PREFIX_STYLE);
-    }
-
-    @Override
-    public AuditLogContext getAuditLogContext()
-    {
-        return new AuditLogContext(AuditLogEntryType.CREATE_TYPE, keyspace(), name.getStringTypeName());
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/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
deleted file mode 100644
index b50a552..0000000
--- a/src/java/org/apache/cassandra/cql3/statements/CreateViewStatement.java
+++ /dev/null
@@ -1,390 +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.statements;
-
-import java.util.*;
-import java.util.stream.Collectors;
-
-import com.google.common.collect.Iterables;
-import com.google.common.collect.Sets;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.cassandra.audit.AuditLogEntryType;
-import org.apache.cassandra.auth.Permission;
-import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.cql3.*;
-import org.apache.cassandra.cql3.restrictions.StatementRestrictions;
-import org.apache.cassandra.cql3.selection.RawSelector;
-import org.apache.cassandra.cql3.selection.Selectable;
-import org.apache.cassandra.db.marshal.AbstractType;
-import org.apache.cassandra.db.marshal.DurationType;
-import org.apache.cassandra.db.marshal.ReversedType;
-import org.apache.cassandra.db.view.View;
-import org.apache.cassandra.exceptions.AlreadyExistsException;
-import org.apache.cassandra.exceptions.InvalidRequestException;
-import org.apache.cassandra.exceptions.RequestValidationException;
-import org.apache.cassandra.exceptions.UnauthorizedException;
-import org.apache.cassandra.schema.ColumnMetadata;
-import org.apache.cassandra.schema.MigrationManager;
-import org.apache.cassandra.schema.Schema;
-import org.apache.cassandra.schema.TableMetadata;
-import org.apache.cassandra.schema.TableParams;
-import org.apache.cassandra.schema.ViewMetadata;
-import org.apache.cassandra.service.ClientState;
-import org.apache.cassandra.service.ClientWarn;
-import org.apache.cassandra.service.QueryState;
-import org.apache.cassandra.transport.Event;
-import org.apache.commons.lang3.builder.ToStringBuilder;
-import org.apache.commons.lang3.builder.ToStringStyle;
-
-public class CreateViewStatement extends SchemaAlteringStatement
-{
-    private static final Logger logger = LoggerFactory.getLogger(CreateViewStatement.class);
-
-    private final CFName baseName;
-    private final List<RawSelector> selectClause;
-    private final WhereClause whereClause;
-    private final List<ColumnMetadata.Raw> partitionKeys;
-    private final List<ColumnMetadata.Raw> clusteringKeys;
-    public final CFProperties properties = new CFProperties();
-    private final boolean ifNotExists;
-
-    public CreateViewStatement(CFName viewName,
-                               CFName baseName,
-                               List<RawSelector> selectClause,
-                               WhereClause whereClause,
-                               List<ColumnMetadata.Raw> partitionKeys,
-                               List<ColumnMetadata.Raw> clusteringKeys,
-                               boolean ifNotExists)
-    {
-        super(viewName);
-        this.baseName = baseName;
-        this.selectClause = selectClause;
-        this.whereClause = whereClause;
-        this.partitionKeys = partitionKeys;
-        this.clusteringKeys = clusteringKeys;
-        this.ifNotExists = ifNotExists;
-    }
-
-
-    public void checkAccess(ClientState state) throws UnauthorizedException, InvalidRequestException
-    {
-        if (!baseName.hasKeyspace())
-            baseName.setKeyspace(keyspace(), true);
-        state.hasColumnFamilyAccess(keyspace(), baseName.getColumnFamily(), Permission.ALTER);
-    }
-
-    public void validate(ClientState state) throws RequestValidationException
-    {
-        // We do validation in announceMigration to reduce doubling up of work
-    }
-
-    private interface AddColumn
-    {
-        void add(ColumnIdentifier identifier, AbstractType<?> type);
-    }
-
-    private void add(TableMetadata baseCfm, Iterable<ColumnIdentifier> columns, AddColumn adder)
-    {
-        for (ColumnIdentifier column : columns)
-        {
-            AbstractType<?> type = baseCfm.getColumn(column).type;
-            if (properties.definedOrdering.containsKey(column))
-            {
-                boolean desc = properties.definedOrdering.get(column);
-                if (!desc && type.isReversed())
-                {
-                    type = ((ReversedType)type).baseType;
-                }
-                else if (desc && !type.isReversed())
-                {
-                    type = ReversedType.getInstance(type);
-                }
-            }
-            adder.add(column, type);
-        }
-    }
-
-    public Event.SchemaChange announceMigration(QueryState queryState, boolean isLocalOnly) throws RequestValidationException
-    {
-        if (!DatabaseDescriptor.enableMaterializedViews())
-        {
-            throw new InvalidRequestException("Materialized views are disabled. Enable in cassandra.yaml to use.");
-        }
-
-        // We need to make sure that:
-        //  - primary key includes all columns in base table's primary key
-        //  - make sure that the select statement does not have anything other than columns
-        //    and their names match the base table's names
-        //  - make sure that primary key does not include any collections
-        //  - make sure there is no where clause in the select statement
-        //  - make sure there is not currently a table or view
-        //  - make sure baseTable gcGraceSeconds > 0
-
-        properties.validate();
-
-        if (properties.useCompactStorage)
-            throw new InvalidRequestException("Cannot use 'COMPACT STORAGE' when defining a materialized view");
-
-        // We enforce the keyspace because if the RF is different, the logic to wait for a
-        // specific replica would break
-        if (!baseName.getKeyspace().equals(keyspace()))
-            throw new InvalidRequestException("Cannot create a materialized view on a table in a separate keyspace");
-
-        TableMetadata metadata = Schema.instance.validateTable(baseName.getKeyspace(), baseName.getColumnFamily());
-
-        if (metadata.isVirtual())
-            throw new InvalidRequestException("Materialized views are not supported on virtual tables");
-        if (metadata.isCounter())
-            throw new InvalidRequestException("Materialized views are not supported on counter tables");
-        if (metadata.isView())
-            throw new InvalidRequestException("Materialized views cannot be created against other materialized views");
-
-        if (metadata.params.gcGraceSeconds == 0)
-        {
-            throw new InvalidRequestException(String.format("Cannot create materialized view '%s' for base table " +
-                                                            "'%s' with gc_grace_seconds of 0, since this value is " +
-                                                            "used to TTL undelivered updates. Setting gc_grace_seconds" +
-                                                            " too low might cause undelivered updates to expire " +
-                                                            "before being replayed.", cfName.getColumnFamily(),
-                                                            baseName.getColumnFamily()));
-        }
-
-        Set<ColumnIdentifier> included = Sets.newHashSetWithExpectedSize(selectClause.size());
-        for (RawSelector selector : selectClause)
-        {
-            Selectable.Raw selectable = selector.selectable;
-            if (selectable instanceof Selectable.WithFieldSelection.Raw)
-                throw new InvalidRequestException("Cannot select out a part of type when defining a materialized view");
-            if (selectable instanceof Selectable.WithFunction.Raw)
-                throw new InvalidRequestException("Cannot use function when defining a materialized view");
-            if (selectable instanceof Selectable.WritetimeOrTTL.Raw)
-                throw new InvalidRequestException("Cannot use function when defining a materialized view");
-            if (selectable instanceof Selectable.WithElementSelection.Raw)
-                throw new InvalidRequestException("Cannot use collection element selection when defining a materialized view");
-            if (selectable instanceof Selectable.WithSliceSelection.Raw)
-                throw new InvalidRequestException("Cannot use collection slice selection when defining a materialized view");
-            if (selector.alias != null)
-                throw new InvalidRequestException("Cannot use alias when defining a materialized view");
-
-            Selectable s = selectable.prepare(metadata);
-            if (s instanceof Term.Raw)
-                throw new InvalidRequestException("Cannot use terms in selection when defining a materialized view");
-
-            ColumnMetadata cdef = (ColumnMetadata)s;
-            included.add(cdef.name);
-        }
-
-        Set<ColumnMetadata.Raw> targetPrimaryKeys = new HashSet<>();
-        for (ColumnMetadata.Raw identifier : Iterables.concat(partitionKeys, clusteringKeys))
-        {
-            if (!targetPrimaryKeys.add(identifier))
-                throw new InvalidRequestException("Duplicate entry found in PRIMARY KEY: "+identifier);
-
-            ColumnMetadata cdef = identifier.prepare(metadata);
-
-            if (cdef.type.isMultiCell())
-                throw new InvalidRequestException(String.format("Cannot use MultiCell column '%s' in PRIMARY KEY of materialized view", identifier));
-
-            if (cdef.isStatic())
-                throw new InvalidRequestException(String.format("Cannot use Static column '%s' in PRIMARY KEY of materialized view", identifier));
-
-            if (cdef.type instanceof DurationType)
-                throw new InvalidRequestException(String.format("Cannot use Duration column '%s' in PRIMARY KEY of materialized view", identifier));
-        }
-
-        // build the select statement
-        Map<ColumnMetadata.Raw, Boolean> orderings = Collections.emptyMap();
-        List<ColumnMetadata.Raw> groups = Collections.emptyList();
-        SelectStatement.Parameters parameters = new SelectStatement.Parameters(orderings, groups, false, true, false);
-
-        SelectStatement.RawStatement rawSelect = new SelectStatement.RawStatement(baseName, parameters, selectClause, whereClause, null, null);
-
-        ClientState state = ClientState.forInternalCalls();
-        state.setKeyspace(keyspace());
-
-        rawSelect.prepareKeyspace(state);
-        rawSelect.setBoundVariables(getBoundVariables());
-
-        ParsedStatement.Prepared prepared = rawSelect.prepare(true);
-        SelectStatement select = (SelectStatement) prepared.statement;
-        StatementRestrictions restrictions = select.getRestrictions();
-
-        if (!prepared.boundNames.isEmpty())
-            throw new InvalidRequestException("Cannot use query parameters in CREATE MATERIALIZED VIEW statements");
-
-        // SEE CASSANDRA-13798, use it if the use case is append-only.
-        final boolean allowFilteringNonKeyColumns = Boolean.parseBoolean(System.getProperty("cassandra.mv.allow_filtering_nonkey_columns_unsafe",
-                                                                                            "false"));
-        if (!restrictions.nonPKRestrictedColumns(false).isEmpty() && !allowFilteringNonKeyColumns)
-        {
-            throw new InvalidRequestException(
-                                              String.format("Non-primary key columns cannot be restricted in the SELECT statement used"
-                                                      + " for materialized view creation (got restrictions on: %s)",
-                                                            restrictions.nonPKRestrictedColumns(false)
-                                                                        .stream()
-                                                                        .map(def -> def.name.toString())
-                                                                        .collect(Collectors.joining(", "))));
-        }
-
-        String whereClauseText = View.relationsToWhereClause(whereClause.relations);
-
-        Set<ColumnIdentifier> basePrimaryKeyCols = new HashSet<>();
-        for (ColumnMetadata definition : Iterables.concat(metadata.partitionKeyColumns(), metadata.clusteringColumns()))
-            basePrimaryKeyCols.add(definition.name);
-
-        List<ColumnIdentifier> targetClusteringColumns = new ArrayList<>();
-        List<ColumnIdentifier> targetPartitionKeys = new ArrayList<>();
-
-        // This is only used as an intermediate state; this is to catch whether multiple non-PK columns are used
-        boolean hasNonPKColumn = false;
-        for (ColumnMetadata.Raw raw : partitionKeys)
-            hasNonPKColumn |= getColumnIdentifier(metadata, basePrimaryKeyCols, hasNonPKColumn, raw, targetPartitionKeys, restrictions);
-
-        for (ColumnMetadata.Raw raw : clusteringKeys)
-            hasNonPKColumn |= getColumnIdentifier(metadata, basePrimaryKeyCols, hasNonPKColumn, raw, targetClusteringColumns, restrictions);
-
-        // We need to include all of the primary key columns from the base table in order to make sure that we do not
-        // overwrite values in the view. We cannot support "collapsing" the base table into a smaller number of rows in
-        // the view because if we need to generate a tombstone, we have no way of knowing which value is currently being
-        // used in the view and whether or not to generate a tombstone. In order to not surprise our users, we require
-        // that they include all of the columns. We provide them with a list of all of the columns left to include.
-        boolean missingClusteringColumns = false;
-        StringBuilder columnNames = new StringBuilder();
-        List<ColumnIdentifier> includedColumns = new ArrayList<>();
-        for (ColumnMetadata def : metadata.columns())
-        {
-            ColumnIdentifier identifier = def.name;
-            boolean includeDef = included.isEmpty() || included.contains(identifier);
-
-            if (includeDef && def.isStatic())
-            {
-                throw new InvalidRequestException(String.format("Unable to include static column '%s' which would be included by Materialized View SELECT * statement", identifier));
-            }
-
-            boolean defInTargetPrimaryKey = targetClusteringColumns.contains(identifier)
-                                            || targetPartitionKeys.contains(identifier);
-
-            if (includeDef && !defInTargetPrimaryKey)
-            {
-                includedColumns.add(identifier);
-            }
-            if (!def.isPrimaryKeyColumn()) continue;
-
-            if (!defInTargetPrimaryKey)
-            {
-                if (missingClusteringColumns)
-                    columnNames.append(',');
-                else
-                    missingClusteringColumns = true;
-                columnNames.append(identifier);
-            }
-        }
-        if (missingClusteringColumns)
-            throw new InvalidRequestException(String.format("Cannot create Materialized View %s without primary key columns from base %s (%s)",
-                                                            columnFamily(), baseName.getColumnFamily(), columnNames.toString()));
-
-        if (targetPartitionKeys.isEmpty())
-            throw new InvalidRequestException("Must select at least a column for a Materialized View");
-
-        if (targetClusteringColumns.isEmpty())
-            throw new InvalidRequestException("No columns are defined for Materialized View other than primary key");
-
-        TableParams params = properties.properties.asNewTableParams();
-
-        if (params.defaultTimeToLive > 0)
-        {
-            throw new InvalidRequestException("Cannot set default_time_to_live for a materialized view. " +
-                                              "Data in a materialized view always expire at the same time than " +
-                                              "the corresponding data in the parent table.");
-        }
-
-        TableMetadata.Builder builder =
-            TableMetadata.builder(keyspace(), columnFamily(), properties.properties.getId())
-                         .kind(TableMetadata.Kind.VIEW)
-                         .params(params);
-
-        add(metadata, targetPartitionKeys, builder::addPartitionKeyColumn);
-        add(metadata, targetClusteringColumns, builder::addClusteringColumn);
-        add(metadata, includedColumns, builder::addRegularColumn);
-
-        ViewMetadata definition = new ViewMetadata(keyspace(),
-                                                   columnFamily(),
-                                                   metadata.id,
-                                                   metadata.name,
-                                                   included.isEmpty(),
-                                                   rawSelect,
-                                                   whereClauseText,
-                                                   builder.build());
-
-        logger.warn("Creating materialized view {} for {}.{}. " +
-                    "Materialized views are experimental and are not recommended for production use.",
-                    definition.name, metadata.keyspace, metadata.name);
-
-        try
-        {
-            ClientWarn.instance.warn("Materialized views are experimental and are not recommended for production use.");
-            MigrationManager.announceNewView(definition, isLocalOnly);
-            return new Event.SchemaChange(Event.SchemaChange.Change.CREATED, Event.SchemaChange.Target.TABLE, keyspace(), columnFamily());
-        }
-        catch (AlreadyExistsException e)
-        {
-            if (ifNotExists)
-                return null;
-            throw e;
-        }
-    }
-
-    private static boolean getColumnIdentifier(TableMetadata cfm,
-                                               Set<ColumnIdentifier> basePK,
-                                               boolean hasNonPKColumn,
-                                               ColumnMetadata.Raw raw,
-                                               List<ColumnIdentifier> columns,
-                                               StatementRestrictions restrictions)
-    {
-        ColumnMetadata def = raw.prepare(cfm);
-
-        boolean isPk = basePK.contains(def.name);
-        if (!isPk && hasNonPKColumn)
-            throw new InvalidRequestException(String.format("Cannot include more than one non-primary key column '%s' in materialized view primary key", def.name));
-
-        // We don't need to include the "IS NOT NULL" filter on a non-composite partition key
-        // because we will never allow a single partition key to be NULL
-        boolean isSinglePartitionKey = def.isPartitionKey()
-                                       && cfm.partitionKeyColumns().size() == 1;
-        if (!isSinglePartitionKey && !restrictions.isRestricted(def))
-            throw new InvalidRequestException(String.format("Primary key column '%s' is required to be filtered by 'IS NOT NULL'", def.name));
-
-        columns.add(def.name);
-        return !isPk;
-    }
-    
-    @Override
-    public String toString()
-    {
-        return ToStringBuilder.reflectionToString(this, ToStringStyle.SHORT_PREFIX_STYLE);
-    }
-    @Override
-    public AuditLogContext getAuditLogContext()
-    {
-        return new AuditLogContext(AuditLogEntryType.CREATE_VIEW, keyspace(), cfName.getColumnFamily());
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/cql3/statements/DeleteStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/DeleteStatement.java b/src/java/org/apache/cassandra/cql3/statements/DeleteStatement.java
index 639286c..129bf87 100644
--- a/src/java/org/apache/cassandra/cql3/statements/DeleteStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/DeleteStatement.java
@@ -19,6 +19,7 @@ package org.apache.cassandra.cql3.statements;
 
 import java.util.List;
 
+import org.apache.cassandra.audit.AuditLogContext;
 import org.apache.cassandra.audit.AuditLogEntryType;
 import org.apache.cassandra.cql3.*;
 import org.apache.cassandra.cql3.conditions.ColumnCondition;
@@ -42,14 +43,14 @@ import static org.apache.cassandra.cql3.statements.RequestValidations.checkTrue;
  */
 public class DeleteStatement extends ModificationStatement
 {
-    private DeleteStatement(int boundTerms,
+    private DeleteStatement(VariableSpecifications bindVariables,
                             TableMetadata cfm,
                             Operations operations,
                             StatementRestrictions restrictions,
                             Conditions conditions,
                             Attributes attrs)
     {
-        super(StatementType.DELETE, boundTerms, cfm, operations, restrictions, conditions, attrs);
+        super(StatementType.DELETE, bindVariables, cfm, operations, restrictions, conditions, attrs);
     }
 
     @Override
@@ -125,7 +126,7 @@ public class DeleteStatement extends ModificationStatement
         private final List<Operation.RawDeletion> deletions;
         private final WhereClause whereClause;
 
-        public Parsed(CFName name,
+        public Parsed(QualifiedName name,
                       Attributes.Raw attrs,
                       List<Operation.RawDeletion> deletions,
                       WhereClause whereClause,
@@ -140,7 +141,7 @@ public class DeleteStatement extends ModificationStatement
 
         @Override
         protected ModificationStatement prepareInternal(TableMetadata metadata,
-                                                        VariableSpecifications boundNames,
+                                                        VariableSpecifications bindVariables,
                                                         Conditions conditions,
                                                         Attributes attrs)
         {
@@ -157,17 +158,17 @@ public class DeleteStatement extends ModificationStatement
                 checkFalse(def.isPrimaryKeyColumn(), "Invalid identifier %s for deletion (should not be a PRIMARY KEY part)", def.name);
 
                 Operation op = deletion.prepare(metadata.keyspace, def, metadata);
-                op.collectMarkerSpecification(boundNames);
+                op.collectMarkerSpecification(bindVariables);
                 operations.add(op);
             }
 
             StatementRestrictions restrictions = newRestrictions(metadata,
-                                                                 boundNames,
+                                                                 bindVariables,
                                                                  operations,
                                                                  whereClause,
                                                                  conditions);
 
-            DeleteStatement stmt = new DeleteStatement(boundNames.size(),
+            DeleteStatement stmt = new DeleteStatement(bindVariables,
                                                        metadata,
                                                        operations,
                                                        restrictions,

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/cql3/statements/DropAggregateStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/DropAggregateStatement.java b/src/java/org/apache/cassandra/cql3/statements/DropAggregateStatement.java
deleted file mode 100644
index 727e9f2..0000000
--- a/src/java/org/apache/cassandra/cql3/statements/DropAggregateStatement.java
+++ /dev/null
@@ -1,167 +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.statements;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
-
-import org.apache.cassandra.audit.AuditLogEntryType;
-import org.apache.cassandra.auth.Permission;
-import org.apache.cassandra.cql3.CQL3Type;
-import org.apache.cassandra.cql3.functions.*;
-import org.apache.cassandra.db.marshal.AbstractType;
-import org.apache.cassandra.exceptions.InvalidRequestException;
-import org.apache.cassandra.exceptions.RequestValidationException;
-import org.apache.cassandra.exceptions.UnauthorizedException;
-import org.apache.cassandra.schema.MigrationManager;
-import org.apache.cassandra.schema.Schema;
-import org.apache.cassandra.service.ClientState;
-import org.apache.cassandra.service.QueryState;
-import org.apache.cassandra.transport.Event;
-import org.apache.commons.lang3.builder.ToStringBuilder;
-import org.apache.commons.lang3.builder.ToStringStyle;
-
-/**
- * A {@code DROP AGGREGATE} statement parsed from a CQL query.
- */
-public final class DropAggregateStatement extends SchemaAlteringStatement
-{
-    private FunctionName functionName;
-    private final boolean ifExists;
-    private final List<CQL3Type.Raw> argRawTypes;
-    private final boolean argsPresent;
-
-    public DropAggregateStatement(FunctionName functionName,
-                                  List<CQL3Type.Raw> argRawTypes,
-                                  boolean argsPresent,
-                                  boolean ifExists)
-    {
-        this.functionName = functionName;
-        this.argRawTypes = argRawTypes;
-        this.argsPresent = argsPresent;
-        this.ifExists = ifExists;
-    }
-
-    public void prepareKeyspace(ClientState state) throws InvalidRequestException
-    {
-        if (!functionName.hasKeyspace() && state.getRawKeyspace() != null)
-            functionName = new FunctionName(state.getKeyspace(), functionName.name);
-
-        if (!functionName.hasKeyspace())
-            throw new InvalidRequestException("Functions must be fully qualified with a keyspace name if a keyspace is not set for the session");
-
-        Schema.validateKeyspaceNotSystem(functionName.keyspace);
-    }
-
-    public void checkAccess(ClientState state) throws UnauthorizedException, InvalidRequestException
-    {
-        // TODO CASSANDRA-7557 (function DDL permission)
-
-        state.hasKeyspaceAccess(functionName.keyspace, Permission.DROP);
-    }
-
-    public void validate(ClientState state) throws RequestValidationException
-    {
-    }
-
-    public Event.SchemaChange announceMigration(QueryState queryState, boolean isLocalOnly) throws RequestValidationException
-    {
-        Collection<Function> olds = Schema.instance.getFunctions(functionName);
-
-        if (!argsPresent && olds != null && olds.size() > 1)
-            throw new InvalidRequestException(String.format("'DROP AGGREGATE %s' matches multiple function definitions; " +
-                                                            "specify the argument types by issuing a statement like " +
-                                                            "'DROP AGGREGATE %s (type, type, ...)'. Hint: use cqlsh " +
-                                                            "'DESCRIBE AGGREGATE %s' command to find all overloads",
-                                                            functionName, functionName, functionName));
-
-        Function old = null;
-        if (argsPresent)
-        {
-            if (Schema.instance.getKeyspaceMetadata(functionName.keyspace) != null)
-            {
-                List<AbstractType<?>> argTypes = new ArrayList<>(argRawTypes.size());
-                for (CQL3Type.Raw rawType : argRawTypes)
-                    argTypes.add(prepareType("arguments", rawType));
-
-                old = Schema.instance.findFunction(functionName, argTypes).orElse(null);
-            }
-            if (old == null || !(old instanceof AggregateFunction))
-            {
-                if (ifExists)
-                    return null;
-                // just build a nicer error message
-                StringBuilder sb = new StringBuilder();
-                for (CQL3Type.Raw rawType : argRawTypes)
-                {
-                    if (sb.length() > 0)
-                        sb.append(", ");
-                    sb.append(rawType);
-                }
-                throw new InvalidRequestException(String.format("Cannot drop non existing aggregate '%s(%s)'",
-                                                                functionName, sb));
-            }
-        }
-        else
-        {
-            if (olds == null || olds.isEmpty() || !(olds.iterator().next() instanceof AggregateFunction))
-            {
-                if (ifExists)
-                    return null;
-                throw new InvalidRequestException(String.format("Cannot drop non existing aggregate '%s'", functionName));
-            }
-            old = olds.iterator().next();
-        }
-
-        if (old.isNative())
-            throw new InvalidRequestException(String.format("Cannot drop aggregate '%s' because it is a " +
-                                                            "native (built-in) function", functionName));
-
-        MigrationManager.announceAggregateDrop((UDAggregate)old, isLocalOnly);
-        return new Event.SchemaChange(Event.SchemaChange.Change.DROPPED, Event.SchemaChange.Target.AGGREGATE,
-                                      old.name().keyspace, old.name().name, AbstractType.asCQLTypeStringList(old.argTypes()));
-
-    }
-
-    private AbstractType<?> prepareType(String typeName, CQL3Type.Raw rawType)
-    {
-        if (rawType.isFrozen())
-            throw new InvalidRequestException(String.format("The function %s should not be frozen; remove the frozen<> modifier", typeName));
-
-        // UDT are not supported non frozen but we do not allow the frozen keyword for argument. So for the moment we
-        // freeze them here
-        if (!rawType.canBeNonFrozen())
-            rawType.freeze();
-
-        AbstractType<?> type = rawType.prepare(functionName.keyspace).getType();
-        return type;
-    }
-    
-    @Override
-    public String toString()
-    {
-        return ToStringBuilder.reflectionToString(this, ToStringStyle.SHORT_PREFIX_STYLE);
-    }
-
-    @Override
-    public AuditLogContext getAuditLogContext()
-    {
-        return new AuditLogContext(AuditLogEntryType.DROP_AGGREGATE, functionName.keyspace, functionName.name);
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/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
deleted file mode 100644
index 0abcffa..0000000
--- a/src/java/org/apache/cassandra/cql3/statements/DropFunctionStatement.java
+++ /dev/null
@@ -1,203 +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.statements;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
-
-import com.google.common.base.Joiner;
-
-import org.apache.cassandra.audit.AuditLogEntryType;
-import org.apache.cassandra.auth.FunctionResource;
-import org.apache.cassandra.auth.Permission;
-import org.apache.cassandra.cql3.CQL3Type;
-import org.apache.cassandra.cql3.functions.*;
-import org.apache.cassandra.db.marshal.AbstractType;
-import org.apache.cassandra.exceptions.InvalidRequestException;
-import org.apache.cassandra.exceptions.RequestValidationException;
-import org.apache.cassandra.exceptions.UnauthorizedException;
-import org.apache.cassandra.schema.KeyspaceMetadata;
-import org.apache.cassandra.schema.MigrationManager;
-import org.apache.cassandra.schema.Schema;
-import org.apache.cassandra.service.ClientState;
-import org.apache.cassandra.service.QueryState;
-import org.apache.cassandra.transport.Event;
-import org.apache.commons.lang3.builder.ToStringBuilder;
-import org.apache.commons.lang3.builder.ToStringStyle;
-
-/**
- * A {@code DROP FUNCTION} statement parsed from a CQL query.
- */
-public final class DropFunctionStatement extends SchemaAlteringStatement
-{
-    private FunctionName functionName;
-    private final boolean ifExists;
-    private final List<CQL3Type.Raw> argRawTypes;
-    private final boolean argsPresent;
-
-    private List<AbstractType<?>> argTypes;
-
-    public DropFunctionStatement(FunctionName functionName,
-                                 List<CQL3Type.Raw> argRawTypes,
-                                 boolean argsPresent,
-                                 boolean ifExists)
-    {
-        this.functionName = functionName;
-        this.argRawTypes = argRawTypes;
-        this.argsPresent = argsPresent;
-        this.ifExists = ifExists;
-    }
-
-    @Override
-    public Prepared prepare() throws InvalidRequestException
-    {
-        if (Schema.instance.getKeyspaceMetadata(functionName.keyspace) != null)
-        {
-            argTypes = new ArrayList<>(argRawTypes.size());
-            for (CQL3Type.Raw rawType : argRawTypes)
-            {
-                if (rawType.isFrozen())
-                    throw new InvalidRequestException("The function arguments should not be frozen; remove the frozen<> modifier");
-
-                // UDT are not supported non frozen but we do not allow the frozen keyword for argument. So for the moment we
-                // freeze them here
-                if (!rawType.canBeNonFrozen())
-                    rawType.freeze();
-
-                argTypes.add(rawType.prepare(functionName.keyspace).getType());
-            }
-        }
-
-        return super.prepare();
-    }
-
-    @Override
-    public void prepareKeyspace(ClientState state) throws InvalidRequestException
-    {
-        if (!functionName.hasKeyspace() && state.getRawKeyspace() != null)
-            functionName = new FunctionName(state.getKeyspace(), functionName.name);
-
-        if (!functionName.hasKeyspace())
-            throw new InvalidRequestException("Functions must be fully qualified with a keyspace name if a keyspace is not set for the session");
-
-        Schema.validateKeyspaceNotSystem(functionName.keyspace);
-    }
-
-    public void checkAccess(ClientState state) throws UnauthorizedException, InvalidRequestException
-    {
-        Function function = findFunction();
-        if (function == null)
-        {
-            if (!ifExists)
-                throw new InvalidRequestException(String.format("Unconfigured function %s.%s(%s)",
-                                                                functionName.keyspace,
-                                                                functionName.name,
-                                                                Joiner.on(",").join(argRawTypes)));
-        }
-        else
-        {
-            state.ensureHasPermission(Permission.DROP, FunctionResource.function(function.name().keyspace,
-                                                                                 function.name().name,
-                                                                                 function.argTypes()));
-        }
-    }
-
-    public void validate(ClientState state)
-    {
-        Collection<Function> olds = Schema.instance.getFunctions(functionName);
-
-        if (!argsPresent && olds != null && olds.size() > 1)
-            throw new InvalidRequestException(String.format("'DROP FUNCTION %s' matches multiple function definitions; " +
-                                                            "specify the argument types by issuing a statement like " +
-                                                            "'DROP FUNCTION %s (type, type, ...)'. Hint: use cqlsh " +
-                                                            "'DESCRIBE FUNCTION %s' command to find all overloads",
-                                                            functionName, functionName, functionName));
-    }
-
-    public Event.SchemaChange announceMigration(QueryState queryState, boolean isLocalOnly) throws RequestValidationException
-    {
-        Function old = findFunction();
-        if (old == null)
-        {
-            if (ifExists)
-                return null;
-            else
-                throw new InvalidRequestException(getMissingFunctionError());
-        }
-
-        KeyspaceMetadata ksm = Schema.instance.getKeyspaceMetadata(old.name().keyspace);
-        Collection<UDAggregate> referrers = ksm.functions.aggregatesUsingFunction(old);
-        if (!referrers.isEmpty())
-            throw new InvalidRequestException(String.format("Function '%s' still referenced by %s", old, referrers));
-
-        MigrationManager.announceFunctionDrop((UDFunction) old, isLocalOnly);
-
-        return new Event.SchemaChange(Event.SchemaChange.Change.DROPPED, Event.SchemaChange.Target.FUNCTION,
-                                      old.name().keyspace, old.name().name, AbstractType.asCQLTypeStringList(old.argTypes()));
-    }
-
-    private String getMissingFunctionError()
-    {
-        // just build a nicer error message
-        StringBuilder sb = new StringBuilder("Cannot drop non existing function '");
-        sb.append(functionName);
-        if (argsPresent)
-            sb.append(Joiner.on(", ").join(argRawTypes));
-        sb.append('\'');
-        return sb.toString();
-    }
-
-    private Function findFunction()
-    {
-        Function old;
-        if (argsPresent)
-        {
-            if (argTypes == null)
-            {
-                return null;
-            }
-
-            old = Schema.instance.findFunction(functionName, argTypes).orElse(null);
-            if (old == null || !(old instanceof ScalarFunction))
-            {
-                return null;
-            }
-        }
-        else
-        {
-            Collection<Function> olds = Schema.instance.getFunctions(functionName);
-            if (olds == null || olds.isEmpty() || !(olds.iterator().next() instanceof ScalarFunction))
-                return null;
-
-            old = olds.iterator().next();
-        }
-        return old;
-    }
-    @Override
-    public String toString()
-    {
-        return ToStringBuilder.reflectionToString(this, ToStringStyle.SHORT_PREFIX_STYLE);
-    }
-
-    @Override
-    public AuditLogContext getAuditLogContext()
-    {
-        return new AuditLogContext(AuditLogEntryType.DROP_FUNCTION, functionName.keyspace, functionName.name);
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/cql3/statements/DropIndexStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/DropIndexStatement.java b/src/java/org/apache/cassandra/cql3/statements/DropIndexStatement.java
deleted file mode 100644
index f61faf1..0000000
--- a/src/java/org/apache/cassandra/cql3/statements/DropIndexStatement.java
+++ /dev/null
@@ -1,131 +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.statements;
-
-import org.apache.cassandra.audit.AuditLogEntryType;
-import org.apache.cassandra.auth.Permission;
-import org.apache.cassandra.schema.TableMetadata;
-import org.apache.cassandra.schema.Schema;
-import org.apache.cassandra.cql3.IndexName;
-import org.apache.cassandra.cql3.QueryOptions;
-import org.apache.cassandra.db.KeyspaceNotDefinedException;
-import org.apache.cassandra.exceptions.*;
-import org.apache.cassandra.schema.KeyspaceMetadata;
-import org.apache.cassandra.service.ClientState;
-import org.apache.cassandra.schema.MigrationManager;
-import org.apache.cassandra.service.QueryState;
-import org.apache.cassandra.transport.Event;
-import org.apache.cassandra.transport.messages.ResultMessage;
-import org.apache.commons.lang3.builder.ToStringBuilder;
-import org.apache.commons.lang3.builder.ToStringStyle;
-
-public class DropIndexStatement extends SchemaAlteringStatement
-{
-    public final String indexName;
-    public final boolean ifExists;
-
-    public DropIndexStatement(IndexName indexName, boolean ifExists)
-    {
-        super(indexName.getCfName());
-        this.indexName = indexName.getIdx();
-        this.ifExists = ifExists;
-    }
-
-    public String columnFamily()
-    {
-        TableMetadata metadata = lookupIndexedTable();
-        return metadata == null ? null : metadata.name;
-    }
-
-    public void checkAccess(ClientState state) throws UnauthorizedException, InvalidRequestException
-    {
-        TableMetadata metadata = lookupIndexedTable();
-        if (metadata == null)
-            return;
-
-        state.hasColumnFamilyAccess(metadata.keyspace, metadata.name, Permission.ALTER);
-    }
-
-    public void validate(ClientState state)
-    {
-        // validated in lookupIndexedTable()
-    }
-
-    @Override
-    public ResultMessage execute(QueryState state, QueryOptions options, long queryStartNanoTime) throws RequestValidationException
-    {
-        Event.SchemaChange ce = announceMigration(state, false);
-        return ce == null ? null : new ResultMessage.SchemaChange(ce);
-    }
-
-    public Event.SchemaChange announceMigration(QueryState queryState, boolean isLocalOnly) throws InvalidRequestException, ConfigurationException
-    {
-        TableMetadata current = lookupIndexedTable();
-        if (current == null)
-            return null;
-
-        TableMetadata updated =
-            current.unbuild()
-                   .indexes(current.indexes.without(indexName))
-                   .build();
-
-        MigrationManager.announceTableUpdate(updated, isLocalOnly);
-        // Dropping an index is akin to updating the CF
-        // Note that we shouldn't call columnFamily() at this point because the index has been dropped and the call to lookupIndexedTable()
-        // in that method would now throw.
-        return new Event.SchemaChange(Event.SchemaChange.Change.UPDATED, Event.SchemaChange.Target.TABLE, current.keyspace, current.name);
-    }
-
-    /**
-     * The table for which the index should be dropped, or null if the index doesn't exist
-     *
-     * @return the metadata for the table containing the dropped index, or {@code null}
-     * if the index to drop cannot be found but "IF EXISTS" is set on the statement.
-     *
-     * @throws InvalidRequestException if the index cannot be found and "IF EXISTS" is not
-     * set on the statement.
-     */
-    private TableMetadata lookupIndexedTable()
-    {
-        KeyspaceMetadata ksm = Schema.instance.getKeyspaceMetadata(keyspace());
-        if (ksm == null)
-            throw new KeyspaceNotDefinedException("Keyspace " + keyspace() + " does not exist");
-
-        return ksm.findIndexedTable(indexName)
-                  .orElseGet(() -> {
-                      if (ifExists)
-                          return null;
-                      else
-                          throw new InvalidRequestException(String.format("Index '%s' could not be found in any " +
-                                                                          "of the tables of keyspace '%s'",
-                                                                          indexName, keyspace()));
-                  });
-    }
-    
-    @Override
-    public String toString()
-    {
-        return ToStringBuilder.reflectionToString(this, ToStringStyle.SHORT_PREFIX_STYLE);
-    }
-
-    @Override
-    public AuditLogContext getAuditLogContext()
-    {
-        return new AuditLogContext(AuditLogEntryType.DROP_INDEX, keyspace(), indexName);
-    }
-}


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


[03/13] cassandra git commit: Make all DDL statements idempotent and not dependent on global state

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/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 234ac4f..688df91 100644
--- a/src/java/org/apache/cassandra/service/ClientState.java
+++ b/src/java/org/apache/cassandra/service/ClientState.java
@@ -46,7 +46,6 @@ import org.apache.cassandra.exceptions.UnauthorizedException;
 import org.apache.cassandra.schema.SchemaKeyspace;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.JVMStabilityInspector;
-import org.apache.cassandra.utils.CassandraVersion;
 
 /**
  * State related to a client connection.
@@ -54,7 +53,6 @@ import org.apache.cassandra.utils.CassandraVersion;
 public class ClientState
 {
     private static final Logger logger = LoggerFactory.getLogger(ClientState.class);
-    public static final CassandraVersion DEFAULT_CQL_VERSION = org.apache.cassandra.cql3.QueryProcessor.CQL_VERSION;
 
     private static final Set<IResource> READABLE_SYSTEM_RESOURCES = new HashSet<>();
     private static final Set<IResource> PROTECTED_AUTH_RESOURCES = new HashSet<>();
@@ -158,6 +156,13 @@ public class ClientState
         return new ClientState();
     }
 
+    public static ClientState forInternalCalls(String keyspace)
+    {
+        ClientState state = new ClientState();
+        state.setKeyspace(keyspace);
+        return state;
+    }
+
     /**
      * @return a ClientState object for external clients (native protocol users).
      */
@@ -298,7 +303,7 @@ public class ClientState
         return keyspace;
     }
 
-    public void setKeyspace(String ks) throws InvalidRequestException
+    public void setKeyspace(String ks)
     {
         // Skip keyspace validation for non-authenticated users. Apparently, some client libraries
         // call set_keyspace() before calling login(), and we have to handle that.
@@ -310,7 +315,7 @@ public class ClientState
     /**
      * Attempts to login the given user.
      */
-    public void login(AuthenticatedUser user) throws AuthenticationException
+    public void login(AuthenticatedUser user)
     {
         // Login privilege is not inherited via granted roles, so just
         // verify that the role with the credentials that were actually
@@ -321,40 +326,35 @@ public class ClientState
             throw new AuthenticationException(String.format("%s is not permitted to log in", user.getName()));
     }
 
-    public void hasAllKeyspacesAccess(Permission perm) throws UnauthorizedException
+    public void ensureAllKeyspacesPermission(Permission perm)
     {
         if (isInternal)
             return;
         validateLogin();
-        ensureHasPermission(perm, DataResource.root());
+        ensurePermission(perm, DataResource.root());
     }
 
-    public void hasKeyspaceAccess(String keyspace, Permission perm) throws UnauthorizedException, InvalidRequestException
+    public void ensureKeyspacePermission(String keyspace, Permission perm)
     {
-        hasAccess(keyspace, perm, DataResource.keyspace(keyspace));
+        ensurePermission(keyspace, perm, DataResource.keyspace(keyspace));
     }
 
-    public void hasColumnFamilyAccess(String keyspace, String columnFamily, Permission perm)
-    throws UnauthorizedException, InvalidRequestException
+    public void ensureTablePermission(String keyspace, String table, Permission perm)
     {
-        Schema.instance.validateTable(keyspace, columnFamily);
-        hasAccess(keyspace, perm, DataResource.table(keyspace, columnFamily));
+        ensurePermission(keyspace, perm, DataResource.table(keyspace, table));
     }
 
-    public void hasColumnFamilyAccess(TableMetadataRef tableRef, Permission perm)
-    throws UnauthorizedException, InvalidRequestException
+    public void ensureTablePermission(TableMetadataRef tableRef, Permission perm)
     {
-        hasColumnFamilyAccess(tableRef.get(), perm);
+        ensureTablePermission(tableRef.get(), perm);
     }
 
-    public void hasColumnFamilyAccess(TableMetadata table, Permission perm)
-    throws UnauthorizedException, InvalidRequestException
+    public void ensureTablePermission(TableMetadata table, Permission perm)
     {
-        hasAccess(table.keyspace, perm, table.resource);
+        ensurePermission(table.keyspace, perm, table.resource);
     }
 
-    private void hasAccess(String keyspace, Permission perm, DataResource resource)
-    throws UnauthorizedException, InvalidRequestException
+    private void ensurePermission(String keyspace, Permission perm, DataResource resource)
     {
         validateKeyspace(keyspace);
 
@@ -371,11 +371,10 @@ public class ClientState
         if (PROTECTED_AUTH_RESOURCES.contains(resource))
             if ((perm == Permission.CREATE) || (perm == Permission.ALTER) || (perm == Permission.DROP))
                 throw new UnauthorizedException(String.format("%s schema is protected", resource));
-
-        ensureHasPermission(perm, resource);
+        ensurePermission(perm, resource);
     }
 
-    public void ensureHasPermission(Permission perm, IResource resource) throws UnauthorizedException
+    public void ensurePermission(Permission perm, IResource resource)
     {
         if (!DatabaseDescriptor.getAuthorizer().requireAuthorization())
             return;
@@ -385,12 +384,12 @@ public class ClientState
             if (((FunctionResource)resource).getKeyspace().equals(SchemaConstants.SYSTEM_KEYSPACE_NAME))
                 return;
 
-        checkPermissionOnResourceChain(perm, resource);
+        ensurePermissionOnResourceChain(perm, resource);
     }
 
-    // Convenience method called from checkAccess method of CQLStatement
+    // Convenience method called from authorize method of CQLStatement
     // Also avoids needlessly creating lots of FunctionResource objects
-    public void ensureHasPermission(Permission permission, Function function)
+    public void ensurePermission(Permission permission, Function function)
     {
         // Save creating a FunctionResource is we don't need to
         if (!DatabaseDescriptor.getAuthorizer().requireAuthorization())
@@ -400,12 +399,12 @@ public class ClientState
         if (function.isNative())
             return;
 
-        checkPermissionOnResourceChain(permission, FunctionResource.function(function.name().keyspace,
-                                                                             function.name().name,
-                                                                             function.argTypes()));
+        ensurePermissionOnResourceChain(permission, FunctionResource.function(function.name().keyspace,
+                                                                              function.name().name,
+                                                                              function.argTypes()));
     }
 
-    private void checkPermissionOnResourceChain(Permission perm, IResource resource)
+    private void ensurePermissionOnResourceChain(Permission perm, IResource resource)
     {
         for (IResource r : Resources.chain(resource))
             if (authorize(r).contains(perm))
@@ -417,7 +416,7 @@ public class ClientState
                                                       resource));
     }
 
-    private void preventSystemKSSchemaModification(String keyspace, DataResource resource, Permission perm) throws UnauthorizedException
+    private void preventSystemKSSchemaModification(String keyspace, DataResource resource, Permission perm)
     {
         // we only care about DDL statements
         if (perm != Permission.ALTER && perm != Permission.DROP && perm != Permission.CREATE)
@@ -438,7 +437,7 @@ public class ClientState
         }
     }
 
-    public void validateLogin() throws UnauthorizedException
+    public void validateLogin()
     {
         if (user == null)
         {
@@ -450,20 +449,20 @@ public class ClientState
         }
     }
 
-    public void ensureNotAnonymous() throws UnauthorizedException
+    public void ensureNotAnonymous()
     {
         validateLogin();
         if (user.isAnonymous())
             throw new UnauthorizedException("You have to be logged in and not anonymous to perform this request");
     }
 
-    public void ensureIsSuper(String message) throws UnauthorizedException
+    public void ensureIsSuperuser(String message)
     {
         if (DatabaseDescriptor.getAuthenticator().requireAuthentication() && (user == null || !user.isSuper()))
             throw new UnauthorizedException(message);
     }
 
-    private static void validateKeyspace(String keyspace) throws InvalidRequestException
+    private static void validateKeyspace(String keyspace)
     {
         if (keyspace == null)
             throw new InvalidRequestException("You have not set a keyspace for this session");
@@ -474,11 +473,6 @@ public class ClientState
         return user;
     }
 
-    public static CassandraVersion[] getCQLSupportedVersion()
-    {
-        return new CassandraVersion[]{ QueryProcessor.CQL_VERSION };
-    }
-
     private Set<Permission> authorize(IResource resource)
     {
         return user.getPermissions(resource);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/service/StorageService.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java
index 96fd63f..cc02e17 100644
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@ -65,6 +65,7 @@ import org.apache.cassandra.concurrent.Stage;
 import org.apache.cassandra.concurrent.StageManager;
 import org.apache.cassandra.config.Config;
 import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.cql3.QueryProcessor;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.commitlog.CommitLog;
 import org.apache.cassandra.db.compaction.CompactionManager;
@@ -631,8 +632,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
     public synchronized void initServer(int delay) throws ConfigurationException
     {
         logger.info("Cassandra version: {}", FBUtilities.getReleaseVersionString());
-        logger.info("CQL supported versions: {} (default: {})",
-                StringUtils.join(ClientState.getCQLSupportedVersion(), ", "), ClientState.DEFAULT_CQL_VERSION);
+        logger.info("CQL version: {}", QueryProcessor.CQL_VERSION);
         logger.info("Native protocol supported versions: {} (default: {})",
                     StringUtils.join(ProtocolVersion.supportedVersions(), ", "), ProtocolVersion.CURRENT);
 
@@ -1590,7 +1590,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         for (String keyspace : Schema.instance.getUserKeyspaces())
         {
             for (ViewMetadata view: Schema.instance.getKeyspaceMetadata(keyspace).views)
-                SystemKeyspace.finishViewBuildStatus(view.keyspace, view.name);
+                SystemKeyspace.finishViewBuildStatus(view.keyspace(), view.name());
         }
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/tools/SSTableOfflineRelevel.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/SSTableOfflineRelevel.java b/src/java/org/apache/cassandra/tools/SSTableOfflineRelevel.java
index 1116575..b88bf0a 100644
--- a/src/java/org/apache/cassandra/tools/SSTableOfflineRelevel.java
+++ b/src/java/org/apache/cassandra/tools/SSTableOfflineRelevel.java
@@ -93,7 +93,7 @@ public class SSTableOfflineRelevel
         Schema.instance.loadFromDisk(false);
 
         if (Schema.instance.getTableMetadataRef(keyspace, columnfamily) == null)
-            throw new IllegalArgumentException(String.format("Unknown keyspace/columnFamily %s.%s",
+            throw new IllegalArgumentException(String.format("Unknown keyspace/table %s.%s",
                     keyspace,
                     columnfamily));
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/tracing/TraceKeyspace.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tracing/TraceKeyspace.java b/src/java/org/apache/cassandra/tracing/TraceKeyspace.java
index c29760e..6477d8c 100644
--- a/src/java/org/apache/cassandra/tracing/TraceKeyspace.java
+++ b/src/java/org/apache/cassandra/tracing/TraceKeyspace.java
@@ -22,7 +22,7 @@ import java.nio.ByteBuffer;
 import java.util.*;
 import java.util.concurrent.TimeUnit;
 
-import org.apache.cassandra.cql3.statements.CreateTableStatement;
+import org.apache.cassandra.cql3.statements.schema.CreateTableStatement;
 import org.apache.cassandra.db.Mutation;
 import org.apache.cassandra.db.partitions.PartitionUpdate;
 import org.apache.cassandra.db.rows.Row;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/transport/Event.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/transport/Event.java b/src/java/org/apache/cassandra/transport/Event.java
index 77edf8a..254e1e1 100644
--- a/src/java/org/apache/cassandra/transport/Event.java
+++ b/src/java/org/apache/cassandra/transport/Event.java
@@ -23,7 +23,10 @@ import java.util.Iterator;
 import java.util.List;
 
 import com.google.common.base.Objects;
+
 import io.netty.buffer.ByteBuf;
+import org.apache.cassandra.cql3.functions.UDAggregate;
+import org.apache.cassandra.cql3.functions.UDFunction;
 import org.apache.cassandra.locator.InetAddressAndPort;
 
 public abstract class Event
@@ -268,6 +271,16 @@ public abstract class Event
             this(change, Target.KEYSPACE, keyspace, null);
         }
 
+        public static SchemaChange forFunction(Change change, UDFunction function)
+        {
+            return new SchemaChange(change, Target.FUNCTION, function.name().keyspace, function.name().name, function.argumentsList());
+        }
+
+        public static SchemaChange forAggregate(Change change, UDAggregate aggregate)
+        {
+            return new SchemaChange(change, Target.AGGREGATE, aggregate.name().keyspace, aggregate.name().name, aggregate.argumentsList());
+        }
+
         // Assumes the type has already been deserialized
         public static SchemaChange deserializeEvent(ByteBuf cb, ProtocolVersion version)
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/transport/messages/BatchMessage.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/transport/messages/BatchMessage.java b/src/java/org/apache/cassandra/transport/messages/BatchMessage.java
index 5ffadac..29f92f7 100644
--- a/src/java/org/apache/cassandra/transport/messages/BatchMessage.java
+++ b/src/java/org/apache/cassandra/transport/messages/BatchMessage.java
@@ -29,12 +29,13 @@ import org.apache.cassandra.audit.AuditLogEntry;
 import org.apache.cassandra.audit.AuditLogEntryType;
 import org.apache.cassandra.cql3.Attributes;
 import org.apache.cassandra.cql3.BatchQueryOptions;
+import org.apache.cassandra.cql3.CQLStatement;
 import org.apache.cassandra.cql3.QueryHandler;
 import org.apache.cassandra.cql3.QueryOptions;
 import org.apache.cassandra.cql3.QueryProcessor;
+import org.apache.cassandra.cql3.VariableSpecifications;
 import org.apache.cassandra.cql3.statements.BatchStatement;
 import org.apache.cassandra.cql3.statements.ModificationStatement;
-import org.apache.cassandra.cql3.statements.ParsedStatement;
 import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.exceptions.PreparedQueryNotFoundException;
 import org.apache.cassandra.service.ClientState;
@@ -182,27 +183,28 @@ public class BatchMessage extends Message.Request
             }
 
             QueryHandler handler = ClientState.getCQLQueryHandler();
-            List<ParsedStatement.Prepared> prepared = new ArrayList<>(queryOrIdList.size());
+            List<QueryHandler.Prepared> prepared = new ArrayList<>(queryOrIdList.size());
             for (int i = 0; i < queryOrIdList.size(); i++)
             {
                 Object query = queryOrIdList.get(i);
-                ParsedStatement.Prepared p;
+                CQLStatement statement;
+                QueryHandler.Prepared p;
                 if (query instanceof String)
                 {
-                    p = QueryProcessor.parseStatement((String)query,
-                                                      state.getClientState().cloneWithKeyspaceIfSet(options.getKeyspace()));
+                    statement = QueryProcessor.parseStatement((String)query, state.getClientState().cloneWithKeyspaceIfSet(options.getKeyspace()));
+                    p = new QueryHandler.Prepared(statement, (String) query);
                 }
                 else
                 {
                     p = handler.getPrepared((MD5Digest)query);
-                    if (p == null)
+                    if (null == p)
                         throw new PreparedQueryNotFoundException((MD5Digest)query);
                 }
 
                 List<ByteBuffer> queryValues = values.get(i);
-                if (queryValues.size() != p.statement.getBoundTerms())
+                if (queryValues.size() != p.statement.getBindVariables().size())
                     throw new InvalidRequestException(String.format("There were %d markers(?) in CQL but %d bound variables",
-                                                                    p.statement.getBoundTerms(),
+                                                                    p.statement.getBindVariables().size(),
                                                                     queryValues.size()));
 
                 prepared.add(p);
@@ -212,18 +214,18 @@ public class BatchMessage extends Message.Request
             List<ModificationStatement> statements = new ArrayList<>(prepared.size());
             for (int i = 0; i < prepared.size(); i++)
             {
-                ParsedStatement.Prepared p = prepared.get(i);
-                batchOptions.prepareStatement(i, p.boundNames);
+                CQLStatement statement = prepared.get(i).statement;
+                batchOptions.prepareStatement(i, statement.getBindVariables());
 
-                if (!(p.statement instanceof ModificationStatement))
+                if (!(statement instanceof ModificationStatement))
                     throw new InvalidRequestException("Invalid statement in batch: only UPDATE, INSERT and DELETE statements are allowed.");
 
-                statements.add((ModificationStatement)p.statement);
+                statements.add((ModificationStatement) statement);
             }
 
             // Note: It's ok at this point to pass a bogus value for the number of bound terms in the BatchState ctor
             // (and no value would be really correct, so we prefer passing a clearly wrong one).
-            BatchStatement batch = new BatchStatement(-1, batchType, statements, Attributes.none());
+            BatchStatement batch = new BatchStatement(batchType, VariableSpecifications.empty(), statements, Attributes.none());
 
             long fqlTime = isLoggingEnabled ? System.currentTimeMillis() : 0;
             Message.Response response = handler.processBatch(batch, state, batchOptions, getCustomPayload(), queryStartNanoTime);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/transport/messages/ExecuteMessage.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/transport/messages/ExecuteMessage.java b/src/java/org/apache/cassandra/transport/messages/ExecuteMessage.java
index cd7f300..6c0b77a 100644
--- a/src/java/org/apache/cassandra/transport/messages/ExecuteMessage.java
+++ b/src/java/org/apache/cassandra/transport/messages/ExecuteMessage.java
@@ -28,8 +28,8 @@ import org.apache.cassandra.cql3.CQLStatement;
 import org.apache.cassandra.cql3.ColumnSpecification;
 import org.apache.cassandra.cql3.QueryHandler;
 import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.QueryProcessor;
 import org.apache.cassandra.cql3.ResultSet;
-import org.apache.cassandra.cql3.statements.ParsedStatement;
 import org.apache.cassandra.exceptions.PreparedQueryNotFoundException;
 import org.apache.cassandra.service.ClientState;
 import org.apache.cassandra.service.QueryState;
@@ -113,12 +113,12 @@ public class ExecuteMessage extends Message.Request
         try
         {
             QueryHandler handler = ClientState.getCQLQueryHandler();
-            ParsedStatement.Prepared prepared = handler.getPrepared(statementId);
+            QueryProcessor.Prepared prepared = handler.getPrepared(statementId);
             if (prepared == null)
                 throw new PreparedQueryNotFoundException(statementId);
 
-            options.prepare(prepared.boundNames);
             CQLStatement statement = prepared.statement;
+            options.prepare(statement.getBindVariables());
 
             if (options.getPageSize() == 0)
                 throw new ProtocolException("The page size cannot be 0");
@@ -143,9 +143,9 @@ public class ExecuteMessage extends Message.Request
                     builder.put("serial_consistency_level", options.getSerialConsistency().name());
                 builder.put("query", prepared.rawCQLStatement);
 
-                for(int i=0;i<prepared.boundNames.size();i++)
+                for(int i = 0; i < statement.getBindVariables().size(); i++)
                 {
-                    ColumnSpecification cs = prepared.boundNames.get(i);
+                    ColumnSpecification cs = statement.getBindVariables().get(i);
                     String boundName = cs.name.toString();
                     String boundValue = cs.type.asCQL3Type().toCQLLiteral(options.getValues().get(i), options.getProtocolVersion());
                     if ( boundValue.length() > 1000 )
@@ -163,7 +163,7 @@ public class ExecuteMessage extends Message.Request
 
             // Some custom QueryHandlers are interested by the bound names. We provide them this information
             // by wrapping the QueryOptions.
-            QueryOptions queryOptions = QueryOptions.addColumnSpecifications(options, prepared.boundNames);
+            QueryOptions queryOptions = QueryOptions.addColumnSpecifications(options, prepared.statement.getBindVariables());
 
             long fqlTime = isLoggingEnabled ? System.currentTimeMillis() : 0;
             Message.Response response = handler.processPrepared(statement, state, queryOptions, getCustomPayload(), queryStartNanoTime);
@@ -230,7 +230,7 @@ public class ExecuteMessage extends Message.Request
                 }
                 else
                 {
-                    ParsedStatement.Prepared prepared = ClientState.getCQLQueryHandler().getPrepared(statementId);
+                    QueryHandler.Prepared prepared = ClientState.getCQLQueryHandler().getPrepared(statementId);
                     if (prepared != null)
                     {
                         AuditLogEntry auditLogEntry = new AuditLogEntry.Builder(state.getClientState())

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/transport/messages/PrepareMessage.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/transport/messages/PrepareMessage.java b/src/java/org/apache/cassandra/transport/messages/PrepareMessage.java
index e5e5248..4ab6e0b 100644
--- a/src/java/org/apache/cassandra/transport/messages/PrepareMessage.java
+++ b/src/java/org/apache/cassandra/transport/messages/PrepareMessage.java
@@ -24,8 +24,8 @@ import com.google.common.collect.ImmutableMap;
 import io.netty.buffer.ByteBuf;
 import org.apache.cassandra.audit.AuditLogEntry;
 import org.apache.cassandra.audit.AuditLogEntryType;
+import org.apache.cassandra.cql3.CQLStatement;
 import org.apache.cassandra.cql3.QueryProcessor;
-import org.apache.cassandra.cql3.statements.ParsedStatement;
 import org.apache.cassandra.service.ClientState;
 import org.apache.cassandra.service.QueryState;
 import org.apache.cassandra.tracing.Tracing;
@@ -119,12 +119,12 @@ public class PrepareMessage extends Message.Request
                                                                                  getCustomPayload());
             if (auditLogEnabled)
             {
-                ParsedStatement.Prepared parsedStmt = QueryProcessor.parseStatement(query, state.getClientState());
+                CQLStatement parsedStmt = QueryProcessor.parseStatement(query, state.getClientState());
                 AuditLogEntry auditLogEntry = new AuditLogEntry.Builder(state.getClientState())
                                               .setOperation(query)
                                               .setType(AuditLogEntryType.PREPARE_STATEMENT)
-                                              .setScope(parsedStmt.statement)
-                                              .setKeyspace(parsedStmt.statement)
+                                              .setScope(parsedStmt)
+                                              .setKeyspace(parsedStmt)
                                               .build();
                 auditLogManager.log(auditLogEntry);
             }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/transport/messages/QueryMessage.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/transport/messages/QueryMessage.java b/src/java/org/apache/cassandra/transport/messages/QueryMessage.java
index 9df9205..4f42b85 100644
--- a/src/java/org/apache/cassandra/transport/messages/QueryMessage.java
+++ b/src/java/org/apache/cassandra/transport/messages/QueryMessage.java
@@ -24,9 +24,9 @@ import com.google.common.collect.ImmutableMap;
 import io.netty.buffer.ByteBuf;
 import org.apache.cassandra.audit.AuditLogEntry;
 import org.apache.cassandra.audit.AuditLogManager;
+import org.apache.cassandra.cql3.CQLStatement;
 import org.apache.cassandra.cql3.QueryOptions;
 import org.apache.cassandra.cql3.QueryProcessor;
-import org.apache.cassandra.cql3.statements.ParsedStatement;
 import org.apache.cassandra.exceptions.RequestExecutionException;
 import org.apache.cassandra.exceptions.RequestValidationException;
 import org.apache.cassandra.service.ClientState;
@@ -122,13 +122,13 @@ public class QueryMessage extends Message.Request
 
             if (isLoggingEnabled)
             {
-                ParsedStatement.Prepared parsedStatement = QueryProcessor.parseStatement(query, state.getClientState());
+                CQLStatement parsedStatement = QueryProcessor.parseStatement(query, state.getClientState());
                 AuditLogEntry auditEntry = new AuditLogEntry.Builder(state.getClientState())
-                                           .setType(parsedStatement.statement.getAuditLogContext().auditLogEntryType)
+                                           .setType(parsedStatement.getAuditLogContext().auditLogEntryType)
                                            .setOperation(query)
                                            .setTimestamp(fqlTime)
-                                           .setScope(parsedStatement.statement)
-                                           .setKeyspace(state, parsedStatement.statement)
+                                           .setScope(parsedStatement)
+                                           .setKeyspace(state, parsedStatement)
                                            .setOptions(options)
                                            .build();
                 AuditLogManager.getInstance().log(auditEntry);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/transport/messages/ResultMessage.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/transport/messages/ResultMessage.java b/src/java/org/apache/cassandra/transport/messages/ResultMessage.java
index b989a7d..4485849 100644
--- a/src/java/org/apache/cassandra/transport/messages/ResultMessage.java
+++ b/src/java/org/apache/cassandra/transport/messages/ResultMessage.java
@@ -22,10 +22,7 @@ import com.google.common.annotations.VisibleForTesting;
 
 import io.netty.buffer.ByteBuf;
 
-import org.apache.cassandra.cql3.ColumnSpecification;
-import org.apache.cassandra.cql3.CQLStatement;
 import org.apache.cassandra.cql3.ResultSet;
-import org.apache.cassandra.cql3.statements.ParsedStatement;
 import org.apache.cassandra.transport.*;
 import org.apache.cassandra.utils.MD5Digest;
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/triggers/TriggerExecutor.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/triggers/TriggerExecutor.java b/src/java/org/apache/cassandra/triggers/TriggerExecutor.java
index 754183f..295003f 100644
--- a/src/java/org/apache/cassandra/triggers/TriggerExecutor.java
+++ b/src/java/org/apache/cassandra/triggers/TriggerExecutor.java
@@ -252,11 +252,11 @@ public class TriggerExecutor
         }
     }
 
-    public synchronized ITrigger loadTriggerInstance(String triggerName) throws Exception
+    public synchronized ITrigger loadTriggerInstance(String triggerClass) throws Exception
     {
         // double check.
-        if (cachedTriggers.get(triggerName) != null)
-            return cachedTriggers.get(triggerName);
-        return (ITrigger) customClassLoader.loadClass(triggerName).getConstructor().newInstance();
+        if (cachedTriggers.get(triggerClass) != null)
+            return cachedTriggers.get(triggerClass);
+        return (ITrigger) customClassLoader.loadClass(triggerClass).getConstructor().newInstance();
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/test/microbench/org/apache/cassandra/test/microbench/MutationBench.java
----------------------------------------------------------------------
diff --git a/test/microbench/org/apache/cassandra/test/microbench/MutationBench.java b/test/microbench/org/apache/cassandra/test/microbench/MutationBench.java
index 45ad258..4a0e646 100644
--- a/test/microbench/org/apache/cassandra/test/microbench/MutationBench.java
+++ b/test/microbench/org/apache/cassandra/test/microbench/MutationBench.java
@@ -25,7 +25,7 @@ import java.util.Collection;
 import java.util.concurrent.*;
 
 import org.apache.cassandra.UpdateBuilder;
-import org.apache.cassandra.cql3.statements.CreateTableStatement;
+import org.apache.cassandra.cql3.statements.schema.CreateTableStatement;
 import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.schema.Schema;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/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 eddaa1e..62b9670 100644
--- a/test/unit/org/apache/cassandra/SchemaLoader.java
+++ b/test/unit/org/apache/cassandra/SchemaLoader.java
@@ -27,29 +27,29 @@ import org.apache.cassandra.auth.IAuthenticator;
 import org.apache.cassandra.auth.IAuthorizer;
 import org.apache.cassandra.auth.INetworkAuthorizer;
 import org.apache.cassandra.auth.IRoleManager;
-import org.apache.cassandra.cql3.statements.CreateTableStatement;
-import org.apache.cassandra.dht.Murmur3Partitioner;
-import org.apache.cassandra.index.sasi.SASIIndex;
-import org.apache.cassandra.index.sasi.disk.OnDiskIndexBuilder;
-import org.junit.After;
-import org.junit.BeforeClass;
-
 import org.apache.cassandra.config.*;
 import org.apache.cassandra.cql3.CQLTester;
 import org.apache.cassandra.cql3.ColumnIdentifier;
-import org.apache.cassandra.cql3.statements.IndexTarget;
+import org.apache.cassandra.cql3.statements.schema.CreateTableStatement;
+import org.apache.cassandra.cql3.statements.schema.IndexTarget;
 import org.apache.cassandra.db.RowUpdateBuilder;
 import org.apache.cassandra.db.commitlog.CommitLog;
 import org.apache.cassandra.db.marshal.*;
+import org.apache.cassandra.dht.Murmur3Partitioner;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.gms.Gossiper;
 import org.apache.cassandra.index.StubIndex;
+import org.apache.cassandra.index.sasi.SASIIndex;
+import org.apache.cassandra.index.sasi.disk.OnDiskIndexBuilder;
 import org.apache.cassandra.io.util.FileUtils;
 import org.apache.cassandra.schema.*;
 import org.apache.cassandra.schema.MigrationManager;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
 
+import org.junit.After;
+import org.junit.BeforeClass;
+
 public class SchemaLoader
 {
     @BeforeClass

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/test/unit/org/apache/cassandra/auth/CassandraNetworkAuthorizerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/auth/CassandraNetworkAuthorizerTest.java b/test/unit/org/apache/cassandra/auth/CassandraNetworkAuthorizerTest.java
index f0eed8c..bf50094 100644
--- a/test/unit/org/apache/cassandra/auth/CassandraNetworkAuthorizerTest.java
+++ b/test/unit/org/apache/cassandra/auth/CassandraNetworkAuthorizerTest.java
@@ -59,7 +59,7 @@ public class CassandraNetworkAuthorizerTest
     {
         ResultMessage.Rows select(SelectStatement statement, QueryOptions options)
         {
-            return statement.executeInternal(QueryState.forInternalCalls(), options);
+            return statement.executeLocally(QueryState.forInternalCalls(), options);
         }
 
         UntypedResultSet process(String query) throws RequestExecutionException
@@ -70,7 +70,7 @@ public class CassandraNetworkAuthorizerTest
         @Override
         void processBatch(BatchStatement statement)
         {
-            statement.executeInternal(QueryState.forInternalCalls(), QueryOptions.DEFAULT);
+            statement.executeLocally(QueryState.forInternalCalls(), QueryOptions.DEFAULT);
         }
     }
 
@@ -78,7 +78,7 @@ public class CassandraNetworkAuthorizerTest
     {
         ResultMessage.Rows select(SelectStatement statement, QueryOptions options)
         {
-            return statement.executeInternal(QueryState.forInternalCalls(), options);
+            return statement.executeLocally(QueryState.forInternalCalls(), options);
         }
 
         UntypedResultSet process(String query, ConsistencyLevel consistencyLevel) throws RequestValidationException, RequestExecutionException
@@ -91,7 +91,7 @@ public class CassandraNetworkAuthorizerTest
     {
         ResultMessage.Rows select(SelectStatement statement, QueryOptions options)
         {
-            return statement.executeInternal(QueryState.forInternalCalls(), options);
+            return statement.executeLocally(QueryState.forInternalCalls(), options);
         }
 
         void process(String query)
@@ -170,7 +170,7 @@ public class CassandraNetworkAuthorizerTest
 
     private static void auth(String query, Object... args)
     {
-        CQLStatement statement = QueryProcessor.parseStatement(String.format(query, args)).prepare().statement;
+        CQLStatement statement = QueryProcessor.parseStatement(String.format(query, args)).prepare(ClientState.forInternalCalls());
         assert statement instanceof CreateRoleStatement
                || statement instanceof AlterRoleStatement
                || statement instanceof DropRoleStatement;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/test/unit/org/apache/cassandra/cql3/CQLTester.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/CQLTester.java b/test/unit/org/apache/cassandra/cql3/CQLTester.java
index 662e804..4a1a365 100644
--- a/test/unit/org/apache/cassandra/cql3/CQLTester.java
+++ b/test/unit/org/apache/cassandra/cql3/CQLTester.java
@@ -52,7 +52,6 @@ import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.schema.*;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.cql3.functions.FunctionName;
-import org.apache.cassandra.cql3.statements.ParsedStatement;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.commitlog.CommitLog;
 import org.apache.cassandra.db.marshal.*;
@@ -62,6 +61,8 @@ import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.exceptions.SyntaxException;
 import org.apache.cassandra.io.util.FileUtils;
 import org.apache.cassandra.locator.AbstractEndpointSnitch;
+import org.apache.cassandra.schema.IndexMetadata;
+import org.apache.cassandra.schema.KeyspaceMetadata;
 import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.serializers.TypeSerializer;
 import org.apache.cassandra.service.ClientState;
@@ -713,7 +714,13 @@ public abstract class CQLTester
             throw new IllegalArgumentException("Table name should be specified: " + formattedQuery);
 
         String column = matcher.group(9);
-        return Indexes.getAvailableIndexName(keyspace, table, Strings.isNullOrEmpty(column) ? null : column);
+
+        String baseName = Strings.isNullOrEmpty(column)
+                        ? IndexMetadata.generateDefaultIndexName(table)
+                        : IndexMetadata.generateDefaultIndexName(table, new ColumnIdentifier(column, true));
+
+        KeyspaceMetadata ks = Schema.instance.getKeyspaceMetadata(keyspace);
+        return ks.findAvailableIndexName(baseName);
     }
 
     /**
@@ -809,16 +816,15 @@ public abstract class CQLTester
     {
         try
         {
-            ClientState state = ClientState.forInternalCalls();
-            state.setKeyspace(SchemaConstants.SYSTEM_KEYSPACE_NAME);
+            ClientState state = ClientState.forInternalCalls(SchemaConstants.SYSTEM_KEYSPACE_NAME);
             QueryState queryState = new QueryState(state);
 
-            ParsedStatement.Prepared prepared = QueryProcessor.parseStatement(query, queryState.getClientState());
-            prepared.statement.validate(state);
+            CQLStatement statement = QueryProcessor.parseStatement(query, queryState.getClientState());
+            statement.validate(state);
 
             QueryOptions options = QueryOptions.forInternalCalls(Collections.<ByteBuffer>emptyList());
 
-            lastSchemaChangeResult = prepared.statement.executeInternal(queryState, options);
+            lastSchemaChangeResult = statement.executeLocally(queryState, options);
         }
         catch (Exception e)
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/test/unit/org/apache/cassandra/cql3/PstmtPersistenceTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/PstmtPersistenceTest.java b/test/unit/org/apache/cassandra/cql3/PstmtPersistenceTest.java
index 2b5a14a..eca6c20 100644
--- a/test/unit/org/apache/cassandra/cql3/PstmtPersistenceTest.java
+++ b/test/unit/org/apache/cassandra/cql3/PstmtPersistenceTest.java
@@ -22,11 +22,10 @@ import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
 
+import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 
-import org.junit.Assert;
-import org.apache.cassandra.cql3.statements.ParsedStatement;
 import org.apache.cassandra.db.SystemKeyspace;
 import org.apache.cassandra.db.marshal.Int32Type;
 import org.apache.cassandra.db.marshal.UTF8Type;
@@ -99,7 +98,7 @@ public class PstmtPersistenceTest extends CQLTester
         for (UntypedResultSet.Row row : QueryProcessor.executeOnceInternal(queryAll))
         {
             MD5Digest digest = MD5Digest.wrap(ByteBufferUtil.getArray(row.getBytes("prepared_id")));
-            ParsedStatement.Prepared prepared = QueryProcessor.instance.getPrepared(digest);
+            QueryProcessor.Prepared prepared = QueryProcessor.instance.getPrepared(digest);
             Assert.assertNotNull(prepared);
         }
 
@@ -128,8 +127,8 @@ public class PstmtPersistenceTest extends CQLTester
 
     private static void validatePstmt(QueryHandler handler, MD5Digest stmtId, QueryOptions options)
     {
-        ParsedStatement.Prepared prepared = handler.getPrepared(stmtId);
-        assertNotNull(prepared);
+        QueryProcessor.Prepared prepared = handler.getPrepared(stmtId);
+        Assert.assertNotNull(prepared);
         handler.processPrepared(prepared.statement, QueryState.forInternalCalls(), options, Collections.emptyMap(), System.nanoTime());
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/test/unit/org/apache/cassandra/cql3/ViewComplexTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/ViewComplexTest.java b/test/unit/org/apache/cassandra/cql3/ViewComplexTest.java
index f6defa8..32319a9 100644
--- a/test/unit/org/apache/cassandra/cql3/ViewComplexTest.java
+++ b/test/unit/org/apache/cassandra/cql3/ViewComplexTest.java
@@ -144,7 +144,7 @@ public class ViewComplexTest extends CQLTester
         executeNet(protocolVersion, "USE " + keyspace());
         createTable("CREATE TABLE %s (k int, c int, a int, b int, e int, f int, PRIMARY KEY (k, c))");
         createView("mv",
-                   "CREATE MATERIALIZED VIEW %s AS SELECT a, b FROM %%s WHERE k IS NOT NULL AND c IS NOT NULL PRIMARY KEY (k,c)");
+                   "CREATE MATERIALIZED VIEW %s AS SELECT a, b, c, k FROM %%s WHERE k IS NOT NULL AND c IS NOT NULL PRIMARY KEY (k,c)");
         Keyspace ks = Keyspace.open(keyspace());
         ks.getColumnFamilyStore("mv").disableAutoCompaction();
 
@@ -401,7 +401,7 @@ public class ViewComplexTest extends CQLTester
         assertRowsIgnoringOrder(execute("SELECT * from %s WHERE c = ? AND p = ?", 0, 0), row(0, 0, null, 1));
         assertRowsIgnoringOrder(execute("SELECT * from mv WHERE c = ? AND p = ?", 0, 0), row(0, 0));
 
-        assertInvalidMessage(String.format("Cannot drop column v2 on base table %s with materialized views.", baseTable), "ALTER TABLE %s DROP v2");
+        assertInvalidMessage(String.format("Cannot drop column v2 on base table %s with materialized views", baseTable), "ALTER TABLE %s DROP v2");
         // // drop unselected base column, unselected metadata should be removed, thus view row is dead
         // updateView("ALTER TABLE %s DROP v2");
         // assertRowsIgnoringOrder(execute("SELECT * from %s WHERE c = ? AND p = ?", 0, 0));
@@ -428,7 +428,7 @@ public class ViewComplexTest extends CQLTester
         executeNet(protocolVersion, "USE " + keyspace());
         String baseTable = createTable("CREATE TABLE %s (k int, c int, a int, b int, l list<int>, s set<int>, m map<int,int>, PRIMARY KEY (k, c))");
         createView("mv",
-                   "CREATE MATERIALIZED VIEW %s AS SELECT a, b FROM %%s WHERE k IS NOT NULL AND c IS NOT NULL PRIMARY KEY (c, k)");
+                   "CREATE MATERIALIZED VIEW %s AS SELECT a, b, c, k FROM %%s WHERE k IS NOT NULL AND c IS NOT NULL PRIMARY KEY (c, k)");
         Keyspace ks = Keyspace.open(keyspace());
         ks.getColumnFamilyStore("mv").disableAutoCompaction();
 
@@ -462,7 +462,7 @@ public class ViewComplexTest extends CQLTester
         assertRowsIgnoringOrder(execute("SELECT k,c,a,b from %s"), row(1, 1, null, null));
         assertRowsIgnoringOrder(execute("SELECT * from mv"), row(1, 1, null, null));
 
-        assertInvalidMessage(String.format("Cannot drop column m on base table %s with materialized views.", baseTable), "ALTER TABLE %s DROP m");
+        assertInvalidMessage(String.format("Cannot drop column m on base table %s with materialized views", baseTable), "ALTER TABLE %s DROP m");
         // executeNet(protocolVersion, "ALTER TABLE %s DROP m");
         // ks.getColumnFamilyStore("mv").forceMajorCompaction();
         // assertRowsIgnoringOrder(execute("SELECT k,c,a,b from %s WHERE k = 1 AND c = 1"));
@@ -931,7 +931,7 @@ public class ViewComplexTest extends CQLTester
         assertRowsIgnoringOrder(execute("SELECT k,a,b from mv"), row(1, 1, 2));
         assertRowsIgnoringOrder(execute("SELECT k,a,b from %s"), row(1, 1, 2));
 
-        assertInvalidMessage(String.format("Cannot drop column a on base table %s with materialized views.", baseTable), "ALTER TABLE %s DROP a");
+        assertInvalidMessage(String.format("Cannot drop column a on base table %s with materialized views", baseTable), "ALTER TABLE %s DROP a");
     }
 
     @Test
@@ -1273,7 +1273,7 @@ public class ViewComplexTest extends CQLTester
                                                   // all selected
                                                   "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE a IS NOT NULL AND b IS NOT NULL PRIMARY KEY (a,b)",
                                                   // unselected e,f
-                                                  "CREATE MATERIALIZED VIEW %s AS SELECT c,d FROM %%s WHERE a IS NOT NULL AND b IS NOT NULL PRIMARY KEY (a,b)",
+                                                  "CREATE MATERIALIZED VIEW %s AS SELECT a,b,c,d FROM %%s WHERE a IS NOT NULL AND b IS NOT NULL PRIMARY KEY (a,b)",
                                                   // no selected
                                                   "CREATE MATERIALIZED VIEW %s AS SELECT a,b FROM %%s WHERE a IS NOT NULL AND b IS NOT NULL PRIMARY KEY (a,b)",
                                                   // all selected, re-order keys

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/test/unit/org/apache/cassandra/cql3/ViewFilteringTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/ViewFilteringTest.java b/test/unit/org/apache/cassandra/cql3/ViewFilteringTest.java
index b7d3647..65d66b5 100644
--- a/test/unit/org/apache/cassandra/cql3/ViewFilteringTest.java
+++ b/test/unit/org/apache/cassandra/cql3/ViewFilteringTest.java
@@ -1848,7 +1848,7 @@ public class ViewFilteringTest extends CQLTester
         executeNet(protocolVersion, "USE " + keyspace());
 
         try {
-            createView("mv_test", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE b IS NOT NULL AND c IS NOT NULL AND d = 1 PRIMARY KEY (a, b, c)");
+            createView("mv_test", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE a IS NOT NULL AND b IS NOT NULL AND c IS NOT NULL AND d = 1 PRIMARY KEY (a, b, c)");
             dropView("mv_test");
         } catch(Exception e) {
             throw new RuntimeException("MV creation with non primary column restrictions failed.", e);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/test/unit/org/apache/cassandra/cql3/ViewSchemaTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/ViewSchemaTest.java b/test/unit/org/apache/cassandra/cql3/ViewSchemaTest.java
index 63ac3e3..093f122 100644
--- a/test/unit/org/apache/cassandra/cql3/ViewSchemaTest.java
+++ b/test/unit/org/apache/cassandra/cql3/ViewSchemaTest.java
@@ -672,7 +672,7 @@ public class ViewSchemaTest extends CQLTester
         executeNet(protocolVersion, "USE " + keyspace());
 
         createView(keyspace() + ".mv1",
-                   "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE b IS NOT NULL AND c IS NOT NULL PRIMARY KEY (a, b, c)");
+                   "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE a IS NOT NULL AND b IS NOT NULL AND c IS NOT NULL PRIMARY KEY (a, b, c)");
 
         try
         {
@@ -681,7 +681,7 @@ public class ViewSchemaTest extends CQLTester
         }
         catch (InvalidQueryException e)
         {
-            Assert.assertEquals("Cannot use DROP TABLE on Materialized View", e.getMessage());
+            Assert.assertEquals("Cannot use DROP TABLE on a materialized view. Please use DROP MATERIALIZED VIEW instead.", e.getMessage());
         }
     }
 
@@ -694,7 +694,7 @@ public class ViewSchemaTest extends CQLTester
 
         executeNet(protocolVersion, "USE " + keyspace());
 
-        assertInvalidMessage("Non-primary key columns cannot be restricted in the SELECT statement used for materialized view creation",
+        assertInvalidMessage("Non-primary key columns can only be restricted with 'IS NOT NULL'",
                              "CREATE MATERIALIZED VIEW " + keyspace() + ".mv AS SELECT * FROM %s "
                                      + "WHERE b IS NOT NULL AND c IS NOT NULL AND a IS NOT NULL "
                                      + "AND d = 1 PRIMARY KEY (c, b, a)");

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/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 40d75c4..aae7d83 100644
--- a/test/unit/org/apache/cassandra/cql3/ViewTest.java
+++ b/test/unit/org/apache/cassandra/cql3/ViewTest.java
@@ -93,8 +93,8 @@ public class ViewTest extends CQLTester
     @Test
     public void testNonExistingOnes() throws Throwable
     {
-        assertInvalidMessage("Cannot drop non existing materialized view", "DROP MATERIALIZED VIEW " + KEYSPACE + ".view_does_not_exist");
-        assertInvalidMessage("Cannot drop non existing materialized view", "DROP MATERIALIZED VIEW keyspace_does_not_exist.view_does_not_exist");
+        assertInvalidMessage(String.format("Materialized view '%s.view_does_not_exist' doesn't exist", KEYSPACE), "DROP MATERIALIZED VIEW " + KEYSPACE + ".view_does_not_exist");
+        assertInvalidMessage("Materialized view 'keyspace_does_not_exist.view_does_not_exist' doesn't exist", "DROP MATERIALIZED VIEW keyspace_does_not_exist.view_does_not_exist");
 
         execute("DROP MATERIALIZED VIEW IF EXISTS " + KEYSPACE + ".view_does_not_exist");
         execute("DROP MATERIALIZED VIEW IF EXISTS keyspace_does_not_exist.view_does_not_exist");
@@ -156,7 +156,7 @@ public class ViewTest extends CQLTester
         execute("USE " + keyspace());
         executeNet(protocolVersion, "USE " + keyspace());
 
-        createView("view1", "CREATE MATERIALIZED VIEW view1 AS SELECT k1 FROM %%s WHERE k1 IS NOT NULL AND c1 IS NOT NULL AND val IS NOT NULL PRIMARY KEY (val, k1, c1)");
+        createView("view1", "CREATE MATERIALIZED VIEW view1 AS SELECT k1, c1, val FROM %%s WHERE k1 IS NOT NULL AND c1 IS NOT NULL AND val IS NOT NULL PRIMARY KEY (val, k1, c1)");
 
         updateView("INSERT INTO %s (k1, c1, val) VALUES (1, 2, 200)");
         updateView("INSERT INTO %s (k1, c1, val) VALUES (1, 3, 300)");
@@ -178,7 +178,7 @@ public class ViewTest extends CQLTester
         execute("USE " + keyspace());
         executeNet(protocolVersion, "USE " + keyspace());
 
-        createView("view1", "CREATE MATERIALIZED VIEW view1 AS SELECT k1 FROM %%s WHERE k1 IS NOT NULL AND c1 IS NOT NULL AND val IS NOT NULL PRIMARY KEY (val, k1, c1)");
+        createView("view1", "CREATE MATERIALIZED VIEW view1 AS SELECT val, k1, c1 FROM %%s WHERE k1 IS NOT NULL AND c1 IS NOT NULL AND val IS NOT NULL PRIMARY KEY (val, k1, c1)");
 
     }
 
@@ -190,7 +190,7 @@ public class ViewTest extends CQLTester
         execute("USE " + keyspace());
         executeNet(protocolVersion, "USE " + keyspace());
 
-        createView("view1", "CREATE MATERIALIZED VIEW view1 AS SELECT k1 FROM %%s WHERE k1 IS NOT NULL AND c1 IS NOT NULL AND val IS NOT NULL PRIMARY KEY (val, k1, c1)");
+        createView("view1", "CREATE MATERIALIZED VIEW view1 AS SELECT k1, c1, val FROM %%s WHERE k1 IS NOT NULL AND c1 IS NOT NULL AND val IS NOT NULL PRIMARY KEY (val, k1, c1)");
 
         updateView("INSERT INTO %s (k1, c1, val) VALUES (1, 2, 200)");
         updateView("INSERT INTO %s (k1, c1, val) VALUES (1, 3, 300)");
@@ -252,8 +252,15 @@ public class ViewTest extends CQLTester
         {
         }
 
-        // Can omit "k IS NOT NULL" because we have a sinlge partition key
-        createView("mv_test", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE bigintval IS NOT NULL AND asciival IS NOT NULL PRIMARY KEY (bigintval, k, asciival)");
+        // Must still include both even when the partition key is composite
+        try
+        {
+            createView("mv_test", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE bigintval IS NOT NULL AND asciival IS NOT NULL PRIMARY KEY (bigintval, k, asciival)");
+            Assert.fail("Should fail if compound primary is not completely filtered as NOT NULL");
+        }
+        catch (Exception e)
+        {
+        }
     }
 
     @Test
@@ -418,7 +425,7 @@ public class ViewTest extends CQLTester
         }
         catch (InvalidQueryException e)
         {
-            Assert.assertEquals("Cannot use Duration column 'result' in PRIMARY KEY of materialized view", e.getMessage());
+            Assert.assertEquals("duration type is not supported for PRIMARY KEY column 'result'", e.getMessage());
         }
     }
 
@@ -760,7 +767,7 @@ public class ViewTest extends CQLTester
             {
                 String query = "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE " + def.name + " IS NOT NULL AND k IS NOT NULL "
                                + (def.name.toString().equals("asciival") ? "" : "AND asciival IS NOT NULL ") + "PRIMARY KEY ((" + def.name + ", k), nonexistentcolumn)";
-                createView("mv3_" + def.name, query);
+                createView("mv4_" + def.name, query);
                 Assert.fail("Should fail with unknown base column");
             }
             catch (InvalidQueryException e)
@@ -1023,10 +1030,10 @@ public class ViewTest extends CQLTester
 
         executeNet(protocolVersion, "USE " + keyspace());
 
-        createView("mv1", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE b IS NOT NULL AND c IS NOT NULL PRIMARY KEY (a, b, c) WITH CLUSTERING ORDER BY (b DESC)");
-        createView("mv2", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE b IS NOT NULL AND c IS NOT NULL PRIMARY KEY (a, c, b) WITH CLUSTERING ORDER BY (c ASC)");
-        createView("mv3", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE b IS NOT NULL AND c IS NOT NULL PRIMARY KEY (a, b, c)");
-        createView("mv4", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE b IS NOT NULL AND c IS NOT NULL PRIMARY KEY (a, c, b) WITH CLUSTERING ORDER BY (c DESC)");
+        createView("mv1", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE a IS NOT NULL AND b IS NOT NULL AND c IS NOT NULL PRIMARY KEY (a, b, c) WITH CLUSTERING ORDER BY (b DESC, c ASC)");
+        createView("mv2", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE a IS NOT NULL AND b IS NOT NULL AND c IS NOT NULL PRIMARY KEY (a, c, b) WITH CLUSTERING ORDER BY (c ASC, b ASC)");
+        createView("mv3", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE a IS NOT NULL AND b IS NOT NULL AND c IS NOT NULL PRIMARY KEY (a, b, c)");
+        createView("mv4", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE a IS NOT NULL AND b IS NOT NULL AND c IS NOT NULL PRIMARY KEY (a, c, b) WITH CLUSTERING ORDER BY (c DESC, b ASC)");
 
         updateView("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 1, 1, 1, 1);
         updateView("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 1, 2, 2, 2);
@@ -1062,7 +1069,7 @@ public class ViewTest extends CQLTester
 
         executeNet(protocolVersion, "USE " + keyspace());
 
-        createView("mv1", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE b IS NOT NULL PRIMARY KEY (b, a)");
+        createView("mv1", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE a IS NOT NULL AND b IS NOT NULL PRIMARY KEY (b, a)");
 
         updateView("INSERT INTO %s (a, b) VALUES (?, ?)", 1, 1);
         updateView("INSERT INTO %s (a, b) VALUES (?, ?)", 1, 2);
@@ -1094,7 +1101,7 @@ public class ViewTest extends CQLTester
         executeNet(protocolVersion, "USE " + keyspace());
 
         // Cannot use SELECT *, as those are always handled by the includeAll shortcut in View.updateAffectsView
-        createView("mv1", "CREATE MATERIALIZED VIEW %s AS SELECT a, b FROM %%s WHERE b IS NOT NULL PRIMARY KEY (b, a)");
+        createView("mv1", "CREATE MATERIALIZED VIEW %s AS SELECT a, b FROM %%s WHERE a IS NOT NULL AND b IS NOT NULL PRIMARY KEY (b, a)");
 
         updateView("INSERT INTO %s (a, b) VALUES (?, ?)", 1, 1);
 
@@ -1183,7 +1190,7 @@ public class ViewTest extends CQLTester
                     "PRIMARY KEY (a))");
 
         executeNet(protocolVersion, "USE " + keyspace());
-        createView("mvmap", "CREATE MATERIALIZED VIEW %s AS SELECT a, b FROM %%s WHERE b IS NOT NULL PRIMARY KEY (b, a)");
+        createView("mvmap", "CREATE MATERIALIZED VIEW %s AS SELECT a, b FROM %%s WHERE a IS NOT NULL AND b IS NOT NULL PRIMARY KEY (b, a)");
 
         updateView("INSERT INTO %s (a, b) VALUES (?, ?)", 0, 0);
         ResultSet mvRows = executeNet(protocolVersion, "SELECT a, b FROM mvmap WHERE b = ?", 0);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/test/unit/org/apache/cassandra/cql3/selection/SelectionColumnMappingTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/selection/SelectionColumnMappingTest.java b/test/unit/org/apache/cassandra/cql3/selection/SelectionColumnMappingTest.java
index 30fbb0d..d6de5ff 100644
--- a/test/unit/org/apache/cassandra/cql3/selection/SelectionColumnMappingTest.java
+++ b/test/unit/org/apache/cassandra/cql3/selection/SelectionColumnMappingTest.java
@@ -124,7 +124,7 @@ public class SelectionColumnMappingTest extends CQLTester
         // we don't use verify like with the other tests because this query will produce no results
         SelectStatement statement = getSelect("SELECT token(a,b) FROM %s");
         verifyColumnMapping(expected, statement);
-        statement.executeInternal(QueryState.forInternalCalls(), QueryOptions.DEFAULT);
+        statement.executeLocally(QueryState.forInternalCalls(), QueryOptions.DEFAULT);
     }
 
     private void testSimpleTypes() throws Throwable
@@ -581,8 +581,8 @@ public class SelectionColumnMappingTest extends CQLTester
     private void checkExecution(SelectStatement statement, List<ColumnSpecification> expectedResultColumns)
     throws RequestExecutionException, RequestValidationException
     {
-        UntypedResultSet rs = UntypedResultSet.create(statement.executeInternal(QueryState.forInternalCalls(),
-                                                                                QueryOptions.DEFAULT).result);
+        UntypedResultSet rs = UntypedResultSet.create(statement.executeLocally(QueryState.forInternalCalls(),
+                                                                               QueryOptions.DEFAULT).result);
 
         assertEquals(expectedResultColumns, rs.one().getColumns());
     }
@@ -590,7 +590,7 @@ public class SelectionColumnMappingTest extends CQLTester
     private SelectStatement getSelect(String query) throws RequestValidationException
     {
         CQLStatement statement = QueryProcessor.getStatement(String.format(query, KEYSPACE + "." + tableName),
-                                                             ClientState.forInternalCalls()).statement;
+                                                             ClientState.forInternalCalls());
         assertTrue(statement instanceof SelectStatement);
         return (SelectStatement)statement;
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/test/unit/org/apache/cassandra/cql3/statements/AlterRoleStatementTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/statements/AlterRoleStatementTest.java b/test/unit/org/apache/cassandra/cql3/statements/AlterRoleStatementTest.java
index 77e5236..883b4f5 100644
--- a/test/unit/org/apache/cassandra/cql3/statements/AlterRoleStatementTest.java
+++ b/test/unit/org/apache/cassandra/cql3/statements/AlterRoleStatementTest.java
@@ -23,13 +23,14 @@ import org.junit.Assert;
 import org.junit.Test;
 
 import org.apache.cassandra.auth.DCPermissions;
+import org.apache.cassandra.cql3.CQLStatement;
 import org.apache.cassandra.cql3.QueryProcessor;
 
 public class AlterRoleStatementTest
 {
     private static AlterRoleStatement parse(String query)
     {
-        ParsedStatement stmt = QueryProcessor.parseStatement(query);
+        CQLStatement.Raw stmt = QueryProcessor.parseStatement(query);
         Assert.assertTrue(stmt instanceof AlterRoleStatement);
         return (AlterRoleStatement) stmt;
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/test/unit/org/apache/cassandra/cql3/statements/CreateRoleStatementTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/statements/CreateRoleStatementTest.java b/test/unit/org/apache/cassandra/cql3/statements/CreateRoleStatementTest.java
index 0ff26b5..7a23da2 100644
--- a/test/unit/org/apache/cassandra/cql3/statements/CreateRoleStatementTest.java
+++ b/test/unit/org/apache/cassandra/cql3/statements/CreateRoleStatementTest.java
@@ -23,6 +23,7 @@ import org.junit.Assert;
 import org.junit.Test;
 
 import org.apache.cassandra.auth.DCPermissions;
+import org.apache.cassandra.cql3.CQLStatement;
 import org.apache.cassandra.cql3.CQLTester;
 import org.apache.cassandra.cql3.QueryProcessor;
 
@@ -31,7 +32,7 @@ public class CreateRoleStatementTest extends CQLTester
 
     private static CreateRoleStatement parse(String query)
     {
-        ParsedStatement stmt = QueryProcessor.parseStatement(query);
+        CQLStatement.Raw stmt = QueryProcessor.parseStatement(query);
         Assert.assertTrue(stmt instanceof CreateRoleStatement);
         return (CreateRoleStatement) stmt;
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/test/unit/org/apache/cassandra/cql3/statements/CreateUserStatementTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/statements/CreateUserStatementTest.java b/test/unit/org/apache/cassandra/cql3/statements/CreateUserStatementTest.java
index 51e38b3..51c61bb 100644
--- a/test/unit/org/apache/cassandra/cql3/statements/CreateUserStatementTest.java
+++ b/test/unit/org/apache/cassandra/cql3/statements/CreateUserStatementTest.java
@@ -22,13 +22,14 @@ import org.junit.Assert;
 import org.junit.Test;
 
 import org.apache.cassandra.auth.DCPermissions;
+import org.apache.cassandra.cql3.CQLStatement;
 import org.apache.cassandra.cql3.QueryProcessor;
 
 public class CreateUserStatementTest
 {
     private static CreateRoleStatement parse(String query)
     {
-        ParsedStatement stmt = QueryProcessor.parseStatement(query);
+        CQLStatement.Raw stmt = QueryProcessor.parseStatement(query);
         Assert.assertTrue(stmt instanceof CreateRoleStatement);
         return (CreateRoleStatement) stmt;
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/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 9ad47c0..54dd0c4 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/entities/CollectionsTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/CollectionsTest.java
@@ -684,11 +684,11 @@ public class CollectionsTest extends CQLTester
     @Test
     public void testDropAndReaddDroppedCollection() throws Throwable
     {
-        createTable("create table %s (k int primary key, v frozen<set<text>>, x int)");
+        createTable("create table %s (k int primary key, v set<text>, x int)");
         execute("insert into %s (k, v) VALUES (0, {'fffffffff'})");
         flush();
         execute("alter table %s drop v");
-        execute("alter table %s add v set<int>");
+        execute("alter table %s add v set<text>");
     }
 
     @Test
@@ -1652,9 +1652,9 @@ public class CollectionsTest extends CQLTester
     public void testCollectionSliceOnMV() throws Throwable
     {
         createTable("CREATE TABLE %s (k int, c int, l text, m map<text, text>, o int, PRIMARY KEY (k, c))");
-        assertInvalidMessage("Cannot use collection element selection when defining a materialized view",
+        assertInvalidMessage("Can only select columns by name when defining a materialized view (got m['abc'])",
                              "CREATE MATERIALIZED VIEW " + KEYSPACE + ".view1 AS SELECT m['abc'] FROM %s WHERE k IS NOT NULL AND c IS NOT NULL AND m IS NOT NULL PRIMARY KEY (c, k)");
-        assertInvalidMessage("Cannot use collection slice selection when defining a materialized view",
+        assertInvalidMessage("Can only select columns by name when defining a materialized view (got m['abc'..'def'])",
                              "CREATE MATERIALIZED VIEW " + KEYSPACE + ".view1 AS SELECT m['abc'..'def'] FROM %s WHERE k IS NOT NULL AND c IS NOT NULL AND m IS NOT NULL PRIMARY KEY (c, k)");
     }
 
@@ -1762,7 +1762,7 @@ public class CollectionsTest extends CQLTester
         String type = createType("CREATE TYPE %s (s set<int>, m map<text, text>)");
 
         assertInvalidMessage("Non-frozen UDTs are not allowed inside collections",
-                             "CREATE TABLE " + KEYSPACE + "t (k int PRIMARY KEY, v map<text, " + type + ">)");
+                             "CREATE TABLE " + KEYSPACE + ".t (k int PRIMARY KEY, v map<text, " + type + ">)");
 
         String mapType = "map<text, frozen<" + type + ">>";
         for (boolean frozen : new boolean[]{false, true})

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/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 f4543ea..38bd4dd 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/entities/CountersTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/CountersTest.java
@@ -160,7 +160,7 @@ public class CountersTest extends CQLTester
     @Test
     public void testProhibitReversedCounterAsPartOfPrimaryKey() throws Throwable
     {
-        assertInvalidThrowMessage("counter type is not supported for PRIMARY KEY part a",
+        assertInvalidThrowMessage("counter type is not supported for PRIMARY KEY column '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()));
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/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 c493267..39ae392 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/entities/FrozenCollectionsTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/FrozenCollectionsTest.java
@@ -554,8 +554,8 @@ public class FrozenCollectionsTest extends CQLTester
         createTable("CREATE TABLE %s (a frozen<map<int, text>> PRIMARY KEY, b frozen<map<int, text>>)");
 
         // for now, we don't support indexing values or keys of collections in the primary key
-        assertInvalidIndexCreationWithMessage("CREATE INDEX ON %s (full(a))", "Cannot create secondary index on partition key column");
-        assertInvalidIndexCreationWithMessage("CREATE INDEX ON %s (keys(a))", "Cannot create secondary index on partition key column");
+        assertInvalidIndexCreationWithMessage("CREATE INDEX ON %s (full(a))", "Cannot create secondary index on the only partition key column");
+        assertInvalidIndexCreationWithMessage("CREATE INDEX ON %s (keys(a))", "Cannot create secondary index on the only partition key column");
         assertInvalidIndexCreationWithMessage("CREATE INDEX ON %s (keys(b))", "Cannot create keys() index on frozen column b. " +
                                                                               "Frozen collections only support full() indexes");
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/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 c763b9b..2b2880d 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/entities/SecondaryIndexTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/SecondaryIndexTest.java
@@ -25,6 +25,7 @@ import java.util.concurrent.CountDownLatch;
 import org.apache.commons.lang3.StringUtils;
 import org.junit.Test;
 
+import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.config.DatabaseDescriptor;
@@ -32,13 +33,12 @@ import org.apache.cassandra.cql3.CQLTester;
 import org.apache.cassandra.cql3.ColumnIdentifier;
 import org.apache.cassandra.cql3.QueryProcessor;
 import org.apache.cassandra.cql3.restrictions.StatementRestrictions;
-import org.apache.cassandra.cql3.statements.IndexTarget;
+import org.apache.cassandra.cql3.statements.schema.IndexTarget;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.DeletionTime;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.db.rows.Cell;
 import org.apache.cassandra.db.rows.Row;
-import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.exceptions.SyntaxException;
 import org.apache.cassandra.index.IndexNotAvailableException;
 import org.apache.cassandra.index.SecondaryIndexManager;
@@ -52,6 +52,8 @@ import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.MD5Digest;
 import org.apache.cassandra.utils.Pair;
 
+import static java.lang.String.format;
+
 import static org.apache.cassandra.Util.throwAssert;
 import static org.apache.cassandra.utils.ByteBufferUtil.EMPTY_BYTE_BUFFER;
 import static org.apache.cassandra.utils.ByteBufferUtil.bytes;
@@ -96,17 +98,17 @@ public class SecondaryIndexTest extends CQLTester
      */
     private void testCreateAndDropIndex(String indexName, boolean addKeyspaceOnDrop) throws Throwable
     {
-        execute("USE system");
-        assertInvalidMessage(String.format("Index '%s' could not be found",
-                                           removeQuotes(indexName.toLowerCase(Locale.US))),
-                             "DROP INDEX " + indexName + ";");
+        assertInvalidMessage(format("Index '%s.%s' doesn't exist",
+                                    KEYSPACE,
+                                    removeQuotes(indexName.toLowerCase(Locale.US))),
+                             format("DROP INDEX %s.%s", KEYSPACE, indexName));
 
         createTable("CREATE TABLE %s (a int primary key, b int);");
         createIndex("CREATE INDEX " + indexName + " ON %s(b);");
         createIndex("CREATE INDEX IF NOT EXISTS " + indexName + " ON %s(b);");
 
-        assertInvalidMessage(String.format("Index %s already exists",
-                                           removeQuotes(indexName.toLowerCase(Locale.US))),
+        assertInvalidMessage(format("Index '%s' already exists",
+                                    removeQuotes(indexName.toLowerCase(Locale.US))),
                              "CREATE INDEX " + indexName + " ON %s(b)");
 
         // IF NOT EXISTS should apply in cases where the new index differs from an existing one in name only
@@ -114,9 +116,9 @@ public class SecondaryIndexTest extends CQLTester
         assertEquals(1, getCurrentColumnFamilyStore().metadata().indexes.size());
         createIndex("CREATE INDEX IF NOT EXISTS " + otherIndexName + " ON %s(b)");
         assertEquals(1, getCurrentColumnFamilyStore().metadata().indexes.size());
-        assertInvalidMessage(String.format("Index %s is a duplicate of existing index %s",
-                                           removeQuotes(otherIndexName.toLowerCase(Locale.US)),
-                                           removeQuotes(indexName.toLowerCase(Locale.US))),
+        assertInvalidMessage(format("Index %s is a duplicate of existing index %s",
+                                    removeQuotes(otherIndexName.toLowerCase(Locale.US)),
+                                    removeQuotes(indexName.toLowerCase(Locale.US))),
                              "CREATE INDEX " + otherIndexName + " ON %s(b)");
 
         execute("INSERT INTO %s (a, b) values (?, ?);", 0, 0);
@@ -125,26 +127,24 @@ public class SecondaryIndexTest extends CQLTester
         execute("INSERT INTO %s (a, b) values (?, ?);", 3, 1);
 
         assertRows(execute("SELECT * FROM %s where b = ?", 1), row(1, 1), row(3, 1));
-        assertInvalidMessage(String.format("Index '%s' could not be found in any of the tables of keyspace 'system'",
-                                           removeQuotes(indexName.toLowerCase(Locale.US))),
-                             "DROP INDEX " + indexName);
 
         if (addKeyspaceOnDrop)
         {
-            dropIndex("DROP INDEX " + KEYSPACE + "." + indexName);
+            dropIndex(format("DROP INDEX %s.%s", KEYSPACE, indexName));
         }
         else
         {
             execute("USE " + KEYSPACE);
-            execute("DROP INDEX " + indexName);
+            execute(format("DROP INDEX %s", indexName));
         }
 
         assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
                              "SELECT * FROM %s where b = ?", 1);
-        dropIndex("DROP INDEX IF EXISTS " + indexName);
-        assertInvalidMessage(String.format("Index '%s' could not be found",
-                                           removeQuotes(indexName.toLowerCase(Locale.US))),
-                             "DROP INDEX " + indexName);
+        dropIndex(format("DROP INDEX IF EXISTS %s.%s", KEYSPACE, indexName));
+        assertInvalidMessage(format("Index '%s.%s' doesn't exist",
+                                    KEYSPACE,
+                                    removeQuotes(indexName.toLowerCase(Locale.US))),
+                             format("DROP INDEX %s.%s", KEYSPACE, indexName));
     }
 
     /**
@@ -239,10 +239,10 @@ public class SecondaryIndexTest extends CQLTester
     public void testUnknownCompressionOptions() throws Throwable
     {
         String tableName = createTableName();
-        assertInvalidThrow(SyntaxException.class, String.format("CREATE TABLE %s (key varchar PRIMARY KEY, password varchar, gender varchar) WITH compression_parameters:sstable_compressor = 'DeflateCompressor'", tableName));
+        assertInvalidThrow(SyntaxException.class, format("CREATE TABLE %s (key varchar PRIMARY KEY, password varchar, gender varchar) WITH compression_parameters:sstable_compressor = 'DeflateCompressor'", tableName));
 
-        assertInvalidThrow(ConfigurationException.class, String.format("CREATE TABLE %s (key varchar PRIMARY KEY, password varchar, gender varchar) WITH compression = { 'sstable_compressor': 'DeflateCompressor' }",
-                                                                       tableName));
+        assertInvalidThrow(ConfigurationException.class, format("CREATE TABLE %s (key varchar PRIMARY KEY, password varchar, gender varchar) WITH compression = { 'sstable_compressor': 'DeflateCompressor' }",
+                                                                tableName));
     }
 
     /**
@@ -558,14 +558,14 @@ public class SecondaryIndexTest extends CQLTester
     {
         String indexName = columnName + "_idx";
         SecondaryIndexManager indexManager = getCurrentColumnFamilyStore().indexManager;
-        createIndex(String.format("CREATE INDEX %s on %%s(%s)", indexName, columnName));
+        createIndex(format("CREATE INDEX %s on %%s(%s)", indexName, columnName));
         IndexMetadata indexDef = indexManager.getIndexByName(indexName).getIndexMetadata();
-        assertEquals(String.format("values(%s)", columnName), indexDef.options.get(IndexTarget.TARGET_OPTION_NAME));
-        dropIndex(String.format("DROP INDEX %s.%s", KEYSPACE, indexName));
+        assertEquals(format("values(%s)", columnName), indexDef.options.get(IndexTarget.TARGET_OPTION_NAME));
+        dropIndex(format("DROP INDEX %s.%s", KEYSPACE, indexName));
         assertFalse(indexManager.hasIndexes());
-        createIndex(String.format("CREATE INDEX %s on %%s(values(%s))", indexName, columnName));
+        createIndex(format("CREATE INDEX %s on %%s(values(%s))", indexName, columnName));
         assertEquals(indexDef, indexManager.getIndexByName(indexName).getIndexMetadata());
-        dropIndex(String.format("DROP INDEX %s.%s", KEYSPACE, indexName));
+        dropIndex(format("DROP INDEX %s.%s", KEYSPACE, indexName));
     }
 
     @Test
@@ -594,15 +594,15 @@ public class SecondaryIndexTest extends CQLTester
     private void createAndDropIndexWithQuotedColumnIdentifier(String target) throws Throwable
     {
         String indexName = "test_mixed_case_idx";
-        createIndex(String.format("CREATE INDEX %s ON %%s(%s)", indexName, target));
+        createIndex(format("CREATE INDEX %s ON %%s(%s)", indexName, target));
         SecondaryIndexManager indexManager = getCurrentColumnFamilyStore().indexManager;
         IndexMetadata indexDef = indexManager.getIndexByName(indexName).getIndexMetadata();
-        dropIndex(String.format("DROP INDEX %s.%s", KEYSPACE, indexName));
+        dropIndex(format("DROP INDEX %s.%s", KEYSPACE, indexName));
         // verify we can re-create the index using the target string
-        createIndex(String.format("CREATE INDEX %s ON %%s(%s)",
-                                  indexName, indexDef.options.get(IndexTarget.TARGET_OPTION_NAME)));
+        createIndex(format("CREATE INDEX %s ON %%s(%s)",
+                           indexName, indexDef.options.get(IndexTarget.TARGET_OPTION_NAME)));
         assertEquals(indexDef, indexManager.getIndexByName(indexName).getIndexMetadata());
-        dropIndex(String.format("DROP INDEX %s.%s", KEYSPACE, indexName));
+        dropIndex(format("DROP INDEX %s.%s", KEYSPACE, indexName));
     }
 
 
@@ -664,7 +664,7 @@ public class SecondaryIndexTest extends CQLTester
         // the indexed value passes validation, but the batch size will
         // exceed the default failure threshold, so temporarily raise it
         // (the non-conditional batch doesn't hit this because
-        // BatchStatement::executeInternal skips the size check but CAS
+        // BatchStatement::executeLocally skips the size check but CAS
         // path does not)
         long batchSizeThreshold = DatabaseDescriptor.getBatchSizeFailThreshold();
         try
@@ -717,7 +717,7 @@ public class SecondaryIndexTest extends CQLTester
         // the indexed value passes validation, but the batch size will
         // exceed the default failure threshold, so temporarily raise it
         // (the non-conditional batch doesn't hit this because
-        // BatchStatement::executeInternal skips the size check but CAS
+        // BatchStatement::executeLocally skips the size check but CAS
         // path does not)
         long batchSizeThreshold = DatabaseDescriptor.getBatchSizeFailThreshold();
         try
@@ -754,15 +754,15 @@ public class SecondaryIndexTest extends CQLTester
     public void prepareStatementsWithLIKEClauses() throws Throwable
     {
         createTable("CREATE TABLE %s (a int, c1 text, c2 text, v1 text, v2 text, v3 int, PRIMARY KEY (a, c1, c2))");
-        createIndex(String.format("CREATE CUSTOM INDEX c1_idx on %%s(c1) USING '%s' WITH OPTIONS = {'mode' : 'PREFIX'}",
-                                  SASIIndex.class.getName()));
-        createIndex(String.format("CREATE CUSTOM INDEX c2_idx on %%s(c2) USING '%s' WITH OPTIONS = {'mode' : 'CONTAINS'}",
-                                  SASIIndex.class.getName()));
-        createIndex(String.format("CREATE CUSTOM INDEX v1_idx on %%s(v1) USING '%s' WITH OPTIONS = {'mode' : 'PREFIX'}",
-                                  SASIIndex.class.getName()));
-        createIndex(String.format("CREATE CUSTOM INDEX v2_idx on %%s(v2) USING '%s' WITH OPTIONS = {'mode' : 'CONTAINS'}",
-                                  SASIIndex.class.getName()));
-        createIndex(String.format("CREATE CUSTOM INDEX v3_idx on %%s(v3) USING '%s'", SASIIndex.class.getName()));
+        createIndex(format("CREATE CUSTOM INDEX c1_idx on %%s(c1) USING '%s' WITH OPTIONS = {'mode' : 'PREFIX'}",
+                           SASIIndex.class.getName()));
+        createIndex(format("CREATE CUSTOM INDEX c2_idx on %%s(c2) USING '%s' WITH OPTIONS = {'mode' : 'CONTAINS'}",
+                           SASIIndex.class.getName()));
+        createIndex(format("CREATE CUSTOM INDEX v1_idx on %%s(v1) USING '%s' WITH OPTIONS = {'mode' : 'PREFIX'}",
+                           SASIIndex.class.getName()));
+        createIndex(format("CREATE CUSTOM INDEX v2_idx on %%s(v2) USING '%s' WITH OPTIONS = {'mode' : 'CONTAINS'}",
+                           SASIIndex.class.getName()));
+        createIndex(format("CREATE CUSTOM INDEX v3_idx on %%s(v3) USING '%s'", SASIIndex.class.getName()));
 
         forcePreparedValues();
         // prefix mode indexes support prefix/contains/matches
@@ -870,8 +870,8 @@ public class SecondaryIndexTest extends CQLTester
         String indexClassName = StubIndex.class.getName();
         createTable("CREATE TABLE %s (a int, b int, c int, PRIMARY KEY ((a), b))");
         // uses different options otherwise the two indexes are considered duplicates
-        createIndex(String.format("CREATE CUSTOM INDEX c_idx_1 ON %%s(c) USING '%s' WITH OPTIONS = {'foo':'a'}", indexClassName));
-        createIndex(String.format("CREATE CUSTOM INDEX c_idx_2 ON %%s(c) USING '%s' WITH OPTIONS = {'foo':'b'}", indexClassName));
+        createIndex(format("CREATE CUSTOM INDEX c_idx_1 ON %%s(c) USING '%s' WITH OPTIONS = {'foo':'a'}", indexClassName));
+        createIndex(format("CREATE CUSTOM INDEX c_idx_2 ON %%s(c) USING '%s' WITH OPTIONS = {'foo':'b'}", indexClassName));
 
         ColumnFamilyStore cfs = getCurrentColumnFamilyStore();
         TableMetadata cfm = cfs.metadata();
@@ -913,7 +913,7 @@ public class SecondaryIndexTest extends CQLTester
 
         String indexClassName = StubIndex.class.getName();
         createTable("CREATE TABLE %s (a int, b int, c int, PRIMARY KEY ((a), b))");
-        createIndex(String.format("CREATE CUSTOM INDEX c_idx ON %%s(c) USING '%s'", indexClassName));
+        createIndex(format("CREATE CUSTOM INDEX c_idx ON %%s(c) USING '%s'", indexClassName));
 
         ColumnFamilyStore cfs = getCurrentColumnFamilyStore();
         TableMetadata cfm = cfs.metadata();
@@ -969,7 +969,7 @@ public class SecondaryIndexTest extends CQLTester
         // Any columns which are unchanged by the update are not passed to the Indexer
         // Note that for simplicity this test resets the index between each scenario
         createTable("CREATE TABLE %s (k int, c int, v1 int, v2 int, PRIMARY KEY (k,c))");
-        createIndex(String.format("CREATE CUSTOM INDEX test_index ON %%s() USING '%s'", StubIndex.class.getName()));
+        createIndex(format("CREATE CUSTOM INDEX test_index ON %%s() USING '%s'", StubIndex.class.getName()));
         execute("INSERT INTO %s (k, c, v1, v2) VALUES (0, 0, 0, 0) USING TIMESTAMP 0");
 
         ColumnMetadata v1 = getCurrentColumnFamilyStore().metadata().getColumn(new ColumnIdentifier("v1", true));
@@ -1386,8 +1386,8 @@ public class SecondaryIndexTest extends CQLTester
         assertRows(execute("SELECT * FROM %s WHERE v = ?", udt1), row(1, udt1));
 
         dropIndex("DROP INDEX %s." + indexName);
-        assertInvalidMessage(String.format("Index '%s' could not be found", indexName),
-                             String.format("DROP INDEX %s.%s", KEYSPACE, indexName));
+        assertInvalidMessage(format("Index '%s.%s' doesn't exist", KEYSPACE, indexName),
+                             format("DROP INDEX %s.%s", KEYSPACE, indexName));
         assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
                              "SELECT * FROM %s WHERE v = ?", udt1);
     }
@@ -1419,8 +1419,8 @@ public class SecondaryIndexTest extends CQLTester
         assertEmpty(execute("SELECT * FROM %s WHERE v = ?", set(udt2)));
 
         dropIndex("DROP INDEX %s." + indexName);
-        assertInvalidMessage(String.format("Index '%s' could not be found", indexName),
-                             String.format("DROP INDEX %s.%s", KEYSPACE, indexName));
+        assertInvalidMessage(format("Index '%s.%s' doesn't exist", KEYSPACE, indexName),
+                             format("DROP INDEX %s.%s", KEYSPACE, indexName));
         assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
                              "SELECT * FROM %s WHERE v CONTAINS ?", udt1);
     }
@@ -1453,8 +1453,8 @@ public class SecondaryIndexTest extends CQLTester
         assertRows(execute("SELECT * FROM %s WHERE v CONTAINS ?", udt2), row(2, set(udt2)));
 
         dropIndex("DROP INDEX %s." + indexName);
-        assertInvalidMessage(String.format("Index '%s' could not be found", indexName),
-                             String.format("DROP INDEX %s.%s", KEYSPACE, indexName));
+        assertInvalidMessage(format("Index '%s.%s' doesn't exist", KEYSPACE, indexName),
+                             format("DROP INDEX %s.%s", KEYSPACE, indexName));
         assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
                              "SELECT * FROM %s WHERE v CONTAINS ?", udt1);
     }
@@ -1464,9 +1464,9 @@ public class SecondaryIndexTest extends CQLTester
     {
         String type = createType("CREATE TYPE %s (a int)");
         createTable("CREATE TABLE %s (k int PRIMARY KEY, v " + type + ")");
-        assertInvalidMessage("Secondary indexes are not supported on non-frozen UDTs", "CREATE INDEX ON %s (v)");
-        assertInvalidMessage("Non-collection columns support only simple indexes", "CREATE INDEX ON %s (keys(v))");
-        assertInvalidMessage("Non-collection columns support only simple indexes", "CREATE INDEX ON %s (values(v))");
+        assertInvalidMessage("Cannot create index on non-frozen UDT column v", "CREATE INDEX ON %s (v)");
+        assertInvalidMessage("Cannot create keys() index on v. Non-collection columns only support simple indexes", "CREATE INDEX ON %s (keys(v))");
+        assertInvalidMessage("Cannot create values() index on v. Non-collection columns only support simple indexes", "CREATE INDEX ON %s (values(v))");
         assertInvalidMessage("full() indexes can only be created on frozen collections", "CREATE INDEX ON %s (full(v))");
     }
 
@@ -1514,7 +1514,7 @@ public class SecondaryIndexTest extends CQLTester
 
     private ResultMessage.Prepared prepareStatement(String cql)
     {
-        return QueryProcessor.prepare(String.format(cql, KEYSPACE, currentTable()),
+        return QueryProcessor.prepare(format(cql, KEYSPACE, currentTable()),
                                       ClientState.forInternalCalls());
     }
 


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


[02/13] cassandra git commit: Make all DDL statements idempotent and not dependent on global state

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/test/unit/org/apache/cassandra/cql3/validation/entities/TypeTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/TypeTest.java b/test/unit/org/apache/cassandra/cql3/validation/entities/TypeTest.java
index bb43d75..ceb96b6 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/entities/TypeTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/TypeTest.java
@@ -28,8 +28,8 @@ public class TypeTest extends CQLTester
     @Test
     public void testNonExistingOnes() throws Throwable
     {
-        assertInvalidMessage("No user type named", "DROP TYPE " + KEYSPACE + ".type_does_not_exist");
-        assertInvalidMessage("Cannot drop type in unknown keyspace", "DROP TYPE keyspace_does_not_exist.type_does_not_exist");
+        assertInvalidMessage(String.format("Type '%s.type_does_not_exist' doesn't exist", KEYSPACE), "DROP TYPE " + KEYSPACE + ".type_does_not_exist");
+        assertInvalidMessage("Type 'keyspace_does_not_exist.type_does_not_exist' doesn't exist", "DROP TYPE keyspace_does_not_exist.type_does_not_exist");
 
         execute("DROP TYPE IF EXISTS " + KEYSPACE + ".type_does_not_exist");
         execute("DROP TYPE IF EXISTS keyspace_does_not_exist.type_does_not_exist");

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/test/unit/org/apache/cassandra/cql3/validation/entities/UFAuthTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/UFAuthTest.java b/test/unit/org/apache/cassandra/cql3/validation/entities/UFAuthTest.java
index ecff0cc..4a2d71f 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/entities/UFAuthTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/UFAuthTest.java
@@ -226,7 +226,7 @@ public class UFAuthTest extends CQLTester
             functions.add(functionName);
             statements.add(stmt);
         }
-        BatchStatement batch = new BatchStatement(-1, BatchStatement.Type.LOGGED, statements, Attributes.none());
+        BatchStatement batch = new BatchStatement(BatchStatement.Type.LOGGED, VariableSpecifications.empty(), statements, Attributes.none());
         assertUnauthorized(batch, functions);
 
         grantExecuteOnFunction(functions.get(0));
@@ -236,7 +236,7 @@ public class UFAuthTest extends CQLTester
         assertUnauthorized(batch, functions.subList(2, functions.size()));
 
         grantExecuteOnFunction(functions.get(2));
-        batch.checkAccess(clientState);
+        batch.authorize(clientState);
     }
 
     @Test
@@ -313,7 +313,7 @@ public class UFAuthTest extends CQLTester
         // with terminal arguments, so evaluated at prepare time
         String cql = String.format("UPDATE %s SET v2 = 0 WHERE k = blobasint(intasblob(0)) and v1 = 0",
                                    KEYSPACE + "." + currentTable());
-        getStatement(cql).checkAccess(clientState);
+        getStatement(cql).authorize(clientState);
 
         // with non-terminal arguments, so evaluated at execution
         String functionName = createSimpleFunction();
@@ -321,7 +321,7 @@ public class UFAuthTest extends CQLTester
         cql = String.format("UPDATE %s SET v2 = 0 WHERE k = blobasint(intasblob(%s)) and v1 = 0",
                             KEYSPACE + "." + currentTable(),
                             functionCall(functionName));
-        getStatement(cql).checkAccess(clientState);
+        getStatement(cql).authorize(clientState);
     }
 
     @Test
@@ -343,7 +343,7 @@ public class UFAuthTest extends CQLTester
         assertUnauthorized(aggDef, fFunc, "int");
         grantExecuteOnFunction(fFunc);
 
-        getStatement(aggDef).checkAccess(clientState);
+        getStatement(aggDef).authorize(clientState);
     }
 
     @Test
@@ -361,24 +361,24 @@ public class UFAuthTest extends CQLTester
         String cql = String.format("SELECT %s(v1) FROM %s",
                                    aggregate,
                                    KEYSPACE + "." + currentTable());
-        getStatement(cql).checkAccess(clientState);
+        getStatement(cql).authorize(clientState);
 
         // check that revoking EXECUTE permission on any one of the
         // component functions means we lose the ability to execute it
         revokeExecuteOnFunction(aggregate);
         assertUnauthorized(cql, aggregate, "int");
         grantExecuteOnFunction(aggregate);
-        getStatement(cql).checkAccess(clientState);
+        getStatement(cql).authorize(clientState);
 
         revokeExecuteOnFunction(sFunc);
         assertUnauthorized(cql, sFunc, "int, int");
         grantExecuteOnFunction(sFunc);
-        getStatement(cql).checkAccess(clientState);
+        getStatement(cql).authorize(clientState);
 
         revokeExecuteOnFunction(fFunc);
         assertUnauthorized(cql, fFunc, "int");
         grantExecuteOnFunction(fFunc);
-        getStatement(cql).checkAccess(clientState);
+        getStatement(cql).authorize(clientState);
     }
 
     @Test
@@ -410,7 +410,7 @@ public class UFAuthTest extends CQLTester
         assertUnauthorized(cql, aggregate, "int");
         grantExecuteOnFunction(aggregate);
 
-        getStatement(cql).checkAccess(clientState);
+        getStatement(cql).authorize(clientState);
     }
 
     @Test
@@ -442,7 +442,7 @@ public class UFAuthTest extends CQLTester
         assertUnauthorized(cql, innerFunc, "int");
         grantExecuteOnFunction(innerFunc);
 
-        getStatement(cql).checkAccess(clientState);
+        getStatement(cql).authorize(clientState);
     }
 
     @Test
@@ -484,7 +484,7 @@ public class UFAuthTest extends CQLTester
         grantExecuteOnFunction(innerFunction);
 
         // now execution of both is permitted
-        getStatement(cql).checkAccess(clientState);
+        getStatement(cql).authorize(clientState);
     }
 
     private void assertPermissionsOnFunction(String cql, String functionName) throws Throwable
@@ -496,14 +496,14 @@ public class UFAuthTest extends CQLTester
     {
         assertUnauthorized(cql, functionName, argTypes);
         grantExecuteOnFunction(functionName);
-        getStatement(cql).checkAccess(clientState);
+        getStatement(cql).authorize(clientState);
     }
 
     private void assertUnauthorized(BatchStatement batch, Iterable<String> functionNames) throws Throwable
     {
         try
         {
-            batch.checkAccess(clientState);
+            batch.authorize(clientState);
             fail("Expected an UnauthorizedException, but none was thrown");
         }
         catch (UnauthorizedException e)
@@ -520,7 +520,7 @@ public class UFAuthTest extends CQLTester
     {
         try
         {
-            getStatement(cql).checkAccess(clientState);
+            getStatement(cql).authorize(clientState);
             fail("Expected an UnauthorizedException, but none was thrown");
         }
         catch (UnauthorizedException e)
@@ -625,7 +625,7 @@ public class UFAuthTest extends CQLTester
 
     private CQLStatement getStatement(String cql)
     {
-        return QueryProcessor.getStatement(cql, clientState).statement;
+        return QueryProcessor.getStatement(cql, clientState);
     }
 
     private FunctionResource functionResource(String functionName)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/test/unit/org/apache/cassandra/cql3/validation/entities/UFIdentificationTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/UFIdentificationTest.java b/test/unit/org/apache/cassandra/cql3/validation/entities/UFIdentificationTest.java
index b2288e4..bba5c92 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/entities/UFIdentificationTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/UFIdentificationTest.java
@@ -28,6 +28,7 @@ import org.junit.Test;
 import org.apache.cassandra.cql3.Attributes;
 import org.apache.cassandra.cql3.CQLStatement;
 import org.apache.cassandra.cql3.QueryProcessor;
+import org.apache.cassandra.cql3.VariableSpecifications;
 import org.apache.cassandra.cql3.functions.Function;
 import org.apache.cassandra.cql3.statements.BatchStatement;
 import org.apache.cassandra.cql3.statements.ModificationStatement;
@@ -307,7 +308,7 @@ public class UFIdentificationTest extends CQLTester
         statements.add(modificationStatement(cql("INSERT INTO %s (key, i_cc, t_cc) VALUES (2, 2, %s)",
                                                  functionCall(tFunc, "'foo'"))));
 
-        BatchStatement batch = new BatchStatement(-1, BatchStatement.Type.LOGGED, statements, Attributes.none());
+        BatchStatement batch = new BatchStatement(BatchStatement.Type.LOGGED, VariableSpecifications.empty(), statements, Attributes.none());
         assertFunctions(batch, iFunc, iFunc2, tFunc);
     }
 
@@ -320,18 +321,18 @@ public class UFIdentificationTest extends CQLTester
         statements.add(modificationStatement(cql("UPDATE %s SET i_val = %s WHERE key=0 AND i_cc=1 and t_cc='foo' IF s_val = %s",
                                                  functionCall(iFunc, "0"), functionCall(sFunc, "{1}"))));
 
-        BatchStatement batch = new BatchStatement(-1, BatchStatement.Type.LOGGED, statements, Attributes.none());
+        BatchStatement batch = new BatchStatement(BatchStatement.Type.LOGGED, VariableSpecifications.empty(), statements, Attributes.none());
         assertFunctions(batch, iFunc, lFunc, sFunc);
     }
 
     private ModificationStatement modificationStatement(String cql)
     {
-        return (ModificationStatement) QueryProcessor.getStatement(cql, ClientState.forInternalCalls()).statement;
+        return (ModificationStatement) QueryProcessor.getStatement(cql, ClientState.forInternalCalls());
     }
 
     private void assertFunctions(String cql, String... function)
     {
-        CQLStatement stmt = QueryProcessor.getStatement(cql, ClientState.forInternalCalls()).statement;
+        CQLStatement stmt = QueryProcessor.getStatement(cql, ClientState.forInternalCalls());
         assertFunctions(stmt, function);
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/test/unit/org/apache/cassandra/cql3/validation/entities/UFJavaTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/UFJavaTest.java b/test/unit/org/apache/cassandra/cql3/validation/entities/UFJavaTest.java
index 187871a..975b21a 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/entities/UFJavaTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/UFJavaTest.java
@@ -113,8 +113,8 @@ public class UFJavaTest extends CQLTester
     @Test
     public void testJavaFunctionInvalidReturn() throws Throwable
     {
-        assertInvalidMessage("system keyspace is not user-modifiable",
-                             "CREATE OR REPLACE FUNCTION jfir(val double) " +
+        assertInvalidMessage("cannot convert from long to Double",
+                             "CREATE OR REPLACE FUNCTION " + KEYSPACE + ".jfir(val double) " +
                              "RETURNS NULL ON NULL INPUT " +
                              "RETURNS double " +
                              "LANGUAGE JAVA\n" +

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/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 7940b92..d21d159 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/entities/UFTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/UFTest.java
@@ -58,10 +58,21 @@ public class UFTest extends CQLTester
     @Test
     public void testNonExistingOnes() throws Throwable
     {
-        assertInvalidThrowMessage("Cannot drop non existing function", InvalidRequestException.class, "DROP FUNCTION " + KEYSPACE + ".func_does_not_exist");
-        assertInvalidThrowMessage("Cannot drop non existing function", InvalidRequestException.class, "DROP FUNCTION " + KEYSPACE + ".func_does_not_exist(int,text)");
-        assertInvalidThrowMessage("Cannot drop non existing function", InvalidRequestException.class, "DROP FUNCTION keyspace_does_not_exist.func_does_not_exist");
-        assertInvalidThrowMessage("Cannot drop non existing function", InvalidRequestException.class, "DROP FUNCTION keyspace_does_not_exist.func_does_not_exist(int,text)");
+        assertInvalidThrowMessage(String.format("Function '%s.func_does_not_exist' doesn't exist", KEYSPACE),
+                                  InvalidRequestException.class,
+                                  "DROP FUNCTION " + KEYSPACE + ".func_does_not_exist");
+
+        assertInvalidThrowMessage(String.format("Function '%s.func_does_not_exist(int, text)' doesn't exist", KEYSPACE),
+                                  InvalidRequestException.class,
+                                  "DROP FUNCTION " + KEYSPACE + ".func_does_not_exist(int, text)");
+
+        assertInvalidThrowMessage("Function 'keyspace_does_not_exist.func_does_not_exist' doesn't exist",
+                                  InvalidRequestException.class,
+                                  "DROP FUNCTION keyspace_does_not_exist.func_does_not_exist");
+
+        assertInvalidThrowMessage("Function 'keyspace_does_not_exist.func_does_not_exist(int, text)' doesn't exist",
+                                  InvalidRequestException.class,
+                                  "DROP FUNCTION keyspace_does_not_exist.func_does_not_exist(int, text)");
 
         execute("DROP FUNCTION IF EXISTS " + KEYSPACE + ".func_does_not_exist");
         execute("DROP FUNCTION IF EXISTS " + KEYSPACE + ".func_does_not_exist(int,text)");
@@ -100,7 +111,7 @@ public class UFTest extends CQLTester
                      "RETURNS NULL ON NULL INPUT " +
                      "RETURNS int " +
                      "LANGUAGE javascript " +
-                     "AS '\"string\";';");
+                     "AS '\"string1\";';");
 
         assertLastSchemaChange(Event.SchemaChange.Change.UPDATED, Event.SchemaChange.Target.FUNCTION,
                                KEYSPACE, parseFunctionName(f).name,
@@ -369,7 +380,7 @@ public class UFTest extends CQLTester
                              "CREATE OR REPLACE FUNCTION " + fSin + " ( input double ) " +
                              "CALLED ON NULL INPUT " +
                              "RETURNS text " +
-                             "LANGUAGE java AS 'return Double.valueOf(42d);'");
+                             "LANGUAGE java AS 'return \"42d\";'");
 
         // proper replacement
         execute("CREATE OR REPLACE FUNCTION " + fSin + " ( input double ) " +
@@ -402,13 +413,13 @@ public class UFTest extends CQLTester
         execute("DROP FUNCTION " + fSin2);
 
         // Drop unexisting function
-        assertInvalidMessage("Cannot drop non existing function", "DROP FUNCTION " + fSin);
+        assertInvalidMessage(String.format("Function '%s' doesn't exist", fSin), "DROP FUNCTION " + fSin);
         // but don't complain with "IF EXISTS"
         execute("DROP FUNCTION IF EXISTS " + fSin);
 
         // can't drop native functions
-        assertInvalidMessage("system keyspace is not user-modifiable", "DROP FUNCTION totimestamp");
-        assertInvalidMessage("system keyspace is not user-modifiable", "DROP FUNCTION uuid");
+        assertInvalidMessage("System keyspace 'system' is not user-modifiable", "DROP FUNCTION totimestamp");
+        assertInvalidMessage("System keyspace 'system' is not user-modifiable", "DROP FUNCTION uuid");
 
         // sin() no longer exists
         assertInvalidMessage("Unknown function", "SELECT key, sin(d) FROM %s");
@@ -509,8 +520,8 @@ public class UFTest extends CQLTester
         assertEmpty(execute("SELECT v FROM %s WHERE k = " + fOverload + "((varchar)?)", "foo"));
 
         // no such functions exist...
-        assertInvalidMessage("non existing function", "DROP FUNCTION " + fOverload + "(boolean)");
-        assertInvalidMessage("non existing function", "DROP FUNCTION " + fOverload + "(bigint)");
+        assertInvalidMessage(String.format("Function '%s(boolean)' doesn't exist", fOverload), "DROP FUNCTION " + fOverload + "(boolean)");
+        assertInvalidMessage(String.format("Function '%s(bigint)' doesn't exist", fOverload), "DROP FUNCTION " + fOverload + "(bigint)");
 
         // 'overloaded' has multiple overloads - so it has to fail (CASSANDRA-7812)
         assertInvalidMessage("matches multiple function definitions", "DROP FUNCTION " + fOverload);
@@ -641,43 +652,43 @@ public class UFTest extends CQLTester
 
                 "AS 'return null;';");
 
-        assertInvalidMessage("system keyspace is not user-modifiable",
+        assertInvalidMessage("System keyspace 'system' is not user-modifiable",
                              "CREATE OR REPLACE FUNCTION system.jnft(val double) " +
                              "RETURNS NULL ON NULL INPUT " +
                              "RETURNS double " +
                              "LANGUAGE JAVA\n" +
                              "AS 'return null;';");
-        assertInvalidMessage("system keyspace is not user-modifiable",
+        assertInvalidMessage("System keyspace 'system' is not user-modifiable",
                              "CREATE OR REPLACE FUNCTION system.totimestamp(val timeuuid) " +
                              "RETURNS NULL ON NULL INPUT " +
                              "RETURNS timestamp " +
                              "LANGUAGE JAVA\n" +
 
                              "AS 'return null;';");
-        assertInvalidMessage("system keyspace is not user-modifiable",
+        assertInvalidMessage("System keyspace 'system' is not user-modifiable",
                              "DROP FUNCTION system.now");
 
-        // KS for executeInternal() is system
-        assertInvalidMessage("system keyspace is not user-modifiable",
+        // KS for executeLocally() is system
+        assertInvalidMessage("System keyspace 'system' is not user-modifiable",
                              "CREATE OR REPLACE FUNCTION jnft(val double) " +
                              "RETURNS NULL ON NULL INPUT " +
                              "RETURNS double " +
                              "LANGUAGE JAVA\n" +
                              "AS 'return null;';");
-        assertInvalidMessage("system keyspace is not user-modifiable",
+        assertInvalidMessage("System keyspace 'system' is not user-modifiable",
                              "CREATE OR REPLACE FUNCTION totimestamp(val timeuuid) " +
                              "RETURNS NULL ON NULL INPUT " +
                              "RETURNS timestamp " +
                              "LANGUAGE JAVA\n" +
                              "AS 'return null;';");
-        assertInvalidMessage("system keyspace is not user-modifiable",
+        assertInvalidMessage("System keyspace 'system' is not user-modifiable",
                              "DROP FUNCTION now");
     }
 
     @Test
     public void testFunctionNonExistingKeyspace() throws Throwable
     {
-        assertInvalidMessage("Keyspace this_ks_does_not_exist doesn't exist",
+        assertInvalidMessage("Keyspace 'this_ks_does_not_exist' doesn't exist",
                              "CREATE OR REPLACE FUNCTION this_ks_does_not_exist.jnft(val double) " +
                              "RETURNS NULL ON NULL INPUT " +
                              "RETURNS double " +
@@ -690,7 +701,7 @@ public class UFTest extends CQLTester
     {
         dropPerTestKeyspace();
 
-        assertInvalidMessage("Keyspace " + KEYSPACE_PER_TEST + " doesn't exist",
+        assertInvalidMessage("Keyspace '" + KEYSPACE_PER_TEST + "' doesn't exist",
                              "CREATE OR REPLACE FUNCTION " + KEYSPACE_PER_TEST + ".jnft(val double) " +
                              "RETURNS NULL ON NULL INPUT " +
                              "RETURNS double " +
@@ -761,7 +772,7 @@ public class UFTest extends CQLTester
     @Test
     public void testDuplicateArgNames() throws Throwable
     {
-        assertInvalidMessage("duplicate argument names for given function",
+        assertInvalidMessage("Duplicate argument names for given function",
                              "CREATE OR REPLACE FUNCTION " + KEYSPACE + ".scrinv(val double, val text) " +
                              "RETURNS NULL ON NULL INPUT " +
                              "RETURNS text " +

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/test/unit/org/apache/cassandra/cql3/validation/entities/UFTypesTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/UFTypesTest.java b/test/unit/org/apache/cassandra/cql3/validation/entities/UFTypesTest.java
index 3f1bcb1..9680bd5 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/entities/UFTypesTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/UFTypesTest.java
@@ -293,14 +293,14 @@ public class UFTypesTest extends CQLTester
         execute("INSERT INTO %s (a, b) VALUES (?, ?)", 2, set(4, 5, 6));
         execute("INSERT INTO %s (a, b) VALUES (?, ?)", 3, set(7, 8, 9));
 
-        assertInvalidMessage("The function arguments should not be frozen",
+        assertInvalidMessage("Argument 'frozen<set<int>>' cannot be frozen; remove frozen<> modifier from 'frozen<set<int>>'",
                              "CREATE OR REPLACE FUNCTION " + KEYSPACE + ".frozenSetArg(values frozen<set<int>>) " +
                              "CALLED ON NULL INPUT " +
                              "RETURNS int " +
                              "LANGUAGE java\n" +
                              "AS 'int sum = 0; for (Object value : values) {sum += value;} return sum;';");
 
-        assertInvalidMessage("The function return type should not be frozen",
+        assertInvalidMessage("Return type 'frozen<set<int>>' cannot be frozen; remove frozen<> modifier from 'frozen<set<int>>'",
                              "CREATE OR REPLACE FUNCTION " + KEYSPACE + ".frozenReturnType(values set<int>) " +
                              "CALLED ON NULL INPUT " +
                              "RETURNS frozen<set<int>> " +
@@ -331,7 +331,7 @@ public class UFTypesTest extends CQLTester
         assertRows(execute("SELECT a FROM %s WHERE b = " + functionName + "(?)", set(1, 2, 3)),
                    row(1));
 
-        assertInvalidMessage("The function arguments should not be frozen",
+        assertInvalidMessage("Argument 'frozen<set<int>>' cannot be frozen; remove frozen<> modifier from 'frozen<set<int>>'",
                              "DROP FUNCTION " + functionName + "(frozen<set<int>>);");
     }
 
@@ -346,14 +346,14 @@ public class UFTypesTest extends CQLTester
         execute("INSERT INTO %s (a, b) VALUES (?, ?)", 2, list(4, 5, 6));
         execute("INSERT INTO %s (a, b) VALUES (?, ?)", 3, list(7, 8, 9));
 
-        assertInvalidMessage("The function arguments should not be frozen",
+        assertInvalidMessage("Argument 'frozen<list<int>>' cannot be frozen; remove frozen<> modifier from 'frozen<list<int>>'",
                              "CREATE OR REPLACE FUNCTION " + KEYSPACE + ".withFrozenArg(values frozen<list<int>>) " +
                              "CALLED ON NULL INPUT " +
                              "RETURNS int " +
                              "LANGUAGE java\n" +
                              "AS 'int sum = 0; for (Object value : values) {sum += value;} return sum;';");
 
-        assertInvalidMessage("The function return type should not be frozen",
+        assertInvalidMessage("Return type 'frozen<list<int>>' cannot be frozen; remove frozen<> modifier from 'frozen<list<int>>'",
                              "CREATE OR REPLACE FUNCTION " + KEYSPACE + ".frozenReturnType(values list<int>) " +
                              "CALLED ON NULL INPUT " +
                              "RETURNS frozen<list<int>> " +
@@ -384,7 +384,7 @@ public class UFTypesTest extends CQLTester
         assertRows(execute("SELECT a FROM %s WHERE b = " + functionName + "(?)", set(1, 2, 3)),
                    row(1));
 
-        assertInvalidMessage("The function arguments should not be frozen",
+        assertInvalidMessage("frozen<list<int>>' cannot be frozen; remove frozen<> modifier from 'frozen<list<int>>'",
                              "DROP FUNCTION " + functionName + "(frozen<list<int>>);");
     }
 
@@ -399,14 +399,14 @@ public class UFTypesTest extends CQLTester
         execute("INSERT INTO %s (a, b) VALUES (?, ?)", 2, map(4, 4, 5, 5, 6, 6));
         execute("INSERT INTO %s (a, b) VALUES (?, ?)", 3, map(7, 7, 8, 8, 9, 9));
 
-        assertInvalidMessage("The function arguments should not be frozen",
+        assertInvalidMessage("Argument 'frozen<map<int, int>>' cannot be frozen; remove frozen<> modifier from 'frozen<map<int, int>>'",
                              "CREATE OR REPLACE FUNCTION " + KEYSPACE + ".withFrozenArg(values frozen<map<int, int>>) " +
                              "CALLED ON NULL INPUT " +
                              "RETURNS int " +
                              "LANGUAGE java\n" +
                              "AS 'int sum = 0; for (Object value : values.values()) {sum += value;} return sum;';");
 
-        assertInvalidMessage("The function return type should not be frozen",
+        assertInvalidMessage("Return type 'frozen<map<int, int>>' cannot be frozen; remove frozen<> modifier from 'frozen<map<int, int>>'",
                              "CREATE OR REPLACE FUNCTION " + KEYSPACE + ".frozenReturnType(values map<int, int>) " +
                              "CALLED ON NULL INPUT " +
                              "RETURNS frozen<map<int, int>> " +
@@ -437,7 +437,7 @@ public class UFTypesTest extends CQLTester
         assertRows(execute("SELECT a FROM %s WHERE b = " + functionName + "(?)", map(1, 1, 2, 2, 3, 3)),
                    row(1));
 
-        assertInvalidMessage("The function arguments should not be frozen",
+        assertInvalidMessage("frozen<map<int, int>>' cannot be frozen; remove frozen<> modifier from 'frozen<map<int, int>>",
                              "DROP FUNCTION " + functionName + "(frozen<map<int, int>>);");
     }
 
@@ -452,14 +452,14 @@ public class UFTypesTest extends CQLTester
         execute("INSERT INTO %s (a, b) VALUES (?, ?)", 2, tuple(4, 5));
         execute("INSERT INTO %s (a, b) VALUES (?, ?)", 3, tuple(7, 8));
 
-        assertInvalidMessage("The function arguments should not be frozen",
+        assertInvalidMessage("Argument 'tuple<int, int>' cannot be frozen; remove frozen<> modifier from 'tuple<int, int>'",
                              "CREATE OR REPLACE FUNCTION " + KEYSPACE + ".withFrozenArg(values frozen<tuple<int, int>>) " +
                              "CALLED ON NULL INPUT " +
                              "RETURNS text " +
                              "LANGUAGE java\n" +
                              "AS 'return values.toString();';");
 
-        assertInvalidMessage("The function return type should not be frozen",
+        assertInvalidMessage("Return type 'tuple<int, int>' cannot be frozen; remove frozen<> modifier from 'tuple<int, int>'",
                              "CREATE OR REPLACE FUNCTION " + KEYSPACE + ".frozenReturnType(values tuple<int, int>) " +
                              "CALLED ON NULL INPUT " +
                              "RETURNS frozen<tuple<int, int>> " +
@@ -490,7 +490,7 @@ public class UFTypesTest extends CQLTester
         assertRows(execute("SELECT a FROM %s WHERE b = " + functionName + "(?)", tuple(1, 2)),
                    row(1));
 
-        assertInvalidMessage("The function arguments should not be frozen",
+        assertInvalidMessage("Argument 'tuple<int, int>' cannot be frozen; remove frozen<> modifier from 'tuple<int, int>'",
                              "DROP FUNCTION " + functionName + "(frozen<tuple<int, int>>);");
     }
 
@@ -506,14 +506,14 @@ public class UFTypesTest extends CQLTester
         execute("INSERT INTO %s (a, b) VALUES (?, {f : ?})", 2, 4);
         execute("INSERT INTO %s (a, b) VALUES (?, {f : ?})", 3, 7);
 
-        assertInvalidMessage("The function arguments should not be frozen",
+        assertInvalidMessage("cannot be frozen; remove frozen<> modifier",
                              "CREATE OR REPLACE FUNCTION " + KEYSPACE + ".withFrozenArg(values frozen<" + myType + ">) " +
                              "CALLED ON NULL INPUT " +
                              "RETURNS text " +
                              "LANGUAGE java\n" +
                              "AS 'return values.toString();';");
 
-        assertInvalidMessage("The function return type should not be frozen",
+        assertInvalidMessage("cannot be frozen; remove frozen<> modifier",
                              "CREATE OR REPLACE FUNCTION " + KEYSPACE + ".frozenReturnType(values " + myType + ") " +
                              "CALLED ON NULL INPUT " +
                              "RETURNS frozen<" + myType + "> " +
@@ -544,7 +544,7 @@ public class UFTypesTest extends CQLTester
         assertRows(execute("SELECT a FROM %s WHERE b = " + functionName + "({f: ?})", 1),
                    row(1));
 
-        assertInvalidMessage("The function arguments should not be frozen",
+        assertInvalidMessage(String.format("frozen<%s>' cannot be frozen; remove frozen<> modifier from 'frozen<%s>'", myType, myType),
                              "DROP FUNCTION " + functionName + "(frozen<" + myType + ">);");
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/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 9ea5572..e39dd35 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/entities/UserTypesTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/UserTypesTest.java
@@ -109,9 +109,9 @@ public class UserTypesTest extends CQLTester
         String myType = KEYSPACE + '.' + typename;
 
         // non-frozen UDTs in a table PK
-        assertInvalidMessage("Invalid non-frozen user-defined type for PRIMARY KEY component k",
+        assertInvalidMessage("Invalid non-frozen user-defined type \"" + myType + "\" for PRIMARY KEY column 'k'",
                 "CREATE TABLE " + KEYSPACE + ".wrong (k " + myType + " PRIMARY KEY , v int)");
-        assertInvalidMessage("Invalid non-frozen user-defined type for PRIMARY KEY component k2",
+        assertInvalidMessage("Invalid non-frozen user-defined type \"" + myType + "\" for PRIMARY KEY column 'k2'",
                 "CREATE TABLE " + KEYSPACE + ".wrong (k1 int, k2 " + myType + ", v int, PRIMARY KEY (k1, k2))");
 
         // non-frozen UDTs in a collection
@@ -620,7 +620,7 @@ public class UserTypesTest extends CQLTester
     private void assertInvalidAlterDropStatements(String t) throws Throwable
     {
         assertInvalidMessage("Cannot alter user type " + typeWithKs(t), "ALTER TYPE " + typeWithKs(t) + " RENAME foo TO bar;");
-        assertInvalidMessage("Cannot drop user type " + typeWithKs(t), "DROP TYPE " + typeWithKs(t) + ';');
+        assertInvalidMessage("Cannot drop user type '" + typeWithKs(t), "DROP TYPE " + typeWithKs(t) + ';');
     }
 
     @Test

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/test/unit/org/apache/cassandra/cql3/validation/entities/VirtualTableTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/VirtualTableTest.java b/test/unit/org/apache/cassandra/cql3/validation/entities/VirtualTableTest.java
index 3581a73..a503a60 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/entities/VirtualTableTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/VirtualTableTest.java
@@ -279,34 +279,34 @@ public class VirtualTableTest extends CQLTester
     @Test
     public void testInvalidDDLOperations() throws Throwable
     {
-        assertInvalidMessage("Cannot drop virtual keyspaces",
+        assertInvalidMessage("Virtual keyspace 'test_virtual_ks' is not user-modifiable",
                              "DROP KEYSPACE test_virtual_ks");
 
-        assertInvalidMessage("Cannot alter virtual keyspaces",
+        assertInvalidMessage("Virtual keyspace 'test_virtual_ks' is not user-modifiable",
                              "ALTER KEYSPACE test_virtual_ks WITH durable_writes = false");
 
-        assertInvalidMessage("Cannot create tables in virtual keyspaces",
+        assertInvalidMessage("Virtual keyspace 'test_virtual_ks' is not user-modifiable",
                              "CREATE TABLE test_virtual_ks.test (id int PRIMARY KEY)");
 
-        assertInvalidMessage("Cannot create types in virtual keyspaces",
+        assertInvalidMessage("Virtual keyspace 'test_virtual_ks' is not user-modifiable",
                              "CREATE TYPE test_virtual_ks.type (id int)");
 
-        assertInvalidMessage("Cannot drop virtual tables",
+        assertInvalidMessage("Virtual keyspace 'test_virtual_ks' is not user-modifiable",
                              "DROP TABLE test_virtual_ks.vt1");
 
-        assertInvalidMessage("Cannot alter virtual tables",
+        assertInvalidMessage("Virtual keyspace 'test_virtual_ks' is not user-modifiable",
                              "ALTER TABLE test_virtual_ks.vt1 DROP v1");
 
-        assertInvalidMessage("Cannot truncate virtual tables",
+        assertInvalidMessage("Error during truncate: Cannot truncate virtual tables",
                              "TRUNCATE TABLE test_virtual_ks.vt1");
 
-        assertInvalidMessage("Secondary indexes are not supported on virtual tables",
+        assertInvalidMessage("Virtual keyspace 'test_virtual_ks' is not user-modifiable",
                              "CREATE INDEX ON test_virtual_ks.vt1 (v1)");
 
-        assertInvalidMessage("Materialized views are not supported on virtual tables",
+        assertInvalidMessage("Virtual keyspace 'test_virtual_ks' is not user-modifiable",
                              "CREATE MATERIALIZED VIEW test_virtual_ks.mvt1 AS SELECT c, v1 FROM test_virtual_ks.vt1 WHERE c IS NOT NULL PRIMARY KEY(c)");
 
-        assertInvalidMessage("Cannot CREATE TRIGGER against a virtual table",
+        assertInvalidMessage("Virtual keyspace 'test_virtual_ks' is not user-modifiable",
                              "CREATE TRIGGER test_trigger ON test_virtual_ks.vt1 USING '" + TestTrigger.class.getName() + '\'');
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/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 f2f6614..703d0ad 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/AggregationTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/AggregationTest.java
@@ -42,18 +42,15 @@ import ch.qos.logback.classic.joran.ReconfigureOnChangeTask;
 import ch.qos.logback.classic.spi.TurboFilterList;
 import ch.qos.logback.classic.turbo.ReconfigureOnChangeFilter;
 import ch.qos.logback.classic.turbo.TurboFilter;
-import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.schema.SchemaConstants;
 import org.apache.cassandra.cql3.CQLTester;
 import org.apache.cassandra.cql3.QueryProcessor;
 import org.apache.cassandra.cql3.UntypedResultSet;
 import org.apache.cassandra.cql3.UntypedResultSet.Row;
-import org.apache.cassandra.cql3.functions.UDAggregate;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.db.marshal.TypeParser;
 import org.apache.cassandra.exceptions.FunctionExecutionException;
 import org.apache.cassandra.exceptions.InvalidRequestException;
-import org.apache.cassandra.schema.KeyspaceMetadata;
 import org.apache.cassandra.service.ClientState;
 import org.apache.cassandra.transport.Event;
 import org.apache.cassandra.transport.ProtocolVersion;
@@ -70,10 +67,21 @@ public class AggregationTest extends CQLTester
     @Test
     public void testNonExistingOnes() throws Throwable
     {
-        assertInvalidThrowMessage("Cannot drop non existing aggregate", InvalidRequestException.class, "DROP AGGREGATE " + KEYSPACE + ".aggr_does_not_exist");
-        assertInvalidThrowMessage("Cannot drop non existing aggregate", InvalidRequestException.class, "DROP AGGREGATE " + KEYSPACE + ".aggr_does_not_exist(int,text)");
-        assertInvalidThrowMessage("Cannot drop non existing aggregate", InvalidRequestException.class, "DROP AGGREGATE keyspace_does_not_exist.aggr_does_not_exist");
-        assertInvalidThrowMessage("Cannot drop non existing aggregate", InvalidRequestException.class, "DROP AGGREGATE keyspace_does_not_exist.aggr_does_not_exist(int,text)");
+        assertInvalidThrowMessage(String.format("Aggregate '%s.aggr_does_not_exist' doesn't exist", KEYSPACE),
+                                  InvalidRequestException.class,
+                                  "DROP AGGREGATE " + KEYSPACE + ".aggr_does_not_exist");
+
+        assertInvalidThrowMessage(String.format("Aggregate '%s.aggr_does_not_exist(int, text)' doesn't exist", KEYSPACE),
+                                  InvalidRequestException.class,
+                                  "DROP AGGREGATE " + KEYSPACE + ".aggr_does_not_exist(int,text)");
+
+        assertInvalidThrowMessage("Aggregate 'keyspace_does_not_exist.aggr_does_not_exist' doesn't exist",
+                                  InvalidRequestException.class,
+                                  "DROP AGGREGATE keyspace_does_not_exist.aggr_does_not_exist");
+
+        assertInvalidThrowMessage("Aggregate 'keyspace_does_not_exist.aggr_does_not_exist(int, text)' doesn't exist",
+                                  InvalidRequestException.class,
+                                  "DROP AGGREGATE keyspace_does_not_exist.aggr_does_not_exist(int,text)");
 
         execute("DROP AGGREGATE IF EXISTS " + KEYSPACE + ".aggr_does_not_exist");
         execute("DROP AGGREGATE IF EXISTS " + KEYSPACE + ".aggr_does_not_exist(int,text)");
@@ -419,7 +427,7 @@ public class AggregationTest extends CQLTester
         schemaChange("CREATE OR REPLACE AGGREGATE " + a + "(double) " +
                      "SFUNC " + shortFunctionName(f) + " " +
                      "STYPE double " +
-                     "INITCOND 0");
+                     "INITCOND 1");
 
         assertLastSchemaChange(Event.SchemaChange.Change.UPDATED, Event.SchemaChange.Target.AGGREGATE,
                                KEYSPACE, parseFunctionName(a).name,
@@ -464,7 +472,7 @@ public class AggregationTest extends CQLTester
 
         // DROP AGGREGATE must not succeed against a scalar
         assertInvalidMessage("matches multiple function definitions", "DROP AGGREGATE " + f);
-        assertInvalidMessage("non existing", "DROP AGGREGATE " + f + "(double, double)");
+        assertInvalidMessage("doesn't exist", "DROP AGGREGATE " + f + "(double, double)");
 
         String a = createAggregate(KEYSPACE,
                                    "double",
@@ -481,7 +489,7 @@ public class AggregationTest extends CQLTester
 
         // DROP FUNCTION must not succeed against an aggregate
         assertInvalidMessage("matches multiple function definitions", "DROP FUNCTION " + a);
-        assertInvalidMessage("non existing function", "DROP FUNCTION " + a + "(double)");
+        assertInvalidMessage("doesn't exist", "DROP FUNCTION " + a + "(double)");
 
         // ambigious
         assertInvalidMessage("matches multiple function definitions", "DROP AGGREGATE " + a);
@@ -660,37 +668,37 @@ public class AggregationTest extends CQLTester
                                         "LANGUAGE java " +
                                         "AS 'return a.toString();'");
 
-        assertInvalidMessage("does not exist or is not a scalar function",
+        assertInvalidMessage("doesn't exist",
                              "CREATE AGGREGATE " + KEYSPACE + ".aggrInvalid(double)" +
                              "SFUNC " + shortFunctionName(fState) + " " +
                              "STYPE double " +
                              "FINALFUNC " + shortFunctionName(fFinal));
-        assertInvalidMessage("does not exist or is not a scalar function",
+        assertInvalidMessage("doesn't exist",
                              "CREATE AGGREGATE " + KEYSPACE + ".aggrInvalid(int)" +
                              "SFUNC " + shortFunctionName(fState) + " " +
                              "STYPE double " +
                              "FINALFUNC " + shortFunctionName(fFinal));
-        assertInvalidMessage("does not exist or is not a scalar function",
+        assertInvalidMessage("doesn't exist",
                              "CREATE AGGREGATE " + KEYSPACE + ".aggrInvalid(double)" +
                              "SFUNC " + shortFunctionName(fState) + " " +
                              "STYPE int " +
                              "FINALFUNC " + shortFunctionName(fFinal));
-        assertInvalidMessage("does not exist or is not a scalar function",
+        assertInvalidMessage("doesn't exist",
                              "CREATE AGGREGATE " + KEYSPACE + ".aggrInvalid(double)" +
                              "SFUNC " + shortFunctionName(fState) + " " +
                              "STYPE int");
-        assertInvalidMessage("does not exist or is not a scalar function",
+        assertInvalidMessage("doesn't exist",
                              "CREATE AGGREGATE " + KEYSPACE + ".aggrInvalid(int)" +
                              "SFUNC " + shortFunctionName(fState) + " " +
                              "STYPE double");
 
-        assertInvalidMessage("does not exist or is not a scalar function",
+        assertInvalidMessage("doesn't exist",
                              "CREATE AGGREGATE " + KEYSPACE + ".aggrInvalid(double)" +
                              "SFUNC " + shortFunctionName(fState2) + " " +
                              "STYPE double " +
                              "FINALFUNC " + shortFunctionName(fFinal));
 
-        assertInvalidMessage("does not exist or is not a scalar function",
+        assertInvalidMessage("doesn't exist",
                              "CREATE AGGREGATE " + KEYSPACE + ".aggrInvalid(double)" +
                              "SFUNC " + shortFunctionName(fState) + " " +
                              "STYPE double " +
@@ -716,13 +724,13 @@ public class AggregationTest extends CQLTester
                                        "LANGUAGE java " +
                                        "AS 'return a.toString();'");
 
-        assertInvalidMessage("does not exist or is not a scalar function",
+        assertInvalidMessage("doesn't exist",
                              "CREATE AGGREGATE " + KEYSPACE + ".aggrInvalid(int)" +
                              "SFUNC " + shortFunctionName(fState) + "_not_there " +
                              "STYPE int " +
                              "FINALFUNC " + shortFunctionName(fFinal));
 
-        assertInvalidMessage("does not exist or is not a scalar function",
+        assertInvalidMessage("doesn't exist",
                              "CREATE AGGREGATE " + KEYSPACE + ".aggrInvalid(int)" +
                              "SFUNC " + shortFunctionName(fState) + " " +
                              "STYPE int " +
@@ -805,7 +813,7 @@ public class AggregationTest extends CQLTester
     @Test
     public void testJavaAggregateWithoutStateOrFinal() throws Throwable
     {
-        assertInvalidMessage("does not exist or is not a scalar function",
+        assertInvalidMessage("doesn't exist",
                              "CREATE AGGREGATE " + KEYSPACE + ".jSumFooNE1(int) " +
                              "SFUNC jSumFooNEstate " +
                              "STYPE int");
@@ -818,7 +826,7 @@ public class AggregationTest extends CQLTester
                                   "LANGUAGE java " +
                                   "AS 'return Integer.valueOf(a + b);'");
 
-        assertInvalidMessage("does not exist or is not a scalar function",
+        assertInvalidMessage("doesn't exist",
                              "CREATE AGGREGATE " + KEYSPACE + ".jSumFooNE2(int) " +
                              "SFUNC " + shortFunctionName(f) + " " +
                              "STYPE int " +
@@ -1144,12 +1152,12 @@ public class AggregationTest extends CQLTester
                                    "SFUNC " + shortFunctionName(fState) + " " +
                                    "STYPE int ");
 
-        assertInvalidMessage("does not exist or is not a scalar function",
+        assertInvalidMessage("doesn't exist",
                              "CREATE AGGREGATE " + KEYSPACE + ".aggInv(int) " +
                              "SFUNC " + shortFunctionName(a) + " " +
                              "STYPE int ");
 
-        assertInvalidMessage("does not exist or is not a scalar function",
+        assertInvalidMessage("isn't a scalar function",
                              "CREATE AGGREGATE " + KEYSPACE + ".aggInv(int) " +
                              "SFUNC " + shortFunctionName(fState) + " " +
                              "STYPE int " +
@@ -1291,41 +1299,6 @@ public class AggregationTest extends CQLTester
     }
 
     @Test
-    public void testBrokenAggregate() throws Throwable
-    {
-        createTable("CREATE TABLE %s (key int primary key, val int)");
-        execute("INSERT INTO %s (key, val) VALUES (?, ?)", 1, 1);
-
-        String fState = createFunction(KEYSPACE,
-                                       "int, int",
-                                       "CREATE FUNCTION %s(a int, b int) " +
-                                       "CALLED ON NULL INPUT " +
-                                       "RETURNS int " +
-                                       "LANGUAGE javascript " +
-                                       "AS 'a + b;'");
-
-        String a = createAggregate(KEYSPACE,
-                                   "int",
-                                   "CREATE AGGREGATE %s(int) " +
-                                   "SFUNC " + shortFunctionName(fState) + " " +
-                                   "STYPE int ");
-
-        KeyspaceMetadata ksm = Schema.instance.getKeyspaceMetadata(keyspace());
-        UDAggregate f = (UDAggregate) ksm.functions.get(parseFunctionName(a)).iterator().next();
-
-        UDAggregate broken = UDAggregate.createBroken(f.name(),
-                                                      f.argTypes(),
-                                                      f.returnType(),
-                                                      null,
-                                                      new InvalidRequestException("foo bar is broken"));
-
-        Schema.instance.load(ksm.withSwapped(ksm.functions.without(f.name(), f.argTypes()).with(broken)));
-
-        assertInvalidThrowMessage("foo bar is broken", InvalidRequestException.class,
-                                  "SELECT " + a + "(val) FROM %s");
-    }
-
-    @Test
     public void testWrongStateType() throws Throwable
     {
         createTable("CREATE TABLE %s (key int primary key, val int)");
@@ -1450,7 +1423,7 @@ public class AggregationTest extends CQLTester
                                        "LANGUAGE java " +
                                        "AS 'return state;'");
 
-        assertInvalidMessage("The function state type should not be frozen",
+        assertInvalidMessage("cannot be frozen",
                              "CREATE AGGREGATE %s(set<int>) " +
                              "SFUNC " + parseFunctionName(fState).name + ' ' +
                              "STYPE frozen<set<int>> " +
@@ -1471,7 +1444,7 @@ public class AggregationTest extends CQLTester
         assertRows(execute("SELECT " + aggregation + "(b) FROM %s"),
                    row(set(7, 8, 9)));
 
-        assertInvalidMessage("The function arguments should not be frozen",
+        assertInvalidMessage("Argument 'frozen<set<int>>' cannot be frozen; remove frozen<> modifier from 'frozen<set<int>>'",
                              "DROP AGGREGATE %s (frozen<set<int>>);");
     }
 
@@ -1502,7 +1475,7 @@ public class AggregationTest extends CQLTester
                                        "LANGUAGE java " +
                                        "AS 'return state;'");
 
-        assertInvalidMessage("The function state type should not be frozen",
+        assertInvalidMessage("cannot be frozen",
                              "CREATE AGGREGATE %s(list<int>) " +
                              "SFUNC " + parseFunctionName(fState).name + ' ' +
                              "STYPE frozen<list<int>> " +
@@ -1520,7 +1493,7 @@ public class AggregationTest extends CQLTester
         assertRows(execute("SELECT " + aggregation + "(b) FROM %s"),
                    row(list(7, 8, 9)));
 
-        assertInvalidMessage("The function arguments should not be frozen",
+        assertInvalidMessage("Argument 'frozen<list<int>>' cannot be frozen; remove frozen<> modifier from 'frozen<list<int>>'",
                              "DROP AGGREGATE %s (frozen<list<int>>);");
     }
 
@@ -1551,7 +1524,7 @@ public class AggregationTest extends CQLTester
                                        "LANGUAGE java " +
                                        "AS 'return state;'");
 
-        assertInvalidMessage("The function state type should not be frozen",
+        assertInvalidMessage("cannot be frozen",
                              "CREATE AGGREGATE %s(map<int, int>) " +
                              "SFUNC " + parseFunctionName(fState).name + ' ' +
                              "STYPE frozen<map<int, int>> " +
@@ -1569,7 +1542,7 @@ public class AggregationTest extends CQLTester
         assertRows(execute("SELECT " + aggregation + "(b) FROM %s"),
                    row(map(7, 8, 9, 10)));
 
-        assertInvalidMessage("The function arguments should not be frozen",
+        assertInvalidMessage("Argument 'frozen<map<int, int>>' cannot be frozen; remove frozen<> modifier from 'frozen<map<int, int>>'",
                              "DROP AGGREGATE %s (frozen<map<int, int>>);");
     }
 
@@ -1600,7 +1573,7 @@ public class AggregationTest extends CQLTester
                                        "LANGUAGE java " +
                                        "AS 'return state;'");
 
-        assertInvalidMessage("The function state type should not be frozen",
+        assertInvalidMessage("cannot be frozen",
                              "CREATE AGGREGATE %s(tuple<int, int>) " +
                              "SFUNC " + parseFunctionName(fState).name + ' ' +
                              "STYPE frozen<tuple<int, int>> " +
@@ -1618,7 +1591,7 @@ public class AggregationTest extends CQLTester
         assertRows(execute("SELECT " + aggregation + "(b) FROM %s"),
                    row(tuple(7, 8)));
 
-        assertInvalidMessage("The function arguments should not be frozen",
+        assertInvalidMessage("Argument 'tuple<int, int>' cannot be frozen; remove frozen<> modifier from 'tuple<int, int>'",
                              "DROP AGGREGATE %s (frozen<tuple<int, int>>);");
     }
 
@@ -1650,7 +1623,7 @@ public class AggregationTest extends CQLTester
                                        "LANGUAGE java " +
                                        "AS 'return state;'");
 
-        assertInvalidMessage("The function state type should not be frozen",
+        assertInvalidMessage("cannot be frozen",
                              "CREATE AGGREGATE %s(" + myType + ") " +
                              "SFUNC " + parseFunctionName(fState).name + ' ' +
                              "STYPE frozen<" + myType + "> " +
@@ -1668,7 +1641,7 @@ public class AggregationTest extends CQLTester
         assertRows(execute("SELECT " + aggregation + "(b).f FROM %s"),
                    row(7));
 
-        assertInvalidMessage("The function arguments should not be frozen",
+        assertInvalidMessage(String.format("Argument 'frozen<%s>' cannot be frozen; remove frozen<> modifier from 'frozen<%s>'", myType, myType),
                              "DROP AGGREGATE %s (frozen<" + myType + ">);");
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/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 c6f255a..a792bcb 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/AlterTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/AlterTest.java
@@ -322,7 +322,7 @@ public class AlterTest extends CQLTester
         execute("insert into %s (k, v) VALUES (0, {'f'})");
         flush();
         execute("alter table %s drop v");
-        execute("alter table %s add v int");
+        execute("alter table %s add v1 int");
     }
 
     @Test
@@ -358,7 +358,7 @@ public class AlterTest extends CQLTester
                            currentTable()),
                    row(map("chunk_length_in_kb", "32", "class", "org.apache.cassandra.io.compress.SnappyCompressor")));
 
-        execute("ALTER TABLE %s WITH compression = { 'sstable_compression' : 'LZ4Compressor', 'chunk_length_kb' : 64 };");
+        execute("ALTER TABLE %s WITH compression = { 'class' : 'LZ4Compressor', 'chunk_length_in_kb' : 64 };");
 
         assertRows(execute(format("SELECT compression FROM %s.%s WHERE keyspace_name = ? and table_name = ?;",
                                   SchemaConstants.SCHEMA_KEYSPACE_NAME,
@@ -367,7 +367,7 @@ public class AlterTest extends CQLTester
                            currentTable()),
                    row(map("chunk_length_in_kb", "64", "class", "org.apache.cassandra.io.compress.LZ4Compressor")));
 
-        execute("ALTER TABLE %s WITH compression = { 'sstable_compression' : 'LZ4Compressor', 'min_compress_ratio' : 2 };");
+        execute("ALTER TABLE %s WITH compression = { 'class' : 'LZ4Compressor', 'min_compress_ratio' : 2 };");
 
         assertRows(execute(format("SELECT compression FROM %s.%s WHERE keyspace_name = ? and table_name = ?;",
                                   SchemaConstants.SCHEMA_KEYSPACE_NAME,
@@ -376,7 +376,7 @@ public class AlterTest extends CQLTester
                            currentTable()),
                    row(map("chunk_length_in_kb", "64", "class", "org.apache.cassandra.io.compress.LZ4Compressor", "min_compress_ratio", "2.0")));
 
-        execute("ALTER TABLE %s WITH compression = { 'sstable_compression' : 'LZ4Compressor', 'min_compress_ratio' : 1 };");
+        execute("ALTER TABLE %s WITH compression = { 'class' : 'LZ4Compressor', 'min_compress_ratio' : 1 };");
 
         assertRows(execute(format("SELECT compression FROM %s.%s WHERE keyspace_name = ? and table_name = ?;",
                                   SchemaConstants.SCHEMA_KEYSPACE_NAME,
@@ -385,7 +385,7 @@ public class AlterTest extends CQLTester
                            currentTable()),
                    row(map("chunk_length_in_kb", "64", "class", "org.apache.cassandra.io.compress.LZ4Compressor", "min_compress_ratio", "1.0")));
 
-        execute("ALTER TABLE %s WITH compression = { 'sstable_compression' : 'LZ4Compressor', 'min_compress_ratio' : 0 };");
+        execute("ALTER TABLE %s WITH compression = { 'class' : 'LZ4Compressor', 'min_compress_ratio' : 0 };");
 
         assertRows(execute(format("SELECT compression FROM %s.%s WHERE keyspace_name = ? and table_name = ?;",
                                   SchemaConstants.SCHEMA_KEYSPACE_NAME,
@@ -394,15 +394,6 @@ public class AlterTest extends CQLTester
                            currentTable()),
                    row(map("chunk_length_in_kb", "64", "class", "org.apache.cassandra.io.compress.LZ4Compressor")));
 
-        execute("ALTER TABLE %s WITH compression = { 'sstable_compression' : '', 'chunk_length_kb' : 32 };");
-
-        assertRows(execute(format("SELECT compression FROM %s.%s WHERE keyspace_name = ? and table_name = ?;",
-                                  SchemaConstants.SCHEMA_KEYSPACE_NAME,
-                                  SchemaKeyspace.TABLES),
-                           KEYSPACE,
-                           currentTable()),
-                   row(map("enabled", "false")));
-
         execute("ALTER TABLE %s WITH compression = { 'class' : 'SnappyCompressor', 'chunk_length_in_kb' : 32 };");
         execute("ALTER TABLE %s WITH compression = { 'enabled' : 'false'};");
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/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 96f88c3..184c5ad 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/CreateTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/CreateTest.java
@@ -25,22 +25,21 @@ import java.util.UUID;
 
 import org.junit.Test;
 
-import org.apache.cassandra.locator.InetAddressAndPort;
-import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.schema.Schema;
-import org.apache.cassandra.schema.SchemaConstants;
 import org.apache.cassandra.cql3.CQLTester;
 import org.apache.cassandra.cql3.Duration;
 import org.apache.cassandra.db.Mutation;
 import org.apache.cassandra.db.partitions.Partition;
 import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.exceptions.SyntaxException;
+import org.apache.cassandra.io.util.DataOutputBuffer;
 import org.apache.cassandra.locator.AbstractEndpointSnitch;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.locator.IEndpointSnitch;
-import org.apache.cassandra.io.util.DataOutputBuffer;
 import org.apache.cassandra.schema.SchemaKeyspace;
 import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.schema.*;
 import org.apache.cassandra.triggers.ITrigger;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
@@ -91,14 +90,14 @@ public class CreateTest extends CQLTester
     @Test
     public void testCreateTableWithDurationColumns() throws Throwable
     {
-        assertInvalidMessage("duration type is not supported for PRIMARY KEY part a",
-                             "CREATE TABLE test (a duration PRIMARY KEY, b int);");
+        assertInvalidMessage("duration type is not supported for PRIMARY KEY column 'a'",
+                             "CREATE TABLE cql_test_keyspace.table0 (a duration PRIMARY KEY, b int);");
 
-        assertInvalidMessage("duration type is not supported for PRIMARY KEY part b",
-                             "CREATE TABLE test (a text, b duration, c duration, primary key (a, b));");
+        assertInvalidMessage("duration type is not supported for PRIMARY KEY column 'b'",
+                             "CREATE TABLE cql_test_keyspace.table0 (a text, b duration, c duration, primary key (a, b));");
 
-        assertInvalidMessage("duration type is not supported for PRIMARY KEY part b",
-                             "CREATE TABLE test (a text, b duration, c duration, primary key (a, b)) with clustering order by (b DESC);");
+        assertInvalidMessage("duration type is not supported for PRIMARY KEY column 'b'",
+                             "CREATE TABLE cql_test_keyspace.table0 (a text, b duration, c duration, primary key (a, b)) with clustering order by (b DESC);");
 
         createTable("CREATE TABLE %s (a int, b int, c duration, primary key (a, b));");
         execute("INSERT INTO %s (a, b, c) VALUES (1, 1, 1y2mo)");
@@ -178,25 +177,25 @@ public class CreateTest extends CQLTester
 
         // Test duration within Map
         assertInvalidMessage("Durations are not allowed as map keys: map<duration, text>",
-                             "CREATE TABLE test(pk int PRIMARY KEY, m map<duration, text>)");
+                             "CREATE TABLE cql_test_keyspace.table0(pk int PRIMARY KEY, m map<duration, text>)");
 
         createTable("CREATE TABLE %s(pk int PRIMARY KEY, m map<text, duration>)");
         execute("INSERT INTO %s (pk, m) VALUES (1, {'one month' : 1mo, '60 days' : 60d})");
         assertRows(execute("SELECT * FROM %s"),
                    row(1, map("one month", Duration.from("1mo"), "60 days", Duration.from("60d"))));
 
-        assertInvalidMessage("duration type is not supported for PRIMARY KEY part m",
-                "CREATE TABLE %s(m frozen<map<text, duration>> PRIMARY KEY, v int)");
+        assertInvalidMessage("duration type is not supported for PRIMARY KEY column 'm'",
+                "CREATE TABLE cql_test_keyspace.table0(m frozen<map<text, duration>> PRIMARY KEY, v int)");
 
-        assertInvalidMessage("duration type is not supported for PRIMARY KEY part m",
-                             "CREATE TABLE %s(pk int, m frozen<map<text, duration>>, v int, PRIMARY KEY (pk, m))");
+        assertInvalidMessage("duration type is not supported for PRIMARY KEY column 'm'",
+                             "CREATE TABLE cql_test_keyspace.table0(pk int, m frozen<map<text, duration>>, v int, PRIMARY KEY (pk, m))");
 
         // Test duration within Set
         assertInvalidMessage("Durations are not allowed inside sets: set<duration>",
-                             "CREATE TABLE %s(pk int PRIMARY KEY, s set<duration>)");
+                             "CREATE TABLE cql_test_keyspace.table0(pk int PRIMARY KEY, s set<duration>)");
 
         assertInvalidMessage("Durations are not allowed inside sets: frozen<set<duration>>",
-                             "CREATE TABLE %s(s frozen<set<duration>> PRIMARY KEY, v int)");
+                             "CREATE TABLE cql_test_keyspace.table0(s frozen<set<duration>> PRIMARY KEY, v int)");
 
         // Test duration within List
         createTable("CREATE TABLE %s(pk int PRIMARY KEY, l list<duration>)");
@@ -204,8 +203,8 @@ public class CreateTest extends CQLTester
         assertRows(execute("SELECT * FROM %s"),
                    row(1, list(Duration.from("1mo"), Duration.from("60d"))));
 
-        assertInvalidMessage("duration type is not supported for PRIMARY KEY part l",
-                             "CREATE TABLE %s(l frozen<list<duration>> PRIMARY KEY, v int)");
+        assertInvalidMessage("duration type is not supported for PRIMARY KEY column 'l'",
+                             "CREATE TABLE cql_test_keyspace.table0(l frozen<list<duration>> PRIMARY KEY, v int)");
 
         // Test duration within Tuple
         createTable("CREATE TABLE %s(pk int PRIMARY KEY, t tuple<int, duration>)");
@@ -213,8 +212,8 @@ public class CreateTest extends CQLTester
         assertRows(execute("SELECT * FROM %s"),
                    row(1, tuple(1, Duration.from("1mo"))));
 
-        assertInvalidMessage("duration type is not supported for PRIMARY KEY part t",
-                             "CREATE TABLE %s(t frozen<tuple<int, duration>> PRIMARY KEY, v int)");
+        assertInvalidMessage("duration type is not supported for PRIMARY KEY column 't'",
+                             "CREATE TABLE cql_test_keyspace.table0(t frozen<tuple<int, duration>> PRIMARY KEY, v int)");
 
         // Test duration within UDT
         String typename = createType("CREATE TYPE %s (a duration)");
@@ -224,12 +223,12 @@ public class CreateTest extends CQLTester
         assertRows(execute("SELECT * FROM %s"),
                    row(1, userType("a", Duration.from("1mo"))));
 
-        assertInvalidMessage("duration type is not supported for PRIMARY KEY part u",
-                             "CREATE TABLE %s(pk int, u frozen<" + myType + ">, v int, PRIMARY KEY(pk, u))");
+        assertInvalidMessage("duration type is not supported for PRIMARY KEY column 'u'",
+                             "CREATE TABLE cql_test_keyspace.table0(pk int, u frozen<" + myType + ">, v int, PRIMARY KEY(pk, u))");
 
         // Test duration with several level of depth
-        assertInvalidMessage("duration type is not supported for PRIMARY KEY part m",
-                "CREATE TABLE %s(pk int, m frozen<map<text, list<tuple<int, duration>>>>, v int, PRIMARY KEY (pk, m))");
+        assertInvalidMessage("duration type is not supported for PRIMARY KEY column 'm'",
+                "CREATE TABLE cql_test_keyspace.table0(pk int, m frozen<map<text, list<tuple<int, duration>>>>, v int, PRIMARY KEY (pk, m))");
     }
 
     private ByteBuffer duration(long months, long days, long nanoseconds) throws IOException
@@ -336,7 +335,7 @@ public class CreateTest extends CQLTester
     @Test
     public void testObsoleteTableProperties() throws Throwable
     {
-        assertInvalidThrow(SyntaxException.class, "CREATE TABLE test (foo text PRIMARY KEY, c int) WITH default_validation=timestamp");
+        assertInvalidThrow(SyntaxException.class, "CREATE TABLE cql_test_keyspace.table0 (foo text PRIMARY KEY, c int) WITH default_validation=timestamp");
 
         createTable("CREATE TABLE %s (foo text PRIMARY KEY, c int)");
         assertInvalidThrow(SyntaxException.class, "ALTER TABLE %s WITH default_validation=int");
@@ -357,7 +356,7 @@ public class CreateTest extends CQLTester
                      "CREATE KEYSPACE My_much_much_too_long_identifier_that_should_not_work WITH replication = { 'class' : 'SimpleStrategy', 'replication_factor' : 1 }");
 
         execute("DROP KEYSPACE testXYZ");
-        assertInvalidThrow(ConfigurationException.class, "DROP KEYSPACE non_existing");
+        assertInvalidThrow(InvalidRequestException.class, "DROP KEYSPACE non_existing");
 
         execute("CREATE KEYSPACE testXYZ WITH replication = { 'class' : 'SimpleStrategy', 'replication_factor' : 1 }");
 
@@ -419,7 +418,15 @@ public class CreateTest extends CQLTester
         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));
+        assertInvalidMessage("Duplicate column 'k' declaration for table", 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));
     }
 
     /**
@@ -682,7 +689,7 @@ public class CreateTest extends CQLTester
     @Test
     public void compactTableTest() throws Throwable
     {
-        assertInvalidMessage("Compact tables are not allowed in Cassandra starting with 4.0 version.",
+        assertInvalidMessage("COMPACT STORAGE tables are not allowed starting with version 4.0",
                              "CREATE TABLE compact_table_create (id text PRIMARY KEY, content text) WITH COMPACT STORAGE;");
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/test/unit/org/apache/cassandra/cql3/validation/operations/DropTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/DropTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/DropTest.java
index 692eb45..90130ad 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/DropTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/DropTest.java
@@ -27,8 +27,8 @@ public class DropTest extends CQLTester
     @Test
     public void testNonExistingOnes() throws Throwable
     {
-        assertInvalidMessage("Cannot drop non existing table", "DROP TABLE " + KEYSPACE + ".table_does_not_exist");
-        assertInvalidMessage("Cannot drop table in unknown keyspace", "DROP TABLE keyspace_does_not_exist.table_does_not_exist");
+        assertInvalidMessage(String.format("Table '%s.table_does_not_exist' doesn't exist", KEYSPACE),  "DROP TABLE " + KEYSPACE + ".table_does_not_exist");
+        assertInvalidMessage("Table 'keyspace_does_not_exist.table_does_not_exist' doesn't exist", "DROP TABLE keyspace_does_not_exist.table_does_not_exist");
 
         execute("DROP TABLE IF EXISTS " + KEYSPACE + ".table_does_not_exist");
         execute("DROP TABLE IF EXISTS keyspace_does_not_exist.table_does_not_exist");

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/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 b40a13e..edfe57a 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/InsertUpdateIfConditionTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/InsertUpdateIfConditionTest.java
@@ -1414,10 +1414,8 @@ public class InsertUpdateIfConditionTest extends CQLTester
     {
         String tableName = createTable("CREATE TABLE %s (id text PRIMARY KEY, value1 blob, value2 blob)with comment = 'foo'");
 
-        execute("use " + KEYSPACE);
-
         // try dropping when doesn't exist
-        schemaChange("DROP INDEX IF EXISTS myindex");
+        schemaChange(format("DROP INDEX IF EXISTS %s.myindex", KEYSPACE));
 
         // create and confirm
         createIndex("CREATE INDEX IF NOT EXISTS myindex ON %s (value1)");
@@ -1428,7 +1426,7 @@ public class InsertUpdateIfConditionTest extends CQLTester
         execute("CREATE INDEX IF NOT EXISTS myindex ON %s (value1)");
 
         // drop and confirm
-        execute("DROP INDEX IF EXISTS myindex");
+        execute(format("DROP INDEX IF EXISTS %s.myindex", KEYSPACE));
 
         Object[][] rows = getRows(execute("select index_name from system.\"IndexInfo\" where table_name = ?", tableName));
         assertEquals(0, rows.length);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/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 09fd464..3795ce5 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/SelectSingleColumnRelationTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/SelectSingleColumnRelationTest.java
@@ -206,7 +206,7 @@ public class SelectSingleColumnRelationTest extends CQLTester
     @Test
     public void testClusteringColumnRelationsWithClusteringOrder() throws Throwable
     {
-        createTable("CREATE TABLE %s (a text, b int, c int, d int, primary key(a, b, c)) WITH CLUSTERING ORDER BY (b DESC);");
+        createTable("CREATE TABLE %s (a text, b int, c int, d int, primary key(a, b, c)) WITH CLUSTERING ORDER BY (b DESC, c ASC);");
         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);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/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 4f12b2b..54e079e 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/SelectTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/SelectTest.java
@@ -1859,7 +1859,7 @@ public class SelectTest extends CQLTester
         });
 
         // test clutering order
-        createTable("CREATE TABLE %s (a int, b int, c int, d int, e int, PRIMARY KEY ((a, b), c, d)) WITH CLUSTERING ORDER BY (c DESC)");
+        createTable("CREATE TABLE %s (a int, b int, c int, d int, e int, PRIMARY KEY ((a, b), c, d)) WITH CLUSTERING ORDER BY (c DESC, d ASC)");
 
         execute("INSERT INTO %s (a,b,c,d,e) VALUES (11, 11, 13, 14, 15)");
         execute("INSERT INTO %s (a,b,c,d,e) VALUES (11, 11, 14, 17, 18)");
@@ -2415,7 +2415,7 @@ public class SelectTest extends CQLTester
                    row("a", 3));
 
         // compound, first column DESC order
-        createTable("CREATE TABLE %s (a text, b int, c int, PRIMARY KEY (a, b, c)) WITH CLUSTERING ORDER BY (b DESC)");
+        createTable("CREATE TABLE %s (a text, b int, c int, PRIMARY KEY (a, b, c)) WITH CLUSTERING ORDER BY (b DESC, c ASC)");
 
         execute("INSERT INTO %s (a, b, c) VALUES ('a', 2, 4)");
         execute("INSERT INTO %s (a, b, c) VALUES ('a', 3, 5)");

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/test/unit/org/apache/cassandra/db/DirectoriesTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/DirectoriesTest.java b/test/unit/org/apache/cassandra/db/DirectoriesTest.java
index bbbb4e6..fa23560 100644
--- a/test/unit/org/apache/cassandra/db/DirectoriesTest.java
+++ b/test/unit/org/apache/cassandra/db/DirectoriesTest.java
@@ -35,7 +35,7 @@ import org.apache.cassandra.schema.Indexes;
 import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.config.Config.DiskFailurePolicy;
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.cql3.statements.IndexTarget;
+import org.apache.cassandra.cql3.statements.schema.IndexTarget;
 import org.apache.cassandra.db.Directories.DataDirectory;
 import org.apache.cassandra.db.marshal.UTF8Type;
 import org.apache.cassandra.index.internal.CassandraIndex;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/test/unit/org/apache/cassandra/db/RangeTombstoneTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/RangeTombstoneTest.java b/test/unit/org/apache/cassandra/db/RangeTombstoneTest.java
index b2c1542..3d1d003 100644
--- a/test/unit/org/apache/cassandra/db/RangeTombstoneTest.java
+++ b/test/unit/org/apache/cassandra/db/RangeTombstoneTest.java
@@ -32,7 +32,7 @@ import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.UpdateBuilder;
 import org.apache.cassandra.Util;
 import org.apache.cassandra.schema.ColumnMetadata;
-import org.apache.cassandra.cql3.statements.IndexTarget;
+import org.apache.cassandra.cql3.statements.schema.IndexTarget;
 import org.apache.cassandra.db.compaction.CompactionManager;
 import org.apache.cassandra.db.filter.ColumnFilter;
 import org.apache.cassandra.db.marshal.Int32Type;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/test/unit/org/apache/cassandra/db/RowIndexEntryTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/RowIndexEntryTest.java b/test/unit/org/apache/cassandra/db/RowIndexEntryTest.java
index e563070..d1c4527 100644
--- a/test/unit/org/apache/cassandra/db/RowIndexEntryTest.java
+++ b/test/unit/org/apache/cassandra/db/RowIndexEntryTest.java
@@ -33,7 +33,7 @@ import org.junit.Test;
 
 import org.apache.cassandra.Util;
 import org.apache.cassandra.cache.IMeasurableMemory;
-import org.apache.cassandra.cql3.statements.CreateTableStatement;
+import org.apache.cassandra.cql3.statements.schema.CreateTableStatement;
 import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.cql3.CQLTester;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/test/unit/org/apache/cassandra/db/SecondaryIndexTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/SecondaryIndexTest.java b/test/unit/org/apache/cassandra/db/SecondaryIndexTest.java
index 38b0a39..5dadc11 100644
--- a/test/unit/org/apache/cassandra/db/SecondaryIndexTest.java
+++ b/test/unit/org/apache/cassandra/db/SecondaryIndexTest.java
@@ -32,7 +32,7 @@ import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
 import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.cql3.Operator;
-import org.apache.cassandra.cql3.statements.IndexTarget;
+import org.apache.cassandra.cql3.statements.schema.IndexTarget;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.db.partitions.*;
 import org.apache.cassandra.db.rows.Row;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/test/unit/org/apache/cassandra/db/TableCQLHelperTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/TableCQLHelperTest.java b/test/unit/org/apache/cassandra/db/TableCQLHelperTest.java
index 701afbb..32fa4e4 100644
--- a/test/unit/org/apache/cassandra/db/TableCQLHelperTest.java
+++ b/test/unit/org/apache/cassandra/db/TableCQLHelperTest.java
@@ -32,7 +32,7 @@ import org.junit.Test;
 
 import org.apache.cassandra.*;
 import org.apache.cassandra.cql3.*;
-import org.apache.cassandra.cql3.statements.*;
+import org.apache.cassandra.cql3.statements.schema.IndexTarget;
 import org.apache.cassandra.db.marshal.*;
 import org.apache.cassandra.exceptions.*;
 import org.apache.cassandra.index.sasi.*;
@@ -406,7 +406,7 @@ public class TableCQLHelperTest extends CQLTester
         String tableName = createTable("CREATE TABLE IF NOT EXISTS %s (" +
                                        "pk1 varint," +
                                        "ck1 varint," +
-                                       "reg1 " + typeB + "," +
+                                       "reg1 frozen<" + typeB + ">," +
                                        "reg2 varint," +
                                        "PRIMARY KEY (pk1, ck1));");
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/test/unit/org/apache/cassandra/db/compaction/AbstractPendingRepairTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/compaction/AbstractPendingRepairTest.java b/test/unit/org/apache/cassandra/db/compaction/AbstractPendingRepairTest.java
index 3e38dfc..a320248 100644
--- a/test/unit/org/apache/cassandra/db/compaction/AbstractPendingRepairTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/AbstractPendingRepairTest.java
@@ -29,13 +29,11 @@ import org.junit.BeforeClass;
 import org.junit.Ignore;
 
 import org.apache.cassandra.SchemaLoader;
-import org.apache.cassandra.cql3.statements.CreateTableStatement;
-import org.apache.cassandra.locator.InetAddressAndPort;
-import org.apache.cassandra.schema.TableMetadata;
-import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.cql3.QueryProcessor;
+import org.apache.cassandra.cql3.statements.schema.CreateTableStatement;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.net.IMessageSink;
 import org.apache.cassandra.net.MessageIn;
 import org.apache.cassandra.net.MessageOut;
@@ -43,6 +41,8 @@ import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.repair.AbstractRepairTest;
 import org.apache.cassandra.repair.consistent.LocalSessionAccessor;
 import org.apache.cassandra.schema.KeyspaceParams;
+import org.apache.cassandra.schema.Schema;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.service.ActiveRepairService;
 
 @Ignore

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/test/unit/org/apache/cassandra/db/compaction/CompactionTaskTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/compaction/CompactionTaskTest.java b/test/unit/org/apache/cassandra/db/compaction/CompactionTaskTest.java
index 4640248..599fc74 100644
--- a/test/unit/org/apache/cassandra/db/compaction/CompactionTaskTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/CompactionTaskTest.java
@@ -32,7 +32,7 @@ import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.cql3.QueryProcessor;
-import org.apache.cassandra.cql3.statements.CreateTableStatement;
+import org.apache.cassandra.cql3.statements.schema.CreateTableStatement;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
 import org.apache.cassandra.io.sstable.format.SSTableReader;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/test/unit/org/apache/cassandra/db/compaction/CompactionsPurgeTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/compaction/CompactionsPurgeTest.java b/test/unit/org/apache/cassandra/db/compaction/CompactionsPurgeTest.java
index fc193e8..4573485 100644
--- a/test/unit/org/apache/cassandra/db/compaction/CompactionsPurgeTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/CompactionsPurgeTest.java
@@ -27,9 +27,9 @@ import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
-import org.apache.cassandra.cql3.statements.CreateTableStatement;
 import org.apache.cassandra.cql3.QueryProcessor;
 import org.apache.cassandra.cql3.UntypedResultSet;
+import org.apache.cassandra.cql3.statements.schema.CreateTableStatement;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.partitions.ImmutableBTreePartition;
 import org.apache.cassandra.db.rows.Row;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/test/unit/org/apache/cassandra/db/repair/CompactionManagerGetSSTablesForValidationTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/repair/CompactionManagerGetSSTablesForValidationTest.java b/test/unit/org/apache/cassandra/db/repair/CompactionManagerGetSSTablesForValidationTest.java
index 365ad7e..76ebfd8 100644
--- a/test/unit/org/apache/cassandra/db/repair/CompactionManagerGetSSTablesForValidationTest.java
+++ b/test/unit/org/apache/cassandra/db/repair/CompactionManagerGetSSTablesForValidationTest.java
@@ -31,9 +31,7 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
-import org.apache.cassandra.cql3.statements.CreateTableStatement;
-import org.apache.cassandra.db.compaction.CompactionManager;
-import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.cql3.statements.schema.CreateTableStatement;
 import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.schema.Schema;
@@ -42,6 +40,7 @@ import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.streaming.PreviewKind;
 import org.apache.cassandra.repair.RepairJobDesc;
 import org.apache.cassandra.repair.Validator;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/test/unit/org/apache/cassandra/db/repair/PendingAntiCompactionTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/repair/PendingAntiCompactionTest.java b/test/unit/org/apache/cassandra/db/repair/PendingAntiCompactionTest.java
index 269a725..447d504 100644
--- a/test/unit/org/apache/cassandra/db/repair/PendingAntiCompactionTest.java
+++ b/test/unit/org/apache/cassandra/db/repair/PendingAntiCompactionTest.java
@@ -41,21 +41,21 @@ import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.cql3.statements.CreateTableStatement;
-import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.db.compaction.CompactionManager;
 import org.apache.cassandra.schema.TableId;
 import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.cql3.QueryProcessor;
+import org.apache.cassandra.cql3.statements.schema.CreateTableStatement;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.dht.ByteOrderedPartitioner;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
-import org.apache.cassandra.streaming.PreviewKind;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.schema.KeyspaceParams;
 import org.apache.cassandra.service.ActiveRepairService;
+import org.apache.cassandra.streaming.PreviewKind;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.UUIDGen;
 import org.apache.cassandra.utils.concurrent.Transactional;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/test/unit/org/apache/cassandra/db/streaming/CassandraStreamHeaderTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/streaming/CassandraStreamHeaderTest.java b/test/unit/org/apache/cassandra/db/streaming/CassandraStreamHeaderTest.java
index 289bb0f..061a4b2 100644
--- a/test/unit/org/apache/cassandra/db/streaming/CassandraStreamHeaderTest.java
+++ b/test/unit/org/apache/cassandra/db/streaming/CassandraStreamHeaderTest.java
@@ -22,7 +22,7 @@ import java.util.ArrayList;
 
 import org.junit.Test;
 
-import org.apache.cassandra.cql3.statements.CreateTableStatement;
+import org.apache.cassandra.cql3.statements.schema.CreateTableStatement;
 import org.apache.cassandra.db.SerializationHeader;
 import org.apache.cassandra.io.compress.CompressionMetadata;
 import org.apache.cassandra.io.sstable.format.SSTableFormat;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/test/unit/org/apache/cassandra/db/streaming/CassandraStreamManagerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/streaming/CassandraStreamManagerTest.java b/test/unit/org/apache/cassandra/db/streaming/CassandraStreamManagerTest.java
index 80e4bfb..86018af 100644
--- a/test/unit/org/apache/cassandra/db/streaming/CassandraStreamManagerTest.java
+++ b/test/unit/org/apache/cassandra/db/streaming/CassandraStreamManagerTest.java
@@ -42,7 +42,7 @@ import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.concurrent.NamedThreadFactory;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.cql3.QueryProcessor;
-import org.apache.cassandra.cql3.statements.CreateTableStatement;
+import org.apache.cassandra.cql3.statements.schema.CreateTableStatement;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.dht.IPartitioner;
 import org.apache.cassandra.dht.Range;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/test/unit/org/apache/cassandra/hints/HintWriteTTLTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/hints/HintWriteTTLTest.java b/test/unit/org/apache/cassandra/hints/HintWriteTTLTest.java
index b06187d..21dbd7e 100644
--- a/test/unit/org/apache/cassandra/hints/HintWriteTTLTest.java
+++ b/test/unit/org/apache/cassandra/hints/HintWriteTTLTest.java
@@ -32,7 +32,7 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
-import org.apache.cassandra.cql3.statements.CreateTableStatement;
+import org.apache.cassandra.cql3.statements.schema.CreateTableStatement;
 import org.apache.cassandra.db.DecoratedKey;
 import org.apache.cassandra.db.Mutation;
 import org.apache.cassandra.db.partitions.PartitionUpdate;


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


[06/13] cassandra git commit: Make all DDL statements idempotent and not dependent on global state

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/cql3/statements/schema/DropIndexStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/schema/DropIndexStatement.java b/src/java/org/apache/cassandra/cql3/statements/schema/DropIndexStatement.java
new file mode 100644
index 0000000..6a6f8d9
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/statements/schema/DropIndexStatement.java
@@ -0,0 +1,110 @@
+/*
+ * 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.statements.schema;
+
+import org.apache.cassandra.audit.AuditLogContext;
+import org.apache.cassandra.audit.AuditLogEntryType;
+import org.apache.cassandra.auth.Permission;
+import org.apache.cassandra.cql3.CQLStatement;
+import org.apache.cassandra.cql3.QualifiedName;
+import org.apache.cassandra.schema.Diff;
+import org.apache.cassandra.schema.*;
+import org.apache.cassandra.schema.KeyspaceMetadata.KeyspaceDiff;
+import org.apache.cassandra.schema.Keyspaces.KeyspacesDiff;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.transport.Event.SchemaChange;
+import org.apache.cassandra.transport.Event.SchemaChange.Change;
+import org.apache.cassandra.transport.Event.SchemaChange.Target;
+
+public final class DropIndexStatement extends AlterSchemaStatement
+{
+    private final String indexName;
+    private final boolean ifExists;
+
+    public DropIndexStatement(String keyspaceName, String indexName, boolean ifExists)
+    {
+        super(keyspaceName);
+        this.indexName = indexName;
+        this.ifExists = ifExists;
+    }
+
+    public Keyspaces apply(Keyspaces schema)
+    {
+        KeyspaceMetadata keyspace = schema.getNullable(keyspaceName);
+
+        TableMetadata table = null == keyspace
+                            ? null
+                            : keyspace.findIndexedTable(indexName).orElse(null);
+
+        if (null == table)
+        {
+            if (ifExists)
+                return schema;
+
+            throw ire("Index '%s.%s' doesn't exist'", keyspaceName, indexName);
+        }
+
+        TableMetadata newTable = table.withSwapped(table.indexes.without(indexName));
+        return schema.withAddedOrUpdated(keyspace.withSwapped(keyspace.tables.withSwapped(newTable)));
+    }
+
+    SchemaChange schemaChangeEvent(KeyspacesDiff diff)
+    {
+        assert diff.altered.size() == 1;
+        KeyspaceDiff ksDiff = diff.altered.get(0);
+
+        assert ksDiff.tables.altered.size() == 1;
+        Diff.Altered<TableMetadata> tableDiff = ksDiff.tables.altered.iterator().next();
+
+        return new SchemaChange(Change.UPDATED, Target.TABLE, keyspaceName, tableDiff.after.name);
+    }
+
+    public void authorize(ClientState client)
+    {
+        KeyspaceMetadata keyspace = Schema.instance.getKeyspaceMetadata(keyspaceName);
+        if (null == keyspace)
+            return;
+
+        keyspace.findIndexedTable(indexName)
+                .ifPresent(t -> client.ensureTablePermission(keyspaceName, t.name, Permission.ALTER));
+    }
+
+    @Override
+    public AuditLogContext getAuditLogContext()
+    {
+        return new AuditLogContext(AuditLogEntryType.DROP_INDEX, keyspaceName, indexName);
+    }
+
+    public static final class Raw extends CQLStatement.Raw
+    {
+        private final QualifiedName name;
+        private final boolean ifExists;
+
+        public Raw(QualifiedName name, boolean ifExists)
+        {
+            this.name = name;
+            this.ifExists = ifExists;
+        }
+
+        public DropIndexStatement prepare(ClientState state)
+        {
+            String keyspaceName = name.hasKeyspace() ? name.getKeyspace() : state.getKeyspace();
+            return new DropIndexStatement(keyspaceName, name.getName(), ifExists);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/cql3/statements/schema/DropKeyspaceStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/schema/DropKeyspaceStatement.java b/src/java/org/apache/cassandra/cql3/statements/schema/DropKeyspaceStatement.java
new file mode 100644
index 0000000..ae5cf06
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/statements/schema/DropKeyspaceStatement.java
@@ -0,0 +1,83 @@
+/*
+ * 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.statements.schema;
+
+import org.apache.cassandra.audit.AuditLogContext;
+import org.apache.cassandra.audit.AuditLogEntryType;
+import org.apache.cassandra.auth.Permission;
+import org.apache.cassandra.cql3.CQLStatement;
+import org.apache.cassandra.schema.Keyspaces;
+import org.apache.cassandra.schema.Keyspaces.KeyspacesDiff;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.transport.Event.SchemaChange;
+import org.apache.cassandra.transport.Event.SchemaChange.Change;
+
+public final class DropKeyspaceStatement extends AlterSchemaStatement
+{
+    private final boolean ifExists;
+
+    public DropKeyspaceStatement(String keyspaceName, boolean ifExists)
+    {
+        super(keyspaceName);
+        this.ifExists = ifExists;
+    }
+
+    public Keyspaces apply(Keyspaces schema)
+    {
+        if (schema.containsKeyspace(keyspaceName))
+            return schema.without(keyspaceName);
+
+        if (ifExists)
+            return schema;
+
+        throw ire("Keyspace '%s' doesn't exist", keyspaceName);
+    }
+
+    SchemaChange schemaChangeEvent(KeyspacesDiff diff)
+    {
+        return new SchemaChange(Change.DROPPED, keyspaceName);
+    }
+
+    public void authorize(ClientState client)
+    {
+        client.ensureKeyspacePermission(keyspaceName, Permission.DROP);
+    }
+
+    @Override
+    public AuditLogContext getAuditLogContext()
+    {
+        return new AuditLogContext(AuditLogEntryType.DROP_KEYSPACE, keyspaceName);
+    }
+
+    public static final class Raw extends CQLStatement.Raw
+    {
+        private final String keyspaceName;
+        private final boolean ifExists;
+
+        public Raw(String keyspaceName, boolean ifExists)
+        {
+            this.keyspaceName = keyspaceName;
+            this.ifExists = ifExists;
+        }
+
+        public DropKeyspaceStatement prepare(ClientState state)
+        {
+            return new DropKeyspaceStatement(keyspaceName, ifExists);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/cql3/statements/schema/DropTableStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/schema/DropTableStatement.java b/src/java/org/apache/cassandra/cql3/statements/schema/DropTableStatement.java
new file mode 100644
index 0000000..9be59af
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/statements/schema/DropTableStatement.java
@@ -0,0 +1,112 @@
+/*
+ * 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.statements.schema;
+
+import org.apache.cassandra.audit.AuditLogContext;
+import org.apache.cassandra.audit.AuditLogEntryType;
+import org.apache.cassandra.auth.Permission;
+import org.apache.cassandra.cql3.CQLStatement;
+import org.apache.cassandra.cql3.QualifiedName;
+import org.apache.cassandra.schema.*;
+import org.apache.cassandra.schema.Keyspaces.KeyspacesDiff;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.transport.Event.SchemaChange;
+import org.apache.cassandra.transport.Event.SchemaChange.Change;
+import org.apache.cassandra.transport.Event.SchemaChange.Target;
+
+import static java.lang.String.join;
+
+import static com.google.common.collect.Iterables.isEmpty;
+import static com.google.common.collect.Iterables.transform;
+
+public final class DropTableStatement extends AlterSchemaStatement
+{
+    private final String tableName;
+    private final boolean ifExists;
+
+    public DropTableStatement(String keyspaceName, String tableName, boolean ifExists)
+    {
+        super(keyspaceName);
+        this.tableName = tableName;
+        this.ifExists = ifExists;
+    }
+
+    public Keyspaces apply(Keyspaces schema)
+    {
+        KeyspaceMetadata keyspace = schema.getNullable(keyspaceName);
+
+        TableMetadata table = null == keyspace
+                            ? null
+                            : keyspace.getTableOrViewNullable(tableName);
+
+        if (null == table)
+        {
+            if (ifExists)
+                return schema;
+
+            throw ire("Table '%s.%s' doesn't exist", keyspaceName, tableName);
+        }
+
+        if (table.isView())
+            throw ire("Cannot use DROP TABLE on a materialized view. Please use DROP MATERIALIZED VIEW instead.");
+
+        Iterable<ViewMetadata> views = keyspace.views.forTable(table.id);
+        if (!isEmpty(views))
+        {
+            throw ire("Cannot drop a table when materialized views still depend on it (%s)",
+                      keyspaceName,
+                      join(", ", transform(views, ViewMetadata::name)));
+        }
+
+        return schema.withAddedOrUpdated(keyspace.withSwapped(keyspace.tables.without(table)));
+    }
+
+    SchemaChange schemaChangeEvent(KeyspacesDiff diff)
+    {
+        return new SchemaChange(Change.DROPPED, Target.TABLE, keyspaceName, tableName);
+    }
+
+    public void authorize(ClientState client)
+    {
+        client.ensureTablePermission(keyspaceName, tableName, Permission.DROP);
+    }
+
+    @Override
+    public AuditLogContext getAuditLogContext()
+    {
+        return new AuditLogContext(AuditLogEntryType.DROP_TABLE, keyspaceName, tableName);
+    }
+
+    public static final class Raw extends CQLStatement.Raw
+    {
+        private final QualifiedName name;
+        private final boolean ifExists;
+
+        public Raw(QualifiedName name, boolean ifExists)
+        {
+            this.name = name;
+            this.ifExists = ifExists;
+        }
+
+        public DropTableStatement prepare(ClientState state)
+        {
+            String keyspaceName = name.hasKeyspace() ? name.getKeyspace() : state.getKeyspace();
+            return new DropTableStatement(keyspaceName, name.getName(), ifExists);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/cql3/statements/schema/DropTriggerStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/schema/DropTriggerStatement.java b/src/java/org/apache/cassandra/cql3/statements/schema/DropTriggerStatement.java
new file mode 100644
index 0000000..8de47c2
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/statements/schema/DropTriggerStatement.java
@@ -0,0 +1,104 @@
+/*
+ * 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.statements.schema;
+
+import org.apache.cassandra.audit.AuditLogContext;
+import org.apache.cassandra.audit.AuditLogEntryType;
+import org.apache.cassandra.cql3.CQLStatement;
+import org.apache.cassandra.cql3.QualifiedName;
+import org.apache.cassandra.schema.*;
+import org.apache.cassandra.schema.Keyspaces.KeyspacesDiff;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.transport.Event.SchemaChange;
+import org.apache.cassandra.transport.Event.SchemaChange.Change;
+import org.apache.cassandra.transport.Event.SchemaChange.Target;
+
+public final class DropTriggerStatement extends AlterSchemaStatement
+{
+    private final String tableName;
+    private final String triggerName;
+    private final boolean ifExists;
+
+    public DropTriggerStatement(String keyspaceName, String tableName, String triggerName, boolean ifExists)
+    {
+        super(keyspaceName);
+        this.tableName = tableName;
+        this.triggerName = triggerName;
+        this.ifExists = ifExists;
+    }
+
+    public Keyspaces apply(Keyspaces schema)
+    {
+        KeyspaceMetadata keyspace = schema.getNullable(keyspaceName);
+
+        TableMetadata table = null == keyspace
+                            ? null
+                            : keyspace.tables.getNullable(tableName);
+
+        TriggerMetadata trigger = null == table
+                                ? null
+                                : table.triggers.get(triggerName).orElse(null);
+
+        if (null == trigger)
+        {
+            if (ifExists)
+                return schema;
+
+            throw ire("Trigger '%s' on '%s.%s' doesn't exist", triggerName, keyspaceName, tableName);
+        }
+
+        TableMetadata newTable = table.withSwapped(table.triggers.without(triggerName));
+        return schema.withAddedOrUpdated(keyspace.withSwapped(keyspace.tables.withSwapped(newTable)));
+    }
+
+    SchemaChange schemaChangeEvent(KeyspacesDiff diff)
+    {
+        return new SchemaChange(Change.UPDATED, Target.TABLE, keyspaceName, tableName);
+    }
+
+    public void authorize(ClientState client)
+    {
+        client.ensureIsSuperuser("Only superusers are allowed to perfrom DROP TRIGGER queries");
+    }
+
+    @Override
+    public AuditLogContext getAuditLogContext()
+    {
+        return new AuditLogContext(AuditLogEntryType.DROP_TRIGGER, keyspaceName, triggerName);
+    }
+
+    public static final class Raw extends CQLStatement.Raw
+    {
+        private final QualifiedName tableName;
+        private final String triggerName;
+        private final boolean ifExists;
+
+        public Raw(QualifiedName tableName, String triggerName, boolean ifExists)
+        {
+            this.tableName = tableName;
+            this.triggerName = triggerName;
+            this.ifExists = ifExists;
+        }
+
+        public DropTriggerStatement prepare(ClientState state)
+        {
+            String keyspaceName = tableName.hasKeyspace() ? tableName.getKeyspace() : state.getKeyspace();
+            return new DropTriggerStatement(keyspaceName, tableName.getName(), triggerName, ifExists);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/cql3/statements/schema/DropTypeStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/schema/DropTypeStatement.java b/src/java/org/apache/cassandra/cql3/statements/schema/DropTypeStatement.java
new file mode 100644
index 0000000..d51954c
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/statements/schema/DropTypeStatement.java
@@ -0,0 +1,149 @@
+/*
+ * 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.statements.schema;
+
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.audit.AuditLogContext;
+import org.apache.cassandra.audit.AuditLogEntryType;
+import org.apache.cassandra.auth.Permission;
+import org.apache.cassandra.cql3.CQLStatement;
+import org.apache.cassandra.cql3.UTName;
+import org.apache.cassandra.cql3.functions.Function;
+import org.apache.cassandra.db.marshal.UserType;
+import org.apache.cassandra.schema.KeyspaceMetadata;
+import org.apache.cassandra.schema.Keyspaces.KeyspacesDiff;
+import org.apache.cassandra.schema.Keyspaces;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.transport.Event.SchemaChange.Change;
+import org.apache.cassandra.transport.Event.SchemaChange.Target;
+import org.apache.cassandra.transport.Event.SchemaChange;
+
+import static java.lang.String.join;
+
+import static com.google.common.collect.Iterables.isEmpty;
+import static com.google.common.collect.Iterables.transform;
+
+import static org.apache.cassandra.utils.ByteBufferUtil.bytes;
+
+public final class DropTypeStatement extends AlterSchemaStatement
+{
+    private final String typeName;
+    private final boolean ifExists;
+
+    public DropTypeStatement(String keyspaceName, String typeName, boolean ifExists)
+    {
+        super(keyspaceName);
+        this.typeName = typeName;
+        this.ifExists = ifExists;
+    }
+
+    // TODO: expand types into tuples in all dropped columns of all tables
+    public Keyspaces apply(Keyspaces schema)
+    {
+        ByteBuffer name = bytes(typeName);
+
+        KeyspaceMetadata keyspace = schema.getNullable(keyspaceName);
+
+        UserType type = null == keyspace
+                      ? null
+                      : keyspace.types.getNullable(name);
+
+        if (null == type)
+        {
+            if (ifExists)
+                return schema;
+
+            throw ire("Type '%s.%s' doesn't exist", keyspaceName, typeName);
+        }
+
+        /*
+         * We don't want to drop a type unless it's not used anymore (mainly because
+         * if someone drops a type and recreates one with the same name but different
+         * definition with the previous name still in use, things can get messy).
+         * We have three places to check:
+         * 1) UDFs and UDAs using the type
+         * 2) other user type that can nest the one we drop and
+         * 3) existing tables referencing the type (maybe in a nested way).
+         */
+        Iterable<Function> functions = keyspace.functions.referencingUserType(name);
+        if (!isEmpty(functions))
+        {
+            throw ire("Cannot drop user type '%s.%s' as it is still used by functions %s",
+                      keyspaceName,
+                      typeName,
+                      join(", ", transform(functions, f -> f.name().toString())));
+        }
+
+        Iterable<UserType> types = keyspace.types.referencingUserType(name);
+        if (!isEmpty(types))
+        {
+            throw ire("Cannot drop user type '%s.%s' as it is still used by user types %s",
+                      keyspaceName,
+                      typeName,
+                      join(", ", transform(types, UserType::getNameAsString)));
+
+        }
+
+        Iterable<TableMetadata> tables = keyspace.tables.referencingUserType(name);
+        if (!isEmpty(tables))
+        {
+            throw ire("Cannot drop user type '%s.%s' as it is still used by tables %s",
+                      keyspaceName,
+                      typeName,
+                      join(", ", transform(tables, t -> t.name)));
+        }
+
+        return schema.withAddedOrUpdated(keyspace.withSwapped(keyspace.types.without(type)));
+    }
+
+    SchemaChange schemaChangeEvent(KeyspacesDiff diff)
+    {
+        return new SchemaChange(Change.DROPPED, Target.TYPE, keyspaceName, typeName);
+    }
+
+    public void authorize(ClientState client)
+    {
+        client.ensureKeyspacePermission(keyspaceName, Permission.DROP);
+    }
+
+    @Override
+    public AuditLogContext getAuditLogContext()
+    {
+        return new AuditLogContext(AuditLogEntryType.DROP_TYPE, keyspaceName, typeName);
+    }
+
+    public static final class Raw extends CQLStatement.Raw
+    {
+        private final UTName name;
+        private final boolean ifExists;
+
+        public Raw(UTName name, boolean ifExists)
+        {
+            this.name = name;
+            this.ifExists = ifExists;
+        }
+
+        public DropTypeStatement prepare(ClientState state)
+        {
+            String keyspaceName = name.hasKeyspace() ? name.getKeyspace() : state.getKeyspace();
+            return new DropTypeStatement(keyspaceName, name.getStringTypeName(), ifExists);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/cql3/statements/schema/DropViewStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/schema/DropViewStatement.java b/src/java/org/apache/cassandra/cql3/statements/schema/DropViewStatement.java
new file mode 100644
index 0000000..807d03d
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/statements/schema/DropViewStatement.java
@@ -0,0 +1,98 @@
+/*
+ * 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.statements.schema;
+
+import org.apache.cassandra.audit.AuditLogContext;
+import org.apache.cassandra.audit.AuditLogEntryType;
+import org.apache.cassandra.auth.Permission;
+import org.apache.cassandra.cql3.CQLStatement;
+import org.apache.cassandra.cql3.QualifiedName;
+import org.apache.cassandra.schema.*;
+import org.apache.cassandra.schema.Keyspaces.KeyspacesDiff;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.transport.Event.SchemaChange;
+import org.apache.cassandra.transport.Event.SchemaChange.Change;
+import org.apache.cassandra.transport.Event.SchemaChange.Target;
+
+public final class DropViewStatement extends AlterSchemaStatement
+{
+    private final String viewName;
+    private final boolean ifExists;
+
+    public DropViewStatement(String keyspaceName, String viewName, boolean ifExists)
+    {
+        super(keyspaceName);
+        this.viewName = viewName;
+        this.ifExists = ifExists;
+    }
+
+    public Keyspaces apply(Keyspaces schema)
+    {
+        KeyspaceMetadata keyspace = schema.getNullable(keyspaceName);
+
+        ViewMetadata view = null == keyspace
+                          ? null
+                          : keyspace.views.getNullable(viewName);
+
+        if (null == view)
+        {
+            if (ifExists)
+                return schema;
+
+            throw ire("Materialized view '%s.%s' doesn't exist", keyspaceName, viewName);
+        }
+
+        return schema.withAddedOrUpdated(keyspace.withSwapped(keyspace.views.without(viewName)));
+    }
+
+    SchemaChange schemaChangeEvent(KeyspacesDiff diff)
+    {
+        return new SchemaChange(Change.DROPPED, Target.TABLE, keyspaceName, viewName);
+    }
+
+    public void authorize(ClientState client)
+    {
+        ViewMetadata view = Schema.instance.getView(keyspaceName, viewName);
+        if (null != view)
+            client.ensureTablePermission(keyspaceName, view.baseTableName, Permission.ALTER);
+    }
+
+    @Override
+    public AuditLogContext getAuditLogContext()
+    {
+        return new AuditLogContext(AuditLogEntryType.DROP_VIEW, keyspaceName, viewName);
+    }
+
+    public static final class Raw extends CQLStatement.Raw
+    {
+        private final QualifiedName name;
+        private final boolean ifExists;
+
+        public Raw(QualifiedName name, boolean ifExists)
+        {
+            this.name = name;
+            this.ifExists = ifExists;
+        }
+
+        public DropViewStatement prepare(ClientState state)
+        {
+            String keyspaceName = name.hasKeyspace() ? name.getKeyspace() : state.getKeyspace();
+            return new DropViewStatement(keyspaceName, name.getName(), ifExists);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/cql3/statements/schema/IndexAttributes.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/schema/IndexAttributes.java b/src/java/org/apache/cassandra/cql3/statements/schema/IndexAttributes.java
new file mode 100644
index 0000000..f30c502
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/statements/schema/IndexAttributes.java
@@ -0,0 +1,77 @@
+/*
+ * 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.statements.schema;
+
+import java.util.*;
+
+import org.apache.cassandra.cql3.statements.PropertyDefinitions;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.exceptions.RequestValidationException;
+import org.apache.cassandra.exceptions.SyntaxException;
+
+public class IndexAttributes extends PropertyDefinitions
+{
+    private static final String KW_OPTIONS = "options";
+
+    private static final Set<String> keywords = new HashSet<>();
+    private static final Set<String> obsoleteKeywords = new HashSet<>();
+
+    public boolean isCustom;
+    public String customClass;
+
+    static
+    {
+        keywords.add(KW_OPTIONS);
+    }
+
+    public void validate() throws RequestValidationException
+    {
+        validate(keywords, obsoleteKeywords);
+
+        if (isCustom && customClass == null)
+            throw new InvalidRequestException("CUSTOM index requires specifiying the index class");
+
+        if (!isCustom && customClass != null)
+            throw new InvalidRequestException("Cannot specify index class for a non-CUSTOM index");
+
+        if (!isCustom && !properties.isEmpty())
+            throw new InvalidRequestException("Cannot specify options for a non-CUSTOM index");
+
+        if (getRawOptions().containsKey(IndexTarget.CUSTOM_INDEX_OPTION_NAME))
+            throw new InvalidRequestException(String.format("Cannot specify %s as a CUSTOM option",
+                                                            IndexTarget.CUSTOM_INDEX_OPTION_NAME));
+
+        if (getRawOptions().containsKey(IndexTarget.TARGET_OPTION_NAME))
+            throw new InvalidRequestException(String.format("Cannot specify %s as a CUSTOM option",
+                                                            IndexTarget.TARGET_OPTION_NAME));
+
+    }
+
+    private Map<String, String> getRawOptions() throws SyntaxException
+    {
+        Map<String, String> options = getMap(KW_OPTIONS);
+        return options == null ? Collections.emptyMap() : options;
+    }
+
+    public Map<String, String> getOptions() throws SyntaxException
+    {
+        Map<String, String> options = new HashMap<>(getRawOptions());
+        options.put(IndexTarget.CUSTOM_INDEX_OPTION_NAME, customClass);
+        return options;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/cql3/statements/schema/IndexTarget.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/schema/IndexTarget.java b/src/java/org/apache/cassandra/cql3/statements/schema/IndexTarget.java
new file mode 100644
index 0000000..dff933d
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/statements/schema/IndexTarget.java
@@ -0,0 +1,133 @@
+/*
+ * 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.statements.schema;
+
+import org.apache.cassandra.cql3.ColumnIdentifier;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.schema.TableMetadata;
+
+public class IndexTarget
+{
+    public static final String TARGET_OPTION_NAME = "target";
+    public static final String CUSTOM_INDEX_OPTION_NAME = "class_name";
+
+    public final ColumnIdentifier column;
+    public final Type type;
+
+    public IndexTarget(ColumnIdentifier column, Type type)
+    {
+        this.column = column;
+        this.type = type;
+    }
+
+    public String asCqlString()
+    {
+        return type == Type.SIMPLE
+             ? column.toCQLString()
+             : String.format("%s(%s)", type.toString(), column.toCQLString());
+    }
+
+    public static class Raw
+    {
+        private final ColumnMetadata.Raw column;
+        private final Type type;
+
+        private Raw(ColumnMetadata.Raw column, Type type)
+        {
+            this.column = column;
+            this.type = type;
+        }
+
+        public static Raw simpleIndexOn(ColumnMetadata.Raw c)
+        {
+            return new Raw(c, Type.SIMPLE);
+        }
+
+        public static Raw valuesOf(ColumnMetadata.Raw c)
+        {
+            return new Raw(c, Type.VALUES);
+        }
+
+        public static Raw keysOf(ColumnMetadata.Raw c)
+        {
+            return new Raw(c, Type.KEYS);
+        }
+
+        public static Raw keysAndValuesOf(ColumnMetadata.Raw c)
+        {
+            return new Raw(c, Type.KEYS_AND_VALUES);
+        }
+
+        public static Raw fullCollection(ColumnMetadata.Raw c)
+        {
+            return new Raw(c, Type.FULL);
+        }
+
+        public IndexTarget prepare(TableMetadata table)
+        {
+            // Until we've prepared the target column, we can't be certain about the target type
+            // because (for backwards compatibility) an index on a collection's values uses the
+            // same syntax as an index on a regular column (i.e. the 'values' in
+            // 'CREATE INDEX on table(values(collection));' is optional). So we correct the target type
+            // when the target column is a collection & the target type is SIMPLE.
+            ColumnMetadata columnDef = column.prepare(table);
+            Type actualType = (type == Type.SIMPLE && columnDef.type.isCollection()) ? Type.VALUES : type;
+            return new IndexTarget(columnDef.name, actualType);
+        }
+    }
+
+    public enum Type
+    {
+        VALUES, KEYS, KEYS_AND_VALUES, FULL, SIMPLE;
+
+        public String toString()
+        {
+            switch (this)
+            {
+                case KEYS: return "keys";
+                case KEYS_AND_VALUES: return "entries";
+                case FULL: return "full";
+                case VALUES: return "values";
+                case SIMPLE: return "";
+                default: return "";
+            }
+        }
+
+        public static Type fromString(String s)
+        {
+            if ("".equals(s))
+                return SIMPLE;
+            else if ("values".equals(s))
+                return VALUES;
+            else if ("keys".equals(s))
+                return KEYS;
+            else if ("entries".equals(s))
+                return KEYS_AND_VALUES;
+            else if ("full".equals(s))
+                return FULL;
+
+            throw new AssertionError("Unrecognized index target type " + s);
+        }
+    }
+    
+    @Override
+    public String toString()
+    {
+        return asCqlString();
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/cql3/statements/schema/KeyspaceAttributes.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/schema/KeyspaceAttributes.java b/src/java/org/apache/cassandra/cql3/statements/schema/KeyspaceAttributes.java
new file mode 100644
index 0000000..06c16a9
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/statements/schema/KeyspaceAttributes.java
@@ -0,0 +1,85 @@
+/*
+ * 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.statements.schema;
+
+import java.util.*;
+
+import com.google.common.collect.ImmutableSet;
+
+import org.apache.cassandra.cql3.statements.PropertyDefinitions;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.schema.KeyspaceParams;
+import org.apache.cassandra.schema.KeyspaceParams.Option;
+import org.apache.cassandra.schema.ReplicationParams;
+
+public final class KeyspaceAttributes extends PropertyDefinitions
+{
+    private static final Set<String> validKeywords;
+    private static final Set<String> obsoleteKeywords;
+
+    static
+    {
+        ImmutableSet.Builder<String> validBuilder = ImmutableSet.builder();
+        for (Option option : Option.values())
+            validBuilder.add(option.toString());
+        validKeywords = validBuilder.build();
+        obsoleteKeywords = ImmutableSet.of();
+    }
+
+    public void validate()
+    {
+        validate(validKeywords, obsoleteKeywords);
+
+        Map<String, String> replicationOptions = getAllReplicationOptions();
+        if (!replicationOptions.isEmpty() && !replicationOptions.containsKey(ReplicationParams.CLASS))
+            throw new ConfigurationException("Missing replication strategy class");
+    }
+
+    private String getReplicationStrategyClass()
+    {
+        return getAllReplicationOptions().get(ReplicationParams.CLASS);
+    }
+
+    private Map<String, String> getAllReplicationOptions()
+    {
+        Map<String, String> replication = getMap(Option.REPLICATION.toString());
+        return replication == null
+             ? Collections.emptyMap()
+             : replication;
+    }
+
+    KeyspaceParams asNewKeyspaceParams()
+    {
+        boolean durableWrites = getBoolean(Option.DURABLE_WRITES.toString(), KeyspaceParams.DEFAULT_DURABLE_WRITES);
+        return KeyspaceParams.create(durableWrites, getAllReplicationOptions());
+    }
+
+    KeyspaceParams asAlteredKeyspaceParams(KeyspaceParams previous)
+    {
+        boolean durableWrites = getBoolean(Option.DURABLE_WRITES.toString(), previous.durableWrites);
+        ReplicationParams replication = getReplicationStrategyClass() == null
+                                      ? previous.replication
+                                      : ReplicationParams.fromMap(getAllReplicationOptions());
+        return new KeyspaceParams(durableWrites, replication);
+    }
+
+    public boolean hasOption(Option option)
+    {
+        return hasProperty(option.toString());
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/cql3/statements/schema/TableAttributes.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/schema/TableAttributes.java b/src/java/org/apache/cassandra/cql3/statements/schema/TableAttributes.java
new file mode 100644
index 0000000..8cc2685
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/statements/schema/TableAttributes.java
@@ -0,0 +1,196 @@
+/*
+ * 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.statements.schema;
+
+import java.util.Map;
+import java.util.Set;
+
+import com.google.common.collect.ImmutableSet;
+
+import org.apache.cassandra.cql3.statements.PropertyDefinitions;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.exceptions.SyntaxException;
+import org.apache.cassandra.schema.*;
+import org.apache.cassandra.schema.TableParams.Option;
+import org.apache.cassandra.service.reads.SpeculativeRetryPolicy;
+
+import static java.lang.String.format;
+
+public final class TableAttributes extends PropertyDefinitions
+{
+    public static final String ID = "id";
+    private static final Set<String> validKeywords;
+    private static final Set<String> obsoleteKeywords;
+
+    static
+    {
+        ImmutableSet.Builder<String> validBuilder = ImmutableSet.builder();
+        for (Option option : Option.values())
+            validBuilder.add(option.toString());
+        validBuilder.add(ID);
+        validKeywords = validBuilder.build();
+        obsoleteKeywords = ImmutableSet.of();
+    }
+
+    public void validate()
+    {
+        validate(validKeywords, obsoleteKeywords);
+        build(TableParams.builder()).validate();
+    }
+
+    TableParams asNewTableParams()
+    {
+        return build(TableParams.builder());
+    }
+
+    TableParams asAlteredTableParams(TableParams previous)
+    {
+        if (getId() != null)
+            throw new ConfigurationException("Cannot alter table id.");
+        return build(previous.unbuild());
+    }
+
+    public TableId getId() throws ConfigurationException
+    {
+        String id = getSimple(ID);
+        try
+        {
+            return id != null ? TableId.fromString(id) : null;
+        }
+        catch (IllegalArgumentException e)
+        {
+            throw new ConfigurationException("Invalid table id", e);
+        }
+    }
+
+    private TableParams build(TableParams.Builder builder)
+    {
+        if (hasOption(Option.BLOOM_FILTER_FP_CHANCE))
+            builder.bloomFilterFpChance(getDouble(Option.BLOOM_FILTER_FP_CHANCE));
+
+        if (hasOption(Option.CACHING))
+            builder.caching(CachingParams.fromMap(getMap(Option.CACHING)));
+
+        if (hasOption(Option.COMMENT))
+            builder.comment(getString(Option.COMMENT));
+
+        if (hasOption(Option.COMPACTION))
+            builder.compaction(CompactionParams.fromMap(getMap(Option.COMPACTION)));
+
+        if (hasOption(Option.COMPRESSION))
+        {
+            //crc_check_chance was "promoted" from a compression property to a top-level-property after #9839
+            //so we temporarily accept it to be defined as a compression option, to maintain backwards compatibility
+            Map<String, String> compressionOpts = getMap(Option.COMPRESSION);
+            if (compressionOpts.containsKey(Option.CRC_CHECK_CHANCE.toString().toLowerCase()))
+            {
+                Double crcCheckChance = getDeprecatedCrcCheckChance(compressionOpts);
+                builder.crcCheckChance(crcCheckChance);
+            }
+            builder.compression(CompressionParams.fromMap(getMap(Option.COMPRESSION)));
+        }
+
+        if (hasOption(Option.DEFAULT_TIME_TO_LIVE))
+            builder.defaultTimeToLive(getInt(Option.DEFAULT_TIME_TO_LIVE));
+
+        if (hasOption(Option.GC_GRACE_SECONDS))
+            builder.gcGraceSeconds(getInt(Option.GC_GRACE_SECONDS));
+
+        if (hasOption(Option.MAX_INDEX_INTERVAL))
+            builder.maxIndexInterval(getInt(Option.MAX_INDEX_INTERVAL));
+
+        if (hasOption(Option.MEMTABLE_FLUSH_PERIOD_IN_MS))
+            builder.memtableFlushPeriodInMs(getInt(Option.MEMTABLE_FLUSH_PERIOD_IN_MS));
+
+        if (hasOption(Option.MIN_INDEX_INTERVAL))
+            builder.minIndexInterval(getInt(Option.MIN_INDEX_INTERVAL));
+
+        if (hasOption(Option.SPECULATIVE_RETRY))
+            builder.speculativeRetry(SpeculativeRetryPolicy.fromString(getString(Option.SPECULATIVE_RETRY)));
+
+        if (hasOption(Option.CRC_CHECK_CHANCE))
+            builder.crcCheckChance(getDouble(Option.CRC_CHECK_CHANCE));
+
+        if (hasOption(Option.CDC))
+            builder.cdc(getBoolean(Option.CDC.toString(), false));
+
+        return builder.build();
+    }
+
+    private Double getDeprecatedCrcCheckChance(Map<String, String> compressionOpts)
+    {
+        String value = compressionOpts.get(Option.CRC_CHECK_CHANCE.toString().toLowerCase());
+        try
+        {
+            return Double.valueOf(value);
+        }
+        catch (NumberFormatException e)
+        {
+            throw new SyntaxException(String.format("Invalid double value %s for crc_check_chance.'", value));
+        }
+    }
+
+    private double getDouble(Option option)
+    {
+        String value = getString(option);
+
+        try
+        {
+            return Double.parseDouble(value);
+        }
+        catch (NumberFormatException e)
+        {
+            throw new SyntaxException(format("Invalid double value %s for '%s'", value, option));
+        }
+    }
+
+    private int getInt(Option option)
+    {
+        String value = getString(option);
+
+        try
+        {
+            return Integer.parseInt(value);
+        }
+        catch (NumberFormatException e)
+        {
+            throw new SyntaxException(String.format("Invalid integer value %s for '%s'", value, option));
+        }
+    }
+
+    private String getString(Option option)
+    {
+        String value = getSimple(option.toString());
+        if (value == null)
+            throw new IllegalStateException(format("Option '%s' is absent", option));
+        return value;
+    }
+
+    private Map<String, String> getMap(Option option)
+    {
+        Map<String, String> value = getMap(option.toString());
+        if (value == null)
+            throw new IllegalStateException(format("Option '%s' is absent", option));
+        return value;
+    }
+
+    public boolean hasOption(Option option)
+    {
+        return hasProperty(option.toString());
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/db/Keyspace.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/Keyspace.java b/src/java/org/apache/cassandra/db/Keyspace.java
index cb62c14..33c0f32 100644
--- a/src/java/org/apache/cassandra/db/Keyspace.java
+++ b/src/java/org/apache/cassandra/db/Keyspace.java
@@ -52,7 +52,6 @@ import org.apache.cassandra.schema.SchemaConstants;
 import org.apache.cassandra.schema.TableId;
 import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.schema.TableMetadataRef;
-import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.tracing.Tracing;
 import org.apache.cassandra.utils.*;
 import org.apache.cassandra.utils.concurrent.OpOrder;
@@ -366,12 +365,8 @@ public class Keyspace
 
     private void createReplicationStrategy(KeyspaceMetadata ksm)
     {
-        logger.info("Creating replication strategy " + ksm .name + " params " + ksm.params);
-        replicationStrategy = AbstractReplicationStrategy.createReplicationStrategy(ksm.name,
-                                                                                    ksm.params.replication.klass,
-                                                                                    StorageService.instance.getTokenMetadata(),
-                                                                                    DatabaseDescriptor.getEndpointSnitch(),
-                                                                                    ksm.params.replication.options);
+        logger.info("Creating replication strategy " + ksm.name + " params " + ksm.params);
+        replicationStrategy = ksm.createReplicationStrategy();
         if (!ksm.params.replication.equals(replicationParams))
         {
             logger.debug("New replication settings for keyspace {} - invalidating disk boundary caches", ksm.name);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/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 b4de801..fb9e889 100644
--- a/src/java/org/apache/cassandra/db/SystemKeyspace.java
+++ b/src/java/org/apache/cassandra/db/SystemKeyspace.java
@@ -43,7 +43,7 @@ import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.cql3.QueryProcessor;
 import org.apache.cassandra.cql3.UntypedResultSet;
 import org.apache.cassandra.cql3.functions.*;
-import org.apache.cassandra.cql3.statements.CreateTableStatement;
+import org.apache.cassandra.cql3.statements.schema.CreateTableStatement;
 import org.apache.cassandra.db.commitlog.CommitLogPosition;
 import org.apache.cassandra.db.compaction.CompactionHistoryTabularData;
 import org.apache.cassandra.db.marshal.*;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/db/TableCQLHelper.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/TableCQLHelper.java b/src/java/org/apache/cassandra/db/TableCQLHelper.java
index 0e9977d..550a6d6 100644
--- a/src/java/org/apache/cassandra/db/TableCQLHelper.java
+++ b/src/java/org/apache/cassandra/db/TableCQLHelper.java
@@ -26,7 +26,7 @@ import java.util.function.*;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.Iterables;
 
-import org.apache.cassandra.cql3.statements.*;
+import org.apache.cassandra.cql3.statements.schema.IndexTarget;
 import org.apache.cassandra.db.marshal.*;
 import org.apache.cassandra.schema.*;
 import org.apache.cassandra.utils.*;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/db/marshal/AbstractCompositeType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/marshal/AbstractCompositeType.java b/src/java/org/apache/cassandra/db/marshal/AbstractCompositeType.java
index 9eb5d82..0248629 100644
--- a/src/java/org/apache/cassandra/db/marshal/AbstractCompositeType.java
+++ b/src/java/org/apache/cassandra/db/marshal/AbstractCompositeType.java
@@ -297,12 +297,6 @@ public abstract class AbstractCompositeType extends AbstractType<ByteBuffer>
         return BytesSerializer.instance;
     }
 
-    @Override
-    public boolean referencesUserType(String name)
-    {
-        return getComponents().stream().anyMatch(f -> f.referencesUserType(name));
-    }
-
     /**
      * @return the comparator for the given component. static CompositeType will consult
      * @param i DynamicCompositeType will read the type information from @param bb

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/db/marshal/AbstractType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/marshal/AbstractType.java b/src/java/org/apache/cassandra/db/marshal/AbstractType.java
index 7a5cc13..f305313 100644
--- a/src/java/org/apache/cassandra/db/marshal/AbstractType.java
+++ b/src/java/org/apache/cassandra/db/marshal/AbstractType.java
@@ -456,11 +456,31 @@ public abstract class AbstractType<T> implements Comparator<ByteBuffer>, Assignm
             ByteBufferUtil.skipWithVIntLength(in);
     }
 
-    public boolean referencesUserType(String userTypeName)
+    public boolean referencesUserType(ByteBuffer name)
     {
         return false;
     }
 
+    /**
+     * Returns an instance of this type with all references to the provided user type recursively replaced with its new
+     * definition.
+     */
+    public AbstractType<?> withUpdatedUserType(UserType udt)
+    {
+        return this;
+    }
+
+    /**
+     * Replace any instances of UserType with equivalent TupleType-s.
+     *
+     * We need it for dropped_columns, to allow safely dropping unused user types later without retaining any references
+     * to them in system_schema.dropped_columns.
+     */
+    public AbstractType<?> expandUserTypes()
+    {
+        return this;
+    }
+
     public boolean referencesDuration()
     {
         return false;
@@ -468,9 +488,6 @@ public abstract class AbstractType<T> implements Comparator<ByteBuffer>, Assignm
 
     /**
      * Tests whether a CQL value having this type can be assigned to the provided receiver.
-     *
-     * @param keyspace the keyspace from which the receiver is.
-     * @param receiver the receiver for which we want to test type compatibility with.
      */
     public AssignmentTestable.TestResult testAssignment(AbstractType<?> receiverType)
     {
@@ -505,17 +522,6 @@ public abstract class AbstractType<T> implements Comparator<ByteBuffer>, Assignm
         return getClass().getName();
     }
 
-    /**
-     * Checks to see if two types are equal when ignoring or not ignoring differences in being frozen, depending on
-     * the value of the ignoreFreezing parameter.
-     * @param other type to compare
-     * @param ignoreFreezing if true, differences in the types being frozen will be ignored
-     */
-    public boolean equals(Object other, boolean ignoreFreezing)
-    {
-        return this.equals(other);
-    }
-
     public void checkComparable()
     {
         switch (comparisonType)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/db/marshal/CollectionType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/marshal/CollectionType.java b/src/java/org/apache/cassandra/db/marshal/CollectionType.java
index e83a041..b198e0c 100644
--- a/src/java/org/apache/cassandra/db/marshal/CollectionType.java
+++ b/src/java/org/apache/cassandra/db/marshal/CollectionType.java
@@ -211,7 +211,7 @@ public abstract class CollectionType<T> extends AbstractType<T>
     }
 
     @Override
-    public boolean equals(Object o, boolean ignoreFreezing)
+    public boolean equals(Object o)
     {
         if (this == o)
             return true;
@@ -224,11 +224,10 @@ public abstract class CollectionType<T> extends AbstractType<T>
         if (kind != other.kind)
             return false;
 
-        if (!ignoreFreezing && isMultiCell() != other.isMultiCell())
+        if (isMultiCell() != other.isMultiCell())
             return false;
 
-        return nameComparator().equals(other.nameComparator(), ignoreFreezing) &&
-               valueComparator().equals(other.valueComparator(), ignoreFreezing);
+        return nameComparator().equals(other.nameComparator()) && valueComparator().equals(other.valueComparator());
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/db/marshal/CompositeType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/marshal/CompositeType.java b/src/java/org/apache/cassandra/db/marshal/CompositeType.java
index 12e7fc3..ac4c69f 100644
--- a/src/java/org/apache/cassandra/db/marshal/CompositeType.java
+++ b/src/java/org/apache/cassandra/db/marshal/CompositeType.java
@@ -32,6 +32,9 @@ import org.apache.cassandra.exceptions.SyntaxException;
 import org.apache.cassandra.serializers.MarshalException;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
+import static com.google.common.collect.Iterables.any;
+import static com.google.common.collect.Iterables.transform;
+
 /*
  * The encoding of a CompositeType column name should be:
  *   <component><component><component> ...
@@ -102,18 +105,10 @@ public class CompositeType extends AbstractCompositeType
     public static CompositeType getInstance(List<AbstractType<?>> types)
     {
         assert types != null && !types.isEmpty();
-
-        CompositeType ct = instances.get(types);
-        if (ct == null)
-        {
-            ct = new CompositeType(types);
-            CompositeType previous = instances.putIfAbsent(types, ct);
-            if (previous != null)
-            {
-                ct = previous;
-            }
-        }
-        return ct;
+        CompositeType t = instances.get(types);
+        return null == t
+             ? instances.computeIfAbsent(types, CompositeType::new)
+             : t;
     }
 
     protected CompositeType(List<AbstractType<?>> types)
@@ -287,6 +282,29 @@ public class CompositeType extends AbstractCompositeType
         return true;
     }
 
+    @Override
+    public boolean referencesUserType(ByteBuffer name)
+    {
+        return any(types, t -> t.referencesUserType(name));
+    }
+
+    @Override
+    public CompositeType withUpdatedUserType(UserType udt)
+    {
+        if (!referencesUserType(udt.name))
+            return this;
+
+        instances.remove(types);
+
+        return getInstance(transform(types, t -> t.withUpdatedUserType(udt)));
+    }
+
+    @Override
+    public AbstractType<?> expandUserTypes()
+    {
+        return getInstance(transform(types, AbstractType::expandUserTypes));
+    }
+
     private static class StaticParsedComparator implements ParsedComparator
     {
         final AbstractType<?> type;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/db/marshal/DynamicCompositeType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/marshal/DynamicCompositeType.java b/src/java/org/apache/cassandra/db/marshal/DynamicCompositeType.java
index 6fa7e87..0458dc8 100644
--- a/src/java/org/apache/cassandra/db/marshal/DynamicCompositeType.java
+++ b/src/java/org/apache/cassandra/db/marshal/DynamicCompositeType.java
@@ -17,24 +17,25 @@
  */
 package org.apache.cassandra.db.marshal;
 
-import java.nio.charset.CharacterCodingException;
 import java.nio.ByteBuffer;
-import java.util.HashMap;
+import java.nio.charset.CharacterCodingException;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
 
-import org.apache.cassandra.cql3.Term;
+import com.google.common.collect.Maps;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.cassandra.cql3.Term;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.exceptions.SyntaxException;
-import org.apache.cassandra.serializers.TypeSerializer;
 import org.apache.cassandra.serializers.MarshalException;
+import org.apache.cassandra.serializers.TypeSerializer;
 import org.apache.cassandra.transport.ProtocolVersion;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
+import static com.google.common.collect.Iterables.any;
+
 /*
  * The encoding of a DynamicCompositeType column name should be:
  *   <component><component><component> ...
@@ -61,9 +62,9 @@ public class DynamicCompositeType extends AbstractCompositeType
     private final Map<Byte, AbstractType<?>> aliases;
 
     // interning instances
-    private static final ConcurrentMap<Map<Byte, AbstractType<?>>, DynamicCompositeType> instances = new ConcurrentHashMap<Map<Byte, AbstractType<?>>, DynamicCompositeType>();
+    private static final ConcurrentHashMap<Map<Byte, AbstractType<?>>, DynamicCompositeType> instances = new ConcurrentHashMap<>();
 
-    public static synchronized DynamicCompositeType getInstance(TypeParser parser) throws ConfigurationException, SyntaxException
+    public static DynamicCompositeType getInstance(TypeParser parser)
     {
         return getInstance(parser.getAliasParameters());
     }
@@ -71,9 +72,9 @@ public class DynamicCompositeType extends AbstractCompositeType
     public static DynamicCompositeType getInstance(Map<Byte, AbstractType<?>> aliases)
     {
         DynamicCompositeType dct = instances.get(aliases);
-        if (dct == null)
-            dct = instances.computeIfAbsent(aliases, k ->  new DynamicCompositeType(k));
-        return dct;
+        return null == dct
+             ? instances.computeIfAbsent(aliases, DynamicCompositeType::new)
+             : dct;
     }
 
     private DynamicCompositeType(Map<Byte, AbstractType<?>> aliases)
@@ -255,6 +256,29 @@ public class DynamicCompositeType extends AbstractCompositeType
         return true;
     }
 
+    @Override
+    public boolean referencesUserType(ByteBuffer name)
+    {
+        return any(aliases.values(), t -> t.referencesUserType(name));
+    }
+
+    @Override
+    public DynamicCompositeType withUpdatedUserType(UserType udt)
+    {
+        if (!referencesUserType(udt.name))
+            return this;
+
+        instances.remove(aliases);
+
+        return getInstance(Maps.transformValues(aliases, v -> v.withUpdatedUserType(udt)));
+    }
+
+    @Override
+    public AbstractType<?> expandUserTypes()
+    {
+        return getInstance(Maps.transformValues(aliases, v -> v.expandUserTypes()));
+    }
+
     private class DynamicParsedComparator implements ParsedComparator
     {
         final AbstractType<?> type;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/db/marshal/ListType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/marshal/ListType.java b/src/java/org/apache/cassandra/db/marshal/ListType.java
index 31b5a11..3dbf4a3 100644
--- a/src/java/org/apache/cassandra/db/marshal/ListType.java
+++ b/src/java/org/apache/cassandra/db/marshal/ListType.java
@@ -20,7 +20,6 @@ package org.apache.cassandra.db.marshal;
 import java.nio.ByteBuffer;
 import java.util.*;
 import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
 
 import org.apache.cassandra.cql3.Json;
 import org.apache.cassandra.cql3.Lists;
@@ -29,20 +28,15 @@ import org.apache.cassandra.db.rows.Cell;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.exceptions.SyntaxException;
 import org.apache.cassandra.serializers.CollectionSerializer;
-import org.apache.cassandra.serializers.MarshalException;
 import org.apache.cassandra.serializers.ListSerializer;
+import org.apache.cassandra.serializers.MarshalException;
 import org.apache.cassandra.transport.ProtocolVersion;
 
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
 public class ListType<T> extends CollectionType<List<T>>
 {
-    private static final Logger logger = LoggerFactory.getLogger(ListType.class);
-
     // interning instances
-    private static final ConcurrentMap<AbstractType<?>, ListType> instances = new ConcurrentHashMap<>();
-    private static final ConcurrentMap<AbstractType<?>, ListType> frozenInstances = new ConcurrentHashMap<>();
+    private static final ConcurrentHashMap<AbstractType<?>, ListType> instances = new ConcurrentHashMap<>();
+    private static final ConcurrentHashMap<AbstractType<?>, ListType> frozenInstances = new ConcurrentHashMap<>();
 
     private final AbstractType<T> elements;
     public final ListSerializer<T> serializer;
@@ -57,13 +51,13 @@ public class ListType<T> extends CollectionType<List<T>>
         return getInstance(l.get(0), true);
     }
 
-    public static <T> ListType<T> getInstance(AbstractType<T> elements, final boolean isMultiCell)
+    public static <T> ListType<T> getInstance(AbstractType<T> elements, boolean isMultiCell)
     {
-        ConcurrentMap<AbstractType<?>, ListType> internMap = isMultiCell ? instances : frozenInstances;
+        ConcurrentHashMap<AbstractType<?>, ListType> internMap = isMultiCell ? instances : frozenInstances;
         ListType<T> t = internMap.get(elements);
-        if (t == null)
-            t = internMap.computeIfAbsent(elements, k -> new ListType<>(k, isMultiCell) );
-        return t;
+        return null == t
+             ? internMap.computeIfAbsent(elements, k -> new ListType<>(k, isMultiCell))
+             : t;
     }
 
     private ListType(AbstractType<T> elements, boolean isMultiCell)
@@ -75,9 +69,26 @@ public class ListType<T> extends CollectionType<List<T>>
     }
 
     @Override
-    public boolean referencesUserType(String userTypeName)
+    public boolean referencesUserType(ByteBuffer name)
+    {
+        return elements.referencesUserType(name);
+    }
+
+    @Override
+    public ListType<?> withUpdatedUserType(UserType udt)
+    {
+        if (!referencesUserType(udt.name))
+            return this;
+
+        (isMultiCell ? instances : frozenInstances).remove(elements);
+
+        return getInstance(elements.withUpdatedUserType(udt), isMultiCell);
+    }
+
+    @Override
+    public AbstractType<?> expandUserTypes()
     {
-        return getElementsType().referencesUserType(userTypeName);
+        return getInstance(elements.expandUserTypes(), isMultiCell);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/db/marshal/MapType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/marshal/MapType.java b/src/java/org/apache/cassandra/db/marshal/MapType.java
index e333493..bab25d3 100644
--- a/src/java/org/apache/cassandra/db/marshal/MapType.java
+++ b/src/java/org/apache/cassandra/db/marshal/MapType.java
@@ -20,7 +20,6 @@ package org.apache.cassandra.db.marshal;
 import java.nio.ByteBuffer;
 import java.util.*;
 import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
 
 import org.apache.cassandra.cql3.Json;
 import org.apache.cassandra.cql3.Maps;
@@ -37,8 +36,8 @@ import org.apache.cassandra.utils.Pair;
 public class MapType<K, V> extends CollectionType<Map<K, V>>
 {
     // interning instances
-    private static final ConcurrentMap<Pair<AbstractType<?>, AbstractType<?>>, MapType> instances = new ConcurrentHashMap<>();
-    private static final ConcurrentMap<Pair<AbstractType<?>, AbstractType<?>>, MapType> frozenInstances = new ConcurrentHashMap<>();
+    private static final ConcurrentHashMap<Pair<AbstractType<?>, AbstractType<?>>, MapType> instances = new ConcurrentHashMap<>();
+    private static final ConcurrentHashMap<Pair<AbstractType<?>, AbstractType<?>>, MapType> frozenInstances = new ConcurrentHashMap<>();
 
     private final AbstractType<K> keys;
     private final AbstractType<V> values;
@@ -56,12 +55,12 @@ public class MapType<K, V> extends CollectionType<Map<K, V>>
 
     public static <K, V> MapType<K, V> getInstance(AbstractType<K> keys, AbstractType<V> values, boolean isMultiCell)
     {
-        ConcurrentMap<Pair<AbstractType<?>, AbstractType<?>>, MapType> internMap = isMultiCell ? instances : frozenInstances;
-        Pair<AbstractType<?>, AbstractType<?>> p = Pair.<AbstractType<?>, AbstractType<?>>create(keys, values);
+        ConcurrentHashMap<Pair<AbstractType<?>, AbstractType<?>>, MapType> internMap = isMultiCell ? instances : frozenInstances;
+        Pair<AbstractType<?>, AbstractType<?>> p = Pair.create(keys, values);
         MapType<K, V> t = internMap.get(p);
-        if (t == null)
-            t = internMap.computeIfAbsent(p, k -> new MapType<>(k.left, k.right, isMultiCell) );
-        return t;
+        return null == t
+             ? internMap.computeIfAbsent(p, k -> new MapType<>(k.left, k.right, isMultiCell))
+             : t;
     }
 
     private MapType(AbstractType<K> keys, AbstractType<V> values, boolean isMultiCell)
@@ -74,10 +73,26 @@ public class MapType<K, V> extends CollectionType<Map<K, V>>
     }
 
     @Override
-    public boolean referencesUserType(String userTypeName)
+    public boolean referencesUserType(ByteBuffer name)
     {
-        return getKeysType().referencesUserType(userTypeName) ||
-               getValuesType().referencesUserType(userTypeName);
+        return keys.referencesUserType(name) || values.referencesUserType(name);
+    }
+
+    @Override
+    public MapType<?,?> withUpdatedUserType(UserType udt)
+    {
+        if (!referencesUserType(udt.name))
+            return this;
+
+        (isMultiCell ? instances : frozenInstances).remove(Pair.create(keys, values));
+
+        return getInstance(keys.withUpdatedUserType(udt), values.withUpdatedUserType(udt), isMultiCell);
+    }
+
+    @Override
+    public AbstractType<?> expandUserTypes()
+    {
+        return getInstance(keys.expandUserTypes(), values.expandUserTypes(), isMultiCell);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/db/marshal/ReversedType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/marshal/ReversedType.java b/src/java/org/apache/cassandra/db/marshal/ReversedType.java
index 250dfdc..63a900a 100644
--- a/src/java/org/apache/cassandra/db/marshal/ReversedType.java
+++ b/src/java/org/apache/cassandra/db/marshal/ReversedType.java
@@ -18,14 +18,13 @@
 package org.apache.cassandra.db.marshal;
 
 import java.nio.ByteBuffer;
-import java.util.HashMap;
 import java.util.Map;
 import java.util.List;
+import java.util.concurrent.ConcurrentHashMap;
 
 import org.apache.cassandra.cql3.CQL3Type;
 import org.apache.cassandra.cql3.Term;
 import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.exceptions.SyntaxException;
 import org.apache.cassandra.serializers.MarshalException;
 import org.apache.cassandra.serializers.TypeSerializer;
 import org.apache.cassandra.transport.ProtocolVersion;
@@ -33,11 +32,11 @@ import org.apache.cassandra.transport.ProtocolVersion;
 public class ReversedType<T> extends AbstractType<T>
 {
     // interning instances
-    private static final Map<AbstractType<?>, ReversedType> instances = new HashMap<AbstractType<?>, ReversedType>();
+    private static final Map<AbstractType<?>, ReversedType> instances = new ConcurrentHashMap<>();
 
     public final AbstractType<T> baseType;
 
-    public static <T> ReversedType<T> getInstance(TypeParser parser) throws ConfigurationException, SyntaxException
+    public static <T> ReversedType<T> getInstance(TypeParser parser)
     {
         List<AbstractType<?>> types = parser.getTypeParameters();
         if (types.size() != 1)
@@ -45,15 +44,12 @@ public class ReversedType<T> extends AbstractType<T>
         return getInstance((AbstractType<T>) types.get(0));
     }
 
-    public static synchronized <T> ReversedType<T> getInstance(AbstractType<T> baseType)
+    public static <T> ReversedType<T> getInstance(AbstractType<T> baseType)
     {
-        ReversedType<T> type = instances.get(baseType);
-        if (type == null)
-        {
-            type = new ReversedType<T>(baseType);
-            instances.put(baseType, type);
-        }
-        return type;
+        ReversedType<T> t = instances.get(baseType);
+        return null == t
+             ? instances.computeIfAbsent(baseType, ReversedType::new)
+             : t;
     }
 
     private ReversedType(AbstractType<T> baseType)
@@ -126,9 +122,27 @@ public class ReversedType<T> extends AbstractType<T>
         return baseType.getSerializer();
     }
 
-    public boolean referencesUserType(String userTypeName)
+    @Override
+    public boolean referencesUserType(ByteBuffer name)
+    {
+        return baseType.referencesUserType(name);
+    }
+
+    @Override
+    public AbstractType<?> expandUserTypes()
     {
-        return baseType.referencesUserType(userTypeName);
+        return getInstance(baseType.expandUserTypes());
+    }
+
+    @Override
+    public ReversedType<?> withUpdatedUserType(UserType udt)
+    {
+        if (!referencesUserType(udt.name))
+            return this;
+
+        instances.remove(baseType);
+
+        return getInstance(baseType.withUpdatedUserType(udt));
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/db/marshal/SetType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/marshal/SetType.java b/src/java/org/apache/cassandra/db/marshal/SetType.java
index 4374612..ae9e0c0 100644
--- a/src/java/org/apache/cassandra/db/marshal/SetType.java
+++ b/src/java/org/apache/cassandra/db/marshal/SetType.java
@@ -20,7 +20,6 @@ package org.apache.cassandra.db.marshal;
 import java.nio.ByteBuffer;
 import java.util.*;
 import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
 
 import org.apache.cassandra.cql3.Json;
 import org.apache.cassandra.cql3.Sets;
@@ -35,8 +34,8 @@ import org.apache.cassandra.transport.ProtocolVersion;
 public class SetType<T> extends CollectionType<Set<T>>
 {
     // interning instances
-    private static final ConcurrentMap<AbstractType<?>, SetType> instances = new ConcurrentHashMap<>();
-    private static final ConcurrentMap<AbstractType<?>, SetType> frozenInstances = new ConcurrentHashMap<>();
+    private static final ConcurrentHashMap<AbstractType<?>, SetType> instances = new ConcurrentHashMap<>();
+    private static final ConcurrentHashMap<AbstractType<?>, SetType> frozenInstances = new ConcurrentHashMap<>();
 
     private final AbstractType<T> elements;
     private final SetSerializer<T> serializer;
@@ -53,11 +52,11 @@ public class SetType<T> extends CollectionType<Set<T>>
 
     public static <T> SetType<T> getInstance(AbstractType<T> elements, boolean isMultiCell)
     {
-        ConcurrentMap<AbstractType<?>, SetType> internMap = isMultiCell ? instances : frozenInstances;
+        ConcurrentHashMap<AbstractType<?>, SetType> internMap = isMultiCell ? instances : frozenInstances;
         SetType<T> t = internMap.get(elements);
-        if (t == null)
-            t = internMap.computeIfAbsent(elements, k -> new SetType<>(k, isMultiCell) );
-        return t;
+        return null == t
+             ? internMap.computeIfAbsent(elements, k -> new SetType<>(k, isMultiCell))
+             : t;
     }
 
     public SetType(AbstractType<T> elements, boolean isMultiCell)
@@ -69,9 +68,26 @@ public class SetType<T> extends CollectionType<Set<T>>
     }
 
     @Override
-    public boolean referencesUserType(String userTypeName)
+    public boolean referencesUserType(ByteBuffer name)
     {
-        return getElementsType().referencesUserType(userTypeName);
+        return elements.referencesUserType(name);
+    }
+
+    @Override
+    public SetType<?> withUpdatedUserType(UserType udt)
+    {
+        if (!referencesUserType(udt.name))
+            return this;
+
+        (isMultiCell ? instances : frozenInstances).remove(elements);
+
+        return getInstance(elements.withUpdatedUserType(udt), isMultiCell);
+    }
+
+    @Override
+    public AbstractType<?> expandUserTypes()
+    {
+        return getInstance(elements.expandUserTypes(), isMultiCell);
     }
 
     public AbstractType<T> getElementsType()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/db/marshal/TupleType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/marshal/TupleType.java b/src/java/org/apache/cassandra/db/marshal/TupleType.java
index be9cc93..00f4d24 100644
--- a/src/java/org/apache/cassandra/db/marshal/TupleType.java
+++ b/src/java/org/apache/cassandra/db/marshal/TupleType.java
@@ -23,9 +23,9 @@ import java.util.Arrays;
 import java.util.Iterator;
 import java.util.List;
 import java.util.regex.Pattern;
-import java.util.stream.Collectors;
 
 import com.google.common.base.Objects;
+import com.google.common.collect.Lists;
 
 import org.apache.cassandra.cql3.*;
 import org.apache.cassandra.exceptions.ConfigurationException;
@@ -35,6 +35,9 @@ import org.apache.cassandra.serializers.*;
 import org.apache.cassandra.transport.ProtocolVersion;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
+import static com.google.common.collect.Iterables.any;
+import static com.google.common.collect.Iterables.transform;
+
 /**
  * This is essentially like a CompositeType, but it's not primarily meant for comparison, just
  * to pack multiple values together so has a more friendly encoding.
@@ -62,8 +65,9 @@ public class TupleType extends AbstractType<ByteBuffer>
     protected TupleType(List<AbstractType<?>> types, boolean freezeInner)
     {
         super(ComparisonType.CUSTOM);
+
         if (freezeInner)
-            this.types = types.stream().map(AbstractType::freeze).collect(Collectors.toList());
+            this.types = Lists.newArrayList(transform(types, AbstractType::freeze));
         else
             this.types = types;
         this.serializer = new TupleSerializer(fieldSerializers(types));
@@ -87,9 +91,23 @@ public class TupleType extends AbstractType<ByteBuffer>
     }
 
     @Override
-    public boolean referencesUserType(String name)
+    public boolean referencesUserType(ByteBuffer name)
+    {
+        return any(types, t -> t.referencesUserType(name));
+    }
+
+    @Override
+    public TupleType withUpdatedUserType(UserType udt)
+    {
+        return referencesUserType(udt.name)
+             ? new TupleType(Lists.newArrayList(transform(types, t -> t.withUpdatedUserType(udt))))
+             : this;
+    }
+
+    @Override
+    public AbstractType<?> expandUserTypes()
     {
-        return allTypes().stream().anyMatch(f -> f.referencesUserType(name));
+        return new TupleType(Lists.newArrayList(transform(types, AbstractType::expandUserTypes)));
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/db/marshal/UserType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/marshal/UserType.java b/src/java/org/apache/cassandra/db/marshal/UserType.java
index 6149e94..01e6a3f 100644
--- a/src/java/org/apache/cassandra/db/marshal/UserType.java
+++ b/src/java/org/apache/cassandra/db/marshal/UserType.java
@@ -22,20 +22,21 @@ import java.util.*;
 import java.util.stream.Collectors;
 
 import com.google.common.base.Objects;
+import com.google.common.collect.Lists;
 
 import org.apache.cassandra.cql3.*;
 import org.apache.cassandra.db.rows.Cell;
 import org.apache.cassandra.db.rows.CellPath;
-import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.exceptions.SyntaxException;
+import org.apache.cassandra.schema.Difference;
 import org.apache.cassandra.serializers.MarshalException;
 import org.apache.cassandra.transport.ProtocolVersion;
 import org.apache.cassandra.serializers.TypeSerializer;
 import org.apache.cassandra.serializers.UserTypeSerializer;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.Pair;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+
+import static com.google.common.collect.Iterables.any;
+import static com.google.common.collect.Iterables.transform;
 
 /**
  * A user defined type.
@@ -44,8 +45,6 @@ import org.slf4j.LoggerFactory;
  */
 public class UserType extends TupleType
 {
-    private static final Logger logger = LoggerFactory.getLogger(UserType.class);
-
     public final String keyspace;
     public final ByteBuffer name;
     private final List<FieldIdentifier> fieldNames;
@@ -73,7 +72,7 @@ public class UserType extends TupleType
         this.serializer = new UserTypeSerializer(fieldSerializers);
     }
 
-    public static UserType getInstance(TypeParser parser) throws ConfigurationException, SyntaxException
+    public static UserType getInstance(TypeParser parser)
     {
         Pair<Pair<String, ByteBuffer>, List<Pair<ByteBuffer, AbstractType>>> params = parser.getUserTypeParameters();
         String keyspace = params.left.left;
@@ -333,34 +332,44 @@ public class UserType extends TupleType
     @Override
     public boolean equals(Object o)
     {
-        return o instanceof UserType && equals(o, false);
-    }
-
-    @Override
-    public boolean equals(Object o, boolean ignoreFreezing)
-    {
         if(!(o instanceof UserType))
             return false;
 
         UserType that = (UserType)o;
 
-        if (!keyspace.equals(that.keyspace) || !name.equals(that.name) || !fieldNames.equals(that.fieldNames))
-            return false;
+        return equalsWithoutTypes(that) && types.equals(that.types);
+    }
 
-        if (!ignoreFreezing && isMultiCell != that.isMultiCell)
-            return false;
+    private boolean equalsWithoutTypes(UserType other)
+    {
+        return name.equals(other.name)
+            && fieldNames.equals(other.fieldNames)
+            && keyspace.equals(other.keyspace)
+            && isMultiCell == other.isMultiCell;
+    }
 
-        if (this.types.size() != that.types.size())
-            return false;
+    public Optional<Difference> compare(UserType other)
+    {
+        if (!equalsWithoutTypes(other))
+            return Optional.of(Difference.SHALLOW);
+
+        boolean differsDeeply = false;
 
-        Iterator<AbstractType<?>> otherTypeIter = that.types.iterator();
-        for (AbstractType<?> type : types)
+        for (int i = 0; i < fieldTypes().size(); i++)
         {
-            if (!type.equals(otherTypeIter.next(), ignoreFreezing))
-                return false;
+            AbstractType<?> thisType = fieldType(i);
+            AbstractType<?> thatType = other.fieldType(i);
+
+            if (!thisType.equals(thatType))
+            {
+                if (thisType.asCQL3Type().toString().equals(thatType.asCQL3Type().toString()))
+                    differsDeeply = true;
+                else
+                    return Optional.of(Difference.SHALLOW);
+            }
         }
 
-        return true;
+        return differsDeeply ? Optional.of(Difference.DEEP) : Optional.empty();
     }
 
     @Override
@@ -370,10 +379,30 @@ public class UserType extends TupleType
     }
 
     @Override
-    public boolean referencesUserType(String userTypeName)
+    public boolean referencesUserType(ByteBuffer name)
+    {
+        return this.name.equals(name) || any(fieldTypes(), t -> t.referencesUserType(name));
+    }
+
+    @Override
+    public UserType withUpdatedUserType(UserType udt)
     {
-        return getNameAsString().equals(userTypeName) ||
-               fieldTypes().stream().anyMatch(f -> f.referencesUserType(userTypeName));
+        if (!referencesUserType(udt.name))
+            return this;
+
+        // preserve frozen/non-frozen status of the updated UDT
+        if (name.equals(udt.name))
+        {
+            return isMultiCell == udt.isMultiCell
+                 ? udt
+                 : new UserType(keyspace, name, udt.fieldNames(), udt.fieldTypes(), isMultiCell);
+        }
+
+        return new UserType(keyspace,
+                            name,
+                            fieldNames,
+                            Lists.newArrayList(transform(fieldTypes(), t -> t.withUpdatedUserType(udt))),
+                            isMultiCell());
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/db/view/TableViews.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/view/TableViews.java b/src/java/org/apache/cassandra/db/view/TableViews.java
index 298fcfd..eedcfbd 100644
--- a/src/java/org/apache/cassandra/db/view/TableViews.java
+++ b/src/java/org/apache/cassandra/db/view/TableViews.java
@@ -85,7 +85,7 @@ public class TableViews extends AbstractCollection<View>
     public Iterable<ColumnFamilyStore> allViewsCfs()
     {
         Keyspace keyspace = Keyspace.open(baseTableMetadata.keyspace);
-        return Iterables.transform(views, view -> keyspace.getColumnFamilyStore(view.getDefinition().name));
+        return Iterables.transform(views, view -> keyspace.getColumnFamilyStore(view.getDefinition().name()));
     }
 
     public void forceBlockingFlush()


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


[07/13] cassandra git commit: Make all DDL statements idempotent and not dependent on global state

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/cql3/statements/schema/CreateFunctionStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/schema/CreateFunctionStatement.java b/src/java/org/apache/cassandra/cql3/statements/schema/CreateFunctionStatement.java
new file mode 100644
index 0000000..13e173f
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/statements/schema/CreateFunctionStatement.java
@@ -0,0 +1,250 @@
+/*
+ * 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.statements.schema;
+
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Lists;
+
+import org.apache.cassandra.audit.AuditLogContext;
+import org.apache.cassandra.audit.AuditLogEntryType;
+import org.apache.cassandra.auth.FunctionResource;
+import org.apache.cassandra.auth.IResource;
+import org.apache.cassandra.auth.*;
+import org.apache.cassandra.cql3.CQL3Type;
+import org.apache.cassandra.cql3.CQLStatement;
+import org.apache.cassandra.cql3.ColumnIdentifier;
+import org.apache.cassandra.cql3.functions.Function;
+import org.apache.cassandra.cql3.functions.FunctionName;
+import org.apache.cassandra.cql3.functions.UDFunction;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.schema.Functions.FunctionsDiff;
+import org.apache.cassandra.schema.KeyspaceMetadata;
+import org.apache.cassandra.schema.Keyspaces;
+import org.apache.cassandra.schema.Keyspaces.KeyspacesDiff;
+import org.apache.cassandra.schema.Schema;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.transport.Event.SchemaChange;
+import org.apache.cassandra.transport.Event.SchemaChange.Change;
+import org.apache.cassandra.transport.Event.SchemaChange.Target;
+
+import static java.util.stream.Collectors.toList;
+
+public final class CreateFunctionStatement extends AlterSchemaStatement
+{
+    private final String functionName;
+    private final List<ColumnIdentifier> argumentNames;
+    private final List<CQL3Type.Raw> rawArgumentTypes;
+    private final CQL3Type.Raw rawReturnType;
+    private final boolean calledOnNullInput;
+    private final String language;
+    private final String body;
+    private final boolean orReplace;
+    private final boolean ifNotExists;
+
+    public CreateFunctionStatement(String keyspaceName,
+                                   String functionName,
+                                   List<ColumnIdentifier> argumentNames,
+                                   List<CQL3Type.Raw> rawArgumentTypes,
+                                   CQL3Type.Raw rawReturnType,
+                                   boolean calledOnNullInput,
+                                   String language,
+                                   String body,
+                                   boolean orReplace,
+                                   boolean ifNotExists)
+    {
+        super(keyspaceName);
+        this.functionName = functionName;
+        this.argumentNames = argumentNames;
+        this.rawArgumentTypes = rawArgumentTypes;
+        this.rawReturnType = rawReturnType;
+        this.calledOnNullInput = calledOnNullInput;
+        this.language = language;
+        this.body = body;
+        this.orReplace = orReplace;
+        this.ifNotExists = ifNotExists;
+    }
+
+    // TODO: replace affected aggregates !!
+    public Keyspaces apply(Keyspaces schema)
+    {
+        if (ifNotExists && orReplace)
+            throw ire("Cannot use both 'OR REPLACE' and 'IF NOT EXISTS' directives");
+
+        UDFunction.assertUdfsEnabled(language);
+
+        if (new HashSet<>(argumentNames).size() != argumentNames.size())
+            throw ire("Duplicate argument names for given function %s with argument names %s", functionName, argumentNames);
+
+        rawArgumentTypes.stream()
+                        .filter(CQL3Type.Raw::isFrozen)
+                        .findFirst()
+                        .ifPresent(t -> { throw ire("Argument '%s' cannot be frozen; remove frozen<> modifier from '%s'", t, t); });
+
+        if (rawReturnType.isFrozen())
+            throw ire("Return type '%s' cannot be frozen; remove frozen<> modifier from '%s'", rawReturnType, rawReturnType);
+
+        KeyspaceMetadata keyspace = schema.getNullable(keyspaceName);
+        if (null == keyspace)
+            throw ire("Keyspace '%s' doesn't exist", keyspaceName);
+
+        List<AbstractType<?>> argumentTypes =
+            rawArgumentTypes.stream()
+                            .map(t -> t.prepare(keyspaceName, keyspace.types).getType())
+                            .collect(toList());
+        AbstractType<?> returnType = rawReturnType.prepare(keyspaceName, keyspace.types).getType();
+
+        UDFunction function =
+            UDFunction.create(new FunctionName(keyspaceName, functionName),
+                              argumentNames,
+                              argumentTypes,
+                              returnType,
+                              calledOnNullInput,
+                              language,
+                              body);
+
+        Function existingFunction = keyspace.functions.find(function.name(), argumentTypes).orElse(null);
+        if (null != existingFunction)
+        {
+            if (existingFunction.isAggregate())
+                throw ire("Function '%s' cannot replace an aggregate", functionName);
+
+            if (ifNotExists)
+                return schema;
+
+            if (!orReplace)
+                throw ire("Function '%s' already exists", functionName);
+
+            if (calledOnNullInput != ((UDFunction) existingFunction).isCalledOnNullInput())
+            {
+                throw ire("Function '%s' must have %s directive",
+                          functionName,
+                          calledOnNullInput ? "CALLED ON NULL INPUT" : "RETURNS NULL ON NULL INPUT");
+            }
+
+            if (!returnType.isCompatibleWith(existingFunction.returnType()))
+            {
+                throw ire("Cannot replace function '%s', the new return type %s is not compatible with the return type %s of existing function",
+                          functionName,
+                          returnType.asCQL3Type(),
+                          existingFunction.returnType().asCQL3Type());
+            }
+
+            // TODO: update dependent aggregates
+        }
+
+        return schema.withAddedOrUpdated(keyspace.withSwapped(keyspace.functions.withAddedOrUpdated(function)));
+    }
+
+    SchemaChange schemaChangeEvent(KeyspacesDiff diff)
+    {
+        assert diff.altered.size() == 1;
+        FunctionsDiff<UDFunction> udfsDiff = diff.altered.get(0).udfs;
+
+        assert udfsDiff.created.size() + udfsDiff.altered.size() == 1;
+        boolean created = !udfsDiff.created.isEmpty();
+
+        return new SchemaChange(created ? Change.CREATED : Change.UPDATED,
+                                Target.FUNCTION,
+                                keyspaceName,
+                                functionName,
+                                rawArgumentTypes.stream().map(CQL3Type.Raw::toString).collect(toList()));
+    }
+
+    public void authorize(ClientState client)
+    {
+        FunctionName name = new FunctionName(keyspaceName, functionName);
+
+        if (Schema.instance.findFunction(name, Lists.transform(rawArgumentTypes, t -> t.prepare(keyspaceName).getType())).isPresent() && orReplace)
+            client.ensurePermission(Permission.ALTER, FunctionResource.functionFromCql(keyspaceName, functionName, rawArgumentTypes));
+        else
+            client.ensurePermission(Permission.CREATE, FunctionResource.keyspace(keyspaceName));
+    }
+
+    @Override
+    Set<IResource> createdResources(KeyspacesDiff diff)
+    {
+        assert diff.altered.size() == 1;
+        FunctionsDiff<UDFunction> udfsDiff = diff.altered.get(0).udfs;
+
+        assert udfsDiff.created.size() + udfsDiff.altered.size() == 1;
+
+        return udfsDiff.created.isEmpty()
+             ? ImmutableSet.of()
+             : ImmutableSet.of(FunctionResource.functionFromCql(keyspaceName, functionName, rawArgumentTypes));
+    }
+
+    @Override
+    public AuditLogContext getAuditLogContext()
+    {
+        return new AuditLogContext(AuditLogEntryType.CREATE_FUNCTION, keyspaceName, functionName);
+    }
+
+    public static final class Raw extends CQLStatement.Raw
+    {
+        private final FunctionName name;
+        private final List<ColumnIdentifier> argumentNames;
+        private final List<CQL3Type.Raw> rawArgumentTypes;
+        private final CQL3Type.Raw rawReturnType;
+        private final boolean calledOnNullInput;
+        private final String language;
+        private final String body;
+        private final boolean orReplace;
+        private final boolean ifNotExists;
+
+        public Raw(FunctionName name,
+                   List<ColumnIdentifier> argumentNames,
+                   List<CQL3Type.Raw> rawArgumentTypes,
+                   CQL3Type.Raw rawReturnType,
+                   boolean calledOnNullInput,
+                   String language,
+                   String body,
+                   boolean orReplace,
+                   boolean ifNotExists)
+        {
+            this.name = name;
+            this.argumentNames = argumentNames;
+            this.rawArgumentTypes = rawArgumentTypes;
+            this.rawReturnType = rawReturnType;
+            this.calledOnNullInput = calledOnNullInput;
+            this.language = language;
+            this.body = body;
+            this.orReplace = orReplace;
+            this.ifNotExists = ifNotExists;
+        }
+
+        public CreateFunctionStatement prepare(ClientState state)
+        {
+            String keyspaceName = name.hasKeyspace() ? name.keyspace : state.getKeyspace();
+
+            return new CreateFunctionStatement(keyspaceName,
+                                               name.name,
+                                               argumentNames,
+                                               rawArgumentTypes,
+                                               rawReturnType,
+                                               calledOnNullInput,
+                                               language,
+                                               body,
+                                               orReplace,
+                                               ifNotExists);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/cql3/statements/schema/CreateIndexStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/schema/CreateIndexStatement.java b/src/java/org/apache/cassandra/cql3/statements/schema/CreateIndexStatement.java
new file mode 100644
index 0000000..0065a4c
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/statements/schema/CreateIndexStatement.java
@@ -0,0 +1,233 @@
+/*
+ * 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.statements.schema;
+
+import java.util.*;
+
+import com.google.common.collect.Lists;
+
+import org.apache.cassandra.audit.AuditLogContext;
+import org.apache.cassandra.audit.AuditLogEntryType;
+import org.apache.cassandra.auth.Permission;
+import org.apache.cassandra.cql3.CQLStatement;
+import org.apache.cassandra.cql3.ColumnIdentifier;
+import org.apache.cassandra.cql3.QualifiedName;
+import org.apache.cassandra.cql3.statements.schema.IndexTarget.Type;
+import org.apache.cassandra.db.marshal.MapType;
+import org.apache.cassandra.schema.*;
+import org.apache.cassandra.schema.Keyspaces.KeyspacesDiff;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.transport.Event.SchemaChange;
+import org.apache.cassandra.transport.Event.SchemaChange.Change;
+import org.apache.cassandra.transport.Event.SchemaChange.Target;
+
+import static com.google.common.collect.Iterables.transform;
+import static com.google.common.collect.Iterables.tryFind;
+
+public final class CreateIndexStatement extends AlterSchemaStatement
+{
+    private final String indexName;
+    private final String tableName;
+    private final List<IndexTarget.Raw> rawIndexTargets;
+    private final IndexAttributes attrs;
+    private final boolean ifNotExists;
+
+    public CreateIndexStatement(String keyspaceName,
+                                String tableName,
+                                String indexName,
+                                List<IndexTarget.Raw> rawIndexTargets,
+                                IndexAttributes attrs,
+                                boolean ifNotExists)
+    {
+        super(keyspaceName);
+        this.tableName = tableName;
+        this.indexName = indexName;
+        this.rawIndexTargets = rawIndexTargets;
+        this.attrs = attrs;
+        this.ifNotExists = ifNotExists;
+    }
+
+    public Keyspaces apply(Keyspaces schema)
+    {
+        attrs.validate();
+
+        KeyspaceMetadata keyspace = schema.getNullable(keyspaceName);
+        if (null == keyspace)
+            throw ire("Keyspace '%s' doesn't exist", keyspaceName);
+
+        TableMetadata table = keyspace.getTableOrViewNullable(tableName);
+        if (null == table)
+            throw ire("Table '%s' doesn't exist", tableName);
+
+        if (null != indexName && keyspace.hasIndex(indexName))
+        {
+            if (ifNotExists)
+                return schema;
+
+            throw ire("Index '%s' already exists", indexName);
+        }
+
+        if (table.isCounter())
+            throw ire("Secondary indexes on counter tables aren't supported");
+
+        if (table.isView())
+            throw ire("Secondary indexes on materialized views aren't supported");
+
+        List<IndexTarget> indexTargets = Lists.newArrayList(transform(rawIndexTargets, t -> t.prepare(table)));
+
+        if (indexTargets.isEmpty() && !attrs.isCustom)
+            throw ire("Only CUSTOM indexes can be created without specifying a target column");
+
+        if (indexTargets.size() > 1)
+        {
+            if (!attrs.isCustom)
+                throw ire("Only CUSTOM indexes support multiple columns");
+
+            Set<ColumnIdentifier> columns = new HashSet<>();
+            for (IndexTarget target : indexTargets)
+                if (!columns.add(target.column))
+                    throw ire("Duplicate column '%s' in index target list", target.column);
+        }
+
+        indexTargets.forEach(t -> validateIndexTarget(table, t));
+
+        String name = null == indexName ? generateIndexName(keyspace, indexTargets) : indexName;
+
+        IndexMetadata.Kind kind = attrs.isCustom ? IndexMetadata.Kind.CUSTOM : IndexMetadata.Kind.COMPOSITES;
+
+        Map<String, String> options = attrs.isCustom ? attrs.getOptions() : Collections.emptyMap();
+
+        IndexMetadata index = IndexMetadata.fromIndexTargets(indexTargets, name, kind, options);
+
+        // check to disallow creation of an index which duplicates an existing one in all but name
+        IndexMetadata equalIndex = tryFind(table.indexes, i -> i.equalsWithoutName(index)).orNull();
+        if (null != equalIndex)
+        {
+            if (ifNotExists)
+                return schema;
+
+            throw ire("Index %s is a duplicate of existing index %s", index.name, equalIndex.name);
+        }
+
+        TableMetadata newTable = table.withSwapped(table.indexes.with(index));
+        newTable.validate();
+
+        return schema.withAddedOrUpdated(keyspace.withSwapped(keyspace.tables.withSwapped(newTable)));
+    }
+
+    private void validateIndexTarget(TableMetadata table, IndexTarget target)
+    {
+        ColumnMetadata column = table.getColumn(target.column);
+
+        if (null == column)
+            throw ire("Column '%s' doesn't exist", target.column);
+
+        if (column.type.referencesDuration())
+        {
+            if (column.type.isCollection())
+                throw ire("Secondary indexes are not supported on collections containing durations");
+
+            if (column.type.isTuple())
+                throw ire("Secondary indexes are not supported on tuples containing durations");
+
+            if (column.type.isUDT())
+                throw  ire("Secondary indexes are not supported on UDTs containing durations");
+
+            throw ire("Secondary indexes are not supported on duration columns");
+        }
+
+        if (column.isPartitionKey() && table.partitionKeyColumns().size() == 1)
+            throw ire("Cannot create secondary index on the only partition key column %s", column);
+
+        if (column.type.isFrozenCollection() && target.type != Type.FULL)
+            throw ire("Cannot create %s() index on frozen column %s. Frozen collections only support full() indexes", target.type, column);
+
+        if (!column.type.isFrozenCollection() && target.type == Type.FULL)
+            throw ire("full() indexes can only be created on frozen collections");
+
+        if (!column.type.isCollection() && target.type != Type.SIMPLE)
+            throw ire("Cannot create %s() index on %s. Non-collection columns only support simple indexes", target.type, column);
+
+        if (!(column.type instanceof MapType && column.type.isMultiCell()) && (target.type == Type.KEYS || target.type == Type.KEYS_AND_VALUES))
+            throw ire("Cannot create index on %s of column %s with non-map type", target.type, column);
+
+        if (column.type.isUDT() && column.type.isMultiCell())
+            throw ire("Cannot create index on non-frozen UDT column %s", column);
+    }
+
+    private String generateIndexName(KeyspaceMetadata keyspace, List<IndexTarget> targets)
+    {
+        String baseName = targets.size() == 1
+                        ? IndexMetadata.generateDefaultIndexName(tableName, targets.get(0).column)
+                        : IndexMetadata.generateDefaultIndexName(tableName);
+        return keyspace.findAvailableIndexName(baseName);
+    }
+
+    SchemaChange schemaChangeEvent(KeyspacesDiff diff)
+    {
+        return new SchemaChange(Change.UPDATED, Target.TABLE, keyspaceName, tableName);
+    }
+
+    public void authorize(ClientState client)
+    {
+        client.ensureTablePermission(keyspaceName, tableName, Permission.ALTER);
+    }
+
+    @Override
+    public AuditLogContext getAuditLogContext()
+    {
+        return new AuditLogContext(AuditLogEntryType.CREATE_INDEX, keyspaceName, indexName);
+    }
+
+    public static final class Raw extends CQLStatement.Raw
+    {
+        private final QualifiedName tableName;
+        private final QualifiedName indexName;
+        private final List<IndexTarget.Raw> rawIndexTargets;
+        private final IndexAttributes attrs;
+        private final boolean ifNotExists;
+
+        public Raw(QualifiedName tableName,
+                   QualifiedName indexName,
+                   List<IndexTarget.Raw> rawIndexTargets,
+                   IndexAttributes attrs,
+                   boolean ifNotExists)
+        {
+            this.tableName = tableName;
+            this.indexName = indexName;
+            this.rawIndexTargets = rawIndexTargets;
+            this.attrs = attrs;
+            this.ifNotExists = ifNotExists;
+        }
+
+        public CreateIndexStatement prepare(ClientState state)
+        {
+            String keyspaceName = tableName.hasKeyspace()
+                                ? tableName.getKeyspace()
+                                : indexName.hasKeyspace() ? indexName.getKeyspace() : state.getKeyspace();
+
+            if (tableName.hasKeyspace() && !keyspaceName.equals(tableName.getKeyspace()))
+                throw ire("Keyspace name '%s' doesn't match table name '%s'", keyspaceName, tableName);
+
+            if (indexName.hasKeyspace() && !keyspaceName.equals(indexName.getKeyspace()))
+                throw ire("Keyspace name '%s' doesn't match index name '%s'", keyspaceName, tableName);
+
+            return new CreateIndexStatement(keyspaceName, tableName.getName(), indexName.getName(), rawIndexTargets, attrs, ifNotExists);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/cql3/statements/schema/CreateKeyspaceStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/schema/CreateKeyspaceStatement.java b/src/java/org/apache/cassandra/cql3/statements/schema/CreateKeyspaceStatement.java
new file mode 100644
index 0000000..ecd19ed
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/statements/schema/CreateKeyspaceStatement.java
@@ -0,0 +1,115 @@
+/*
+ * 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.statements.schema;
+
+import java.util.Set;
+
+import com.google.common.collect.ImmutableSet;
+
+import org.apache.cassandra.audit.AuditLogContext;
+import org.apache.cassandra.audit.AuditLogEntryType;
+import org.apache.cassandra.auth.*;
+import org.apache.cassandra.cql3.CQLStatement;
+import org.apache.cassandra.exceptions.AlreadyExistsException;
+import org.apache.cassandra.locator.LocalStrategy;
+import org.apache.cassandra.schema.KeyspaceMetadata;
+import org.apache.cassandra.schema.KeyspaceParams.Option;
+import org.apache.cassandra.schema.Keyspaces;
+import org.apache.cassandra.schema.Keyspaces.KeyspacesDiff;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.transport.Event.SchemaChange;
+import org.apache.cassandra.transport.Event.SchemaChange.Change;
+
+public final class CreateKeyspaceStatement extends AlterSchemaStatement
+{
+    private final KeyspaceAttributes attrs;
+    private final boolean ifNotExists;
+
+    public CreateKeyspaceStatement(String keyspaceName, KeyspaceAttributes attrs, boolean ifNotExists)
+    {
+        super(keyspaceName);
+        this.attrs = attrs;
+        this.ifNotExists = ifNotExists;
+    }
+
+    public Keyspaces apply(Keyspaces schema)
+    {
+        attrs.validate();
+
+        if (!attrs.hasOption(Option.REPLICATION))
+            throw ire("Missing mandatory option '%s'", Option.REPLICATION);
+
+        if (schema.containsKeyspace(keyspaceName))
+        {
+            if (ifNotExists)
+                return schema;
+
+            throw new AlreadyExistsException(keyspaceName);
+        }
+
+        KeyspaceMetadata keyspace = KeyspaceMetadata.create(keyspaceName, attrs.asNewKeyspaceParams());
+
+        if (keyspace.params.replication.klass.equals(LocalStrategy.class))
+            throw ire("Unable to use given strategy class: LocalStrategy is reserved for internal use.");
+
+        keyspace.params.validate(keyspaceName);
+
+        return schema.withAddedOrUpdated(keyspace);
+    }
+
+    SchemaChange schemaChangeEvent(KeyspacesDiff diff)
+    {
+        return new SchemaChange(Change.CREATED, keyspaceName);
+    }
+
+    public void authorize(ClientState client)
+    {
+        client.ensureAllKeyspacesPermission(Permission.CREATE);
+    }
+
+    @Override
+    Set<IResource> createdResources(KeyspacesDiff diff)
+    {
+        return ImmutableSet.of(DataResource.keyspace(keyspaceName), FunctionResource.keyspace(keyspaceName));
+    }
+
+    @Override
+    public AuditLogContext getAuditLogContext()
+    {
+        return new AuditLogContext(AuditLogEntryType.CREATE_KEYSPACE, keyspaceName);
+    }
+
+    public static final class Raw extends CQLStatement.Raw
+    {
+        public final String keyspaceName;
+        private final KeyspaceAttributes attrs;
+        private final boolean ifNotExists;
+
+        public Raw(String keyspaceName, KeyspaceAttributes attrs, boolean ifNotExists)
+        {
+            this.keyspaceName = keyspaceName;
+            this.attrs = attrs;
+            this.ifNotExists = ifNotExists;
+        }
+
+        public CreateKeyspaceStatement prepare(ClientState state)
+        {
+            return new CreateKeyspaceStatement(keyspaceName, attrs, ifNotExists);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/cql3/statements/schema/CreateTableStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/schema/CreateTableStatement.java b/src/java/org/apache/cassandra/cql3/statements/schema/CreateTableStatement.java
new file mode 100644
index 0000000..ff26f0d
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/statements/schema/CreateTableStatement.java
@@ -0,0 +1,348 @@
+/*
+ * 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.statements.schema;
+
+import java.util.*;
+
+import com.google.common.collect.ImmutableSet;
+
+import org.apache.cassandra.audit.AuditLogContext;
+import org.apache.cassandra.audit.AuditLogEntryType;
+import org.apache.cassandra.auth.DataResource;
+import org.apache.cassandra.auth.IResource;
+import org.apache.cassandra.auth.Permission;
+import org.apache.cassandra.cql3.*;
+import org.apache.cassandra.db.marshal.*;
+import org.apache.cassandra.exceptions.AlreadyExistsException;
+import org.apache.cassandra.schema.*;
+import org.apache.cassandra.schema.Keyspaces.KeyspacesDiff;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.transport.Event.SchemaChange;
+import org.apache.cassandra.transport.Event.SchemaChange.Change;
+import org.apache.cassandra.transport.Event.SchemaChange.Target;
+
+import static java.util.Comparator.comparing;
+
+import static com.google.common.collect.Iterables.concat;
+
+public final class CreateTableStatement extends AlterSchemaStatement
+{
+    private final String tableName;
+
+    private final Map<ColumnIdentifier, CQL3Type.Raw> rawColumns;
+    private final Set<ColumnIdentifier> staticColumns;
+    private final List<ColumnIdentifier> partitionKeyColumns;
+    private final List<ColumnIdentifier> clusteringColumns;
+
+    private final LinkedHashMap<ColumnIdentifier, Boolean> clusteringOrder;
+    private final TableAttributes attrs;
+
+    private final boolean ifNotExists;
+
+    public CreateTableStatement(String keyspaceName,
+                                String tableName,
+
+                                Map<ColumnIdentifier, CQL3Type.Raw> rawColumns,
+                                Set<ColumnIdentifier> staticColumns,
+                                List<ColumnIdentifier> partitionKeyColumns,
+                                List<ColumnIdentifier> clusteringColumns,
+
+                                LinkedHashMap<ColumnIdentifier, Boolean> clusteringOrder,
+                                TableAttributes attrs,
+
+                                boolean ifNotExists)
+    {
+        super(keyspaceName);
+        this.tableName = tableName;
+
+        this.rawColumns = rawColumns;
+        this.staticColumns = staticColumns;
+        this.partitionKeyColumns = partitionKeyColumns;
+        this.clusteringColumns = clusteringColumns;
+
+        this.clusteringOrder = clusteringOrder;
+        this.attrs = attrs;
+
+        this.ifNotExists = ifNotExists;
+    }
+
+    public Keyspaces apply(Keyspaces schema)
+    {
+        KeyspaceMetadata keyspace = schema.getNullable(keyspaceName);
+        if (null == keyspace)
+            throw ire("Keyspace '%s' doesn't exist", keyspaceName);
+
+        if (keyspace.hasTable(tableName))
+        {
+            if (ifNotExists)
+                return schema;
+
+            throw new AlreadyExistsException(keyspaceName, tableName);
+        }
+
+        TableMetadata table = builder(keyspace.types).build();
+        table.validate();
+
+        return schema.withAddedOrUpdated(keyspace.withSwapped(keyspace.tables.with(table)));
+    }
+
+    SchemaChange schemaChangeEvent(KeyspacesDiff diff)
+    {
+        return new SchemaChange(Change.CREATED, Target.TABLE, keyspaceName, tableName);
+    }
+
+    public void authorize(ClientState client)
+    {
+        client.ensureKeyspacePermission(keyspaceName, Permission.CREATE);
+    }
+
+    @Override
+    Set<IResource> createdResources(KeyspacesDiff diff)
+    {
+        return ImmutableSet.of(DataResource.table(keyspaceName, tableName));
+    }
+
+    @Override
+    public AuditLogContext getAuditLogContext()
+    {
+        return new AuditLogContext(AuditLogEntryType.CREATE_TABLE, keyspaceName, tableName);
+    }
+
+    public TableMetadata.Builder builder(Types types)
+    {
+        attrs.validate();
+        TableParams params = attrs.asNewTableParams();
+
+        // use a TreeMap to preserve ordering across JDK versions (see CASSANDRA-9492) - important for stable unit tests
+        Map<ColumnIdentifier, CQL3Type> columns = new TreeMap<>(comparing(o -> o.bytes));
+        rawColumns.forEach((column, type) -> columns.put(column, type.prepare(keyspaceName, types)));
+
+        // check for nested non-frozen UDTs or collections in a non-frozen UDT
+        columns.forEach((column, type) ->
+        {
+            if (type.isUDT() && type.getType().isMultiCell())
+            {
+                ((UserType) type.getType()).fieldTypes().forEach(field ->
+                {
+                    if (field.isMultiCell())
+                        throw ire("Non-frozen UDTs with nested non-frozen collections are not supported");
+                });
+            }
+        });
+
+        /*
+         * Deal with PRIMARY KEY columns
+         */
+
+        HashSet<ColumnIdentifier> primaryKeyColumns = new HashSet<>();
+        concat(partitionKeyColumns, clusteringColumns).forEach(column ->
+        {
+            CQL3Type type = columns.get(column);
+            if (null == type)
+                throw ire("Unknown column '%s' referenced in PRIMARY KEY for table '%s'", column, tableName);
+
+            if (!primaryKeyColumns.add(column))
+                throw ire("Duplicate column '%s' in PRIMARY KEY clause for table '%s'", column, tableName);
+
+            if (type.getType().isMultiCell())
+            {
+                if (type.isCollection())
+                    throw ire("Invalid non-frozen collection type %s for PRIMARY KEY column '%s'", type, column);
+                else
+                    throw ire("Invalid non-frozen user-defined type %s for PRIMARY KEY column '%s'", type, column);
+            }
+
+            if (type.getType().isCounter())
+                throw ire("counter type is not supported for PRIMARY KEY column '%s'", column);
+
+            if (type.getType().referencesDuration())
+                throw ire("duration type is not supported for PRIMARY KEY column '%s'", column);
+
+            if (staticColumns.contains(column))
+                throw ire("Static column '%s' cannot be part of the PRIMARY KEY", column);
+        });
+
+        List<AbstractType<?>> partitionKeyTypes = new ArrayList<>();
+        List<AbstractType<?>> clusteringTypes = new ArrayList<>();
+
+        partitionKeyColumns.forEach(column ->
+        {
+            CQL3Type type = columns.remove(column);
+            partitionKeyTypes.add(type.getType());
+        });
+
+        clusteringColumns.forEach(column ->
+        {
+            CQL3Type type = columns.remove(column);
+            boolean reverse = !clusteringOrder.getOrDefault(column, true);
+            clusteringTypes.add(reverse ? ReversedType.getInstance(type.getType()) : type.getType());
+        });
+
+        // If we give a clustering order, we must explicitly do so for all aliases and in the order of the PK
+        // This wasn't previously enforced because of a bug in the implementation
+        if (!clusteringOrder.isEmpty() && !clusteringColumns.equals(new ArrayList<>(clusteringOrder.keySet())))
+            throw ire("Clustering key columns must exactly match columns in CLUSTERING ORDER BY directive");
+
+        // Static columns only make sense if we have at least one clustering column. Otherwise everything is static anyway
+        if (clusteringColumns.isEmpty() && !staticColumns.isEmpty())
+            throw ire("Static columns are only useful (and thus allowed) if the table has at least one clustering column");
+
+        /*
+         * Counter table validation
+         */
+
+        boolean hasCounters = rawColumns.values().stream().anyMatch(CQL3Type.Raw::isCounter);
+        if (hasCounters)
+        {
+            // We've handled anything that is not a PRIMARY KEY so columns only contains NON-PK columns. So
+            // if it's a counter table, make sure we don't have non-counter types
+            if (columns.values().stream().anyMatch(t -> !t.getType().isCounter()))
+                throw ire("Cannot mix counter and non counter columns in the same table");
+
+            if (params.defaultTimeToLive > 0)
+                throw ire("Cannot set %s on a table with counters", TableParams.Option.DEFAULT_TIME_TO_LIVE);
+        }
+
+        /*
+         * Create the builder
+         */
+
+        TableMetadata.Builder builder = TableMetadata.builder(keyspaceName, tableName);
+
+        if (attrs.hasProperty(TableAttributes.ID))
+            builder.id(attrs.getId());
+
+        builder.isCounter(hasCounters)
+               .params(params);
+
+        for (int i = 0; i < partitionKeyColumns.size(); i++)
+            builder.addPartitionKeyColumn(partitionKeyColumns.get(i), partitionKeyTypes.get(i));
+
+        for (int i = 0; i < clusteringColumns.size(); i++)
+            builder.addClusteringColumn(clusteringColumns.get(i), clusteringTypes.get(i));
+
+        columns.forEach((column, type) ->
+        {
+            if (staticColumns.contains(column))
+                builder.addStaticColumn(column, type.getType());
+            else
+                builder.addRegularColumn(column, type.getType());
+        });
+
+        return builder;
+    }
+
+    public static TableMetadata.Builder parse(String cql, String keyspace)
+    {
+        return CQLFragmentParser.parseAny(CqlParser::createTableStatement, cql, "CREATE TABLE")
+                                .keyspace(keyspace)
+                                .prepare(null) // works around a messy ClientState/QueryProcessor class init deadlock
+                                .builder(Types.none());
+    }
+
+    public final static class Raw extends CQLStatement.Raw
+    {
+        private final QualifiedName name;
+        private final boolean ifNotExists;
+
+        private final Map<ColumnIdentifier, CQL3Type.Raw> rawColumns = new HashMap<>();
+        private final Set<ColumnIdentifier> staticColumns = new HashSet<>();
+        private final List<ColumnIdentifier> clusteringColumns = new ArrayList<>();
+
+        private List<ColumnIdentifier> partitionKeyColumns;
+
+        private final LinkedHashMap<ColumnIdentifier, Boolean> clusteringOrder = new LinkedHashMap<>();
+        public final TableAttributes attrs = new TableAttributes();
+
+        public Raw(QualifiedName name, boolean ifNotExists)
+        {
+            this.name = name;
+            this.ifNotExists = ifNotExists;
+        }
+
+        public CreateTableStatement prepare(ClientState state)
+        {
+            String keyspaceName = name.hasKeyspace() ? name.getKeyspace() : state.getKeyspace();
+
+            if (null == partitionKeyColumns)
+                throw ire("No PRIMARY KEY specifed for table '%s' (exactly one required)", name);
+
+            return new CreateTableStatement(keyspaceName,
+                                            name.getName(),
+
+                                            rawColumns,
+                                            staticColumns,
+                                            partitionKeyColumns,
+                                            clusteringColumns,
+
+                                            clusteringOrder,
+                                            attrs,
+
+                                            ifNotExists);
+        }
+
+        public String keyspace()
+        {
+            return name.getKeyspace();
+        }
+
+        public Raw keyspace(String keyspace)
+        {
+            name.setKeyspace(keyspace, true);
+            return this;
+        }
+
+        public String table()
+        {
+            return name.getName();
+        }
+
+        public void addColumn(ColumnIdentifier column, CQL3Type.Raw type, boolean isStatic)
+        {
+            if (null != rawColumns.put(column, type))
+                throw ire("Duplicate column '%s' declaration for table '%s'", column, name);
+
+            if (isStatic)
+                staticColumns.add(column);
+        }
+
+        public void setPartitionKeyColumn(ColumnIdentifier column)
+        {
+            setPartitionKeyColumns(Collections.singletonList(column));
+        }
+
+        public void setPartitionKeyColumns(List<ColumnIdentifier> columns)
+        {
+            if (null != partitionKeyColumns)
+                throw ire("Multiple PRIMARY KEY specified for table '%s' (exactly one required)", name);
+
+            partitionKeyColumns = columns;
+        }
+
+        public void markClusteringColumn(ColumnIdentifier column)
+        {
+            clusteringColumns.add(column);
+        }
+
+        public void extendClusteringOrder(ColumnIdentifier column, boolean ascending)
+        {
+            if (null != clusteringOrder.put(column, ascending))
+                throw ire("Duplicate column '%s' in CLUSTERING ORDER BY clause for table '%s'", column, name);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/cql3/statements/schema/CreateTriggerStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/schema/CreateTriggerStatement.java b/src/java/org/apache/cassandra/cql3/statements/schema/CreateTriggerStatement.java
new file mode 100644
index 0000000..cb6d14e
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/statements/schema/CreateTriggerStatement.java
@@ -0,0 +1,120 @@
+/*
+ * 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.statements.schema;
+
+import org.apache.cassandra.audit.AuditLogContext;
+import org.apache.cassandra.audit.AuditLogEntryType;
+import org.apache.cassandra.cql3.CQLStatement;
+import org.apache.cassandra.cql3.QualifiedName;
+import org.apache.cassandra.schema.*;
+import org.apache.cassandra.schema.Keyspaces.KeyspacesDiff;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.triggers.TriggerExecutor;
+import org.apache.cassandra.transport.Event.SchemaChange;
+import org.apache.cassandra.transport.Event.SchemaChange.Change;
+import org.apache.cassandra.transport.Event.SchemaChange.Target;
+
+public final class CreateTriggerStatement extends AlterSchemaStatement
+{
+    private final String tableName;
+    private final String triggerName;
+    private final String triggerClass;
+    private final boolean ifNotExists;
+
+    public CreateTriggerStatement(String keyspaceName, String tableName, String triggerName, String triggerClass, boolean ifNotExists)
+    {
+        super(keyspaceName);
+        this.tableName = tableName;
+        this.triggerName = triggerName;
+        this.triggerClass = triggerClass;
+        this.ifNotExists = ifNotExists;
+    }
+
+    public Keyspaces apply(Keyspaces schema)
+    {
+        KeyspaceMetadata keyspace = schema.getNullable(keyspaceName);
+        if (null == keyspace)
+            throw ire("Keyspace '%s' doesn't exist", keyspaceName);
+
+        TableMetadata table = keyspace.getTableOrViewNullable(tableName);
+        if (null == table)
+            throw ire("Table '%s' doesn't exist", tableName);
+
+        if (table.isView())
+            throw ire("Cannot CREATE TRIGGER for a materialized view");
+
+        TriggerMetadata existingTrigger = table.triggers.get(triggerName).orElse(null);
+        if (null != existingTrigger)
+        {
+            if (ifNotExists)
+                return schema;
+
+            throw ire("Trigger '%s' already exists", triggerName);
+        }
+
+        try
+        {
+            TriggerExecutor.instance.loadTriggerInstance(triggerClass);
+        }
+        catch (Exception e)
+        {
+            throw ire("Trigger class '%s' couldn't be loaded", triggerClass);
+        }
+
+        TableMetadata newTable = table.withSwapped(table.triggers.with(TriggerMetadata.create(triggerName, triggerClass)));
+        return schema.withAddedOrUpdated(keyspace.withSwapped(keyspace.tables.withSwapped(newTable)));
+    }
+
+    SchemaChange schemaChangeEvent(KeyspacesDiff diff)
+    {
+        return new SchemaChange(Change.UPDATED, Target.TABLE, keyspaceName, tableName);
+    }
+
+    public void authorize(ClientState client)
+    {
+        client.ensureIsSuperuser("Only superusers are allowed to perform CREATE TRIGGER queries");
+    }
+
+    @Override
+    public AuditLogContext getAuditLogContext()
+    {
+        return new AuditLogContext(AuditLogEntryType.CREATE_TRIGGER, keyspaceName, triggerName);
+    }
+
+    public static final class Raw extends CQLStatement.Raw
+    {
+        private final QualifiedName tableName;
+        private final String triggerName;
+        private final String triggerClass;
+        private final boolean ifNotExists;
+
+        public Raw(QualifiedName tableName, String triggerName, String triggerClass, boolean ifNotExists)
+        {
+            this.tableName = tableName;
+            this.triggerName = triggerName;
+            this.triggerClass = triggerClass;
+            this.ifNotExists = ifNotExists;
+        }
+
+        public CreateTriggerStatement prepare(ClientState state)
+        {
+            String keyspaceName = tableName.hasKeyspace() ? tableName.getKeyspace() : state.getKeyspace();
+            return new CreateTriggerStatement(keyspaceName, tableName.getName(), triggerName, triggerClass, ifNotExists);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/cql3/statements/schema/CreateTypeStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/schema/CreateTypeStatement.java b/src/java/org/apache/cassandra/cql3/statements/schema/CreateTypeStatement.java
new file mode 100644
index 0000000..c328eb7
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/statements/schema/CreateTypeStatement.java
@@ -0,0 +1,151 @@
+/*
+ * 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.statements.schema;
+
+import java.util.*;
+
+import org.apache.cassandra.audit.AuditLogContext;
+import org.apache.cassandra.audit.AuditLogEntryType;
+import org.apache.cassandra.auth.Permission;
+import org.apache.cassandra.cql3.CQL3Type;
+import org.apache.cassandra.cql3.CQLStatement;
+import org.apache.cassandra.cql3.FieldIdentifier;
+import org.apache.cassandra.cql3.UTName;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.UserType;
+import org.apache.cassandra.schema.KeyspaceMetadata;
+import org.apache.cassandra.schema.Keyspaces;
+import org.apache.cassandra.schema.Keyspaces.KeyspacesDiff;
+import org.apache.cassandra.schema.Types;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.transport.Event.SchemaChange;
+import org.apache.cassandra.transport.Event.SchemaChange.Change;
+import org.apache.cassandra.transport.Event.SchemaChange.Target;
+
+import static org.apache.cassandra.utils.ByteBufferUtil.bytes;
+
+import static java.util.stream.Collectors.toList;
+
+public final class CreateTypeStatement extends AlterSchemaStatement
+{
+    private final String typeName;
+    private final List<FieldIdentifier> fieldNames;
+    private final List<CQL3Type.Raw> rawFieldTypes;
+    private final boolean ifNotExists;
+
+    public CreateTypeStatement(String keyspaceName,
+                               String typeName,
+                               List<FieldIdentifier> fieldNames,
+                               List<CQL3Type.Raw> rawFieldTypes,
+                               boolean ifNotExists)
+    {
+        super(keyspaceName);
+        this.typeName = typeName;
+        this.fieldNames = fieldNames;
+        this.rawFieldTypes = rawFieldTypes;
+        this.ifNotExists = ifNotExists;
+    }
+
+    public Keyspaces apply(Keyspaces schema)
+    {
+        KeyspaceMetadata keyspace = schema.getNullable(keyspaceName);
+        if (null == keyspace)
+            throw ire("Keyspace '%s' doesn't exist", keyspaceName);
+
+        UserType existingType = keyspace.types.getNullable(bytes(typeName));
+        if (null != existingType)
+        {
+            if (ifNotExists)
+                return schema;
+
+            throw ire("A user type with name '%s' already exists", typeName);
+        }
+
+        Set<FieldIdentifier> usedNames = new HashSet<>();
+        for (FieldIdentifier name : fieldNames)
+            if (!usedNames.add(name))
+                throw ire("Duplicate field name '%s' in type '%s'", name, typeName);
+
+        for (CQL3Type.Raw type : rawFieldTypes)
+        {
+            if (type.isCounter())
+                throw ire("A user type cannot contain counters");
+
+            if (type.isUDT() && !type.isFrozen())
+                throw ire("A user type cannot contain non-frozen UDTs");
+        }
+
+        List<AbstractType<?>> fieldTypes =
+            rawFieldTypes.stream()
+                         .map(t -> t.prepare(keyspaceName, keyspace.types).getType())
+                         .collect(toList());
+
+        UserType udt = new UserType(keyspaceName, bytes(typeName), fieldNames, fieldTypes, true);
+        return schema.withAddedOrUpdated(keyspace.withSwapped(keyspace.types.with(udt)));
+    }
+
+    SchemaChange schemaChangeEvent(KeyspacesDiff diff)
+    {
+        return new SchemaChange(Change.CREATED, Target.TYPE, keyspaceName, typeName);
+    }
+
+    public void authorize(ClientState client)
+    {
+        client.ensureKeyspacePermission(keyspaceName, Permission.CREATE);
+    }
+
+    @Override
+    public AuditLogContext getAuditLogContext()
+    {
+        return new AuditLogContext(AuditLogEntryType.CREATE_TYPE, keyspaceName, typeName);
+    }
+
+    public static final class Raw extends CQLStatement.Raw
+    {
+        private final UTName name;
+        private final boolean ifNotExists;
+
+        private final List<FieldIdentifier> fieldNames = new ArrayList<>();
+        private final List<CQL3Type.Raw> rawFieldTypes = new ArrayList<>();
+
+        public Raw(UTName name, boolean ifNotExists)
+        {
+            this.name = name;
+            this.ifNotExists = ifNotExists;
+        }
+
+        public CreateTypeStatement prepare(ClientState state)
+        {
+            String keyspaceName = name.hasKeyspace() ? name.getKeyspace() : state.getKeyspace();
+            return new CreateTypeStatement(keyspaceName, name.getStringTypeName(), fieldNames, rawFieldTypes, ifNotExists);
+        }
+
+        public void addField(FieldIdentifier name, CQL3Type.Raw type)
+        {
+            fieldNames.add(name);
+            rawFieldTypes.add(type);
+        }
+
+        public void addToRawBuilder(Types.RawBuilder builder)
+        {
+            builder.add(name.getStringTypeName(),
+                        fieldNames.stream().map(FieldIdentifier::toString).collect(toList()),
+                        rawFieldTypes.stream().map(CQL3Type.Raw::toString).collect(toList()));
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/cql3/statements/schema/CreateViewStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/schema/CreateViewStatement.java b/src/java/org/apache/cassandra/cql3/statements/schema/CreateViewStatement.java
new file mode 100644
index 0000000..f97b0fe
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/statements/schema/CreateViewStatement.java
@@ -0,0 +1,413 @@
+/*
+ * 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.statements.schema;
+
+import java.util.*;
+
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Lists;
+
+import org.apache.cassandra.audit.AuditLogContext;
+import org.apache.cassandra.audit.AuditLogEntryType;
+import org.apache.cassandra.auth.Permission;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.cql3.*;
+import org.apache.cassandra.cql3.restrictions.StatementRestrictions;
+import org.apache.cassandra.cql3.selection.RawSelector;
+import org.apache.cassandra.cql3.selection.Selectable;
+import org.apache.cassandra.cql3.statements.StatementType;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.ReversedType;
+import org.apache.cassandra.exceptions.AlreadyExistsException;
+import org.apache.cassandra.schema.*;
+import org.apache.cassandra.schema.Keyspaces.KeyspacesDiff;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.transport.Event.SchemaChange;
+import org.apache.cassandra.transport.Event.SchemaChange.Change;
+import org.apache.cassandra.transport.Event.SchemaChange.Target;
+
+import static java.lang.String.join;
+
+import static com.google.common.collect.Iterables.concat;
+import static com.google.common.collect.Iterables.filter;
+import static com.google.common.collect.Iterables.transform;
+
+public final class CreateViewStatement extends AlterSchemaStatement
+{
+    private final String tableName;
+    private final String viewName;
+
+    private final List<RawSelector> rawColumns;
+    private final List<ColumnIdentifier> partitionKeyColumns;
+    private final List<ColumnIdentifier> clusteringColumns;
+
+    private final WhereClause whereClause;
+
+    private final LinkedHashMap<ColumnIdentifier, Boolean> clusteringOrder;
+    private final TableAttributes attrs;
+
+    private final boolean ifNotExists;
+
+    public CreateViewStatement(String keyspaceName,
+                               String tableName,
+                               String viewName,
+
+                               List<RawSelector> rawColumns,
+                               List<ColumnIdentifier> partitionKeyColumns,
+                               List<ColumnIdentifier> clusteringColumns,
+
+                               WhereClause whereClause,
+
+                               LinkedHashMap<ColumnIdentifier, Boolean> clusteringOrder,
+                               TableAttributes attrs,
+
+                               boolean ifNotExists)
+    {
+        super(keyspaceName);
+        this.tableName = tableName;
+        this.viewName = viewName;
+
+        this.rawColumns = rawColumns;
+        this.partitionKeyColumns = partitionKeyColumns;
+        this.clusteringColumns = clusteringColumns;
+
+        this.whereClause = whereClause;
+
+        this.clusteringOrder = clusteringOrder;
+        this.attrs = attrs;
+
+        this.ifNotExists = ifNotExists;
+    }
+
+    public Keyspaces apply(Keyspaces schema)
+    {
+        if (!DatabaseDescriptor.enableMaterializedViews())
+            throw ire("Materialized views are disabled. Enable in cassandra.yaml to use.");
+
+        /*
+         * Basic dependency validations
+         */
+
+        KeyspaceMetadata keyspace = schema.getNullable(keyspaceName);
+        if (null == keyspace)
+            throw ire("Keyspace '%s' doesn't exist", keyspaceName);
+
+        TableMetadata table = keyspace.tables.getNullable(tableName);
+        if (null == table)
+            throw ire("Base table '%s' doesn't exist", tableName);
+
+        if (keyspace.hasTable(viewName))
+            throw ire("Cannot create materialized view '%s' - a table with the same name already exists", viewName);
+
+        if (keyspace.hasView(viewName))
+        {
+            if (ifNotExists)
+                return schema;
+
+            throw new AlreadyExistsException(keyspaceName, viewName);
+        }
+
+        /*
+         * Base table validation
+         */
+
+        if (table.isCounter())
+            throw ire("Materialized views are not supported on counter tables");
+
+        if (table.isView())
+            throw ire("Materialized views cannot be created against other materialized views");
+
+        if (table.params.gcGraceSeconds == 0)
+        {
+            throw ire("Cannot create materialized view '%s' for base table " +
+                      "'%s' with gc_grace_seconds of 0, since this value is " +
+                      "used to TTL undelivered updates. Setting gc_grace_seconds" +
+                      " too low might cause undelivered updates to expire " +
+                      "before being replayed.",
+                      viewName, tableName);
+        }
+
+        /*
+         * Process SELECT clause
+         */
+
+        Set<ColumnIdentifier> selectedColumns = new HashSet<>();
+
+        if (rawColumns.isEmpty()) // SELECT *
+            table.columns().forEach(c -> selectedColumns.add(c.name));
+
+        rawColumns.forEach(selector ->
+        {
+            if (null != selector.alias)
+                throw ire("Cannot use aliases when defining a materialized view (got %s)", selector);
+
+            if (!(selector.selectable instanceof Selectable.RawIdentifier))
+                throw ire("Can only select columns by name when defining a materialized view (got %s)", selector.selectable);
+
+            // will throw IRE if the column doesn't exist in the base table
+            ColumnMetadata column = (ColumnMetadata) selector.selectable.prepare(table);
+
+            selectedColumns.add(column.name);
+        });
+
+        selectedColumns.stream()
+                       .map(table::getColumn)
+                       .filter(ColumnMetadata::isStatic)
+                       .findAny()
+                       .ifPresent(c -> { throw ire("Cannot include static column '%s' in materialized view '%s'", c, viewName); });
+
+        /*
+         * Process PRIMARY KEY columns and CLUSTERING ORDER BY clause
+         */
+
+        if (partitionKeyColumns.isEmpty())
+            throw ire("Must provide at least one partition key column for materialized view '%s'", viewName);
+
+        HashSet<ColumnIdentifier> primaryKeyColumns = new HashSet<>();
+
+        concat(partitionKeyColumns, clusteringColumns).forEach(name ->
+        {
+            ColumnMetadata column = table.getColumn(name);
+            if (null == column || !selectedColumns.contains(name))
+                throw ire("Unknown column '%s' referenced in PRIMARY KEY for materialized view '%s'", name, viewName);
+
+            if (!primaryKeyColumns.add(name))
+                throw ire("Duplicate column '%s' in PRIMARY KEY clause for materialized view '%s'", name, viewName);
+
+            AbstractType<?> type = column.type;
+
+            if (type.isMultiCell())
+            {
+                if (type.isCollection())
+                    throw ire("Invalid non-frozen collection type '%s' for PRIMARY KEY column '%s'", type, name);
+                else
+                    throw ire("Invalid non-frozen user-defined type '%s' for PRIMARY KEY column '%s'", type, name);
+            }
+
+            if (type.isCounter())
+                throw ire("counter type is not supported for PRIMARY KEY column '%s'", name);
+
+            if (type.referencesDuration())
+                throw ire("duration type is not supported for PRIMARY KEY column '%s'", name);
+        });
+
+        // If we give a clustering order, we must explicitly do so for all aliases and in the order of the PK
+        if (!clusteringOrder.isEmpty() && !clusteringColumns.equals(new ArrayList<>(clusteringOrder.keySet())))
+            throw ire("Clustering key columns must exactly match columns in CLUSTERING ORDER BY directive");
+
+        /*
+         * We need to include all of the primary key columns from the base table in order to make sure that we do not
+         * overwrite values in the view. We cannot support "collapsing" the base table into a smaller number of rows in
+         * the view because if we need to generate a tombstone, we have no way of knowing which value is currently being
+         * used in the view and whether or not to generate a tombstone. In order to not surprise our users, we require
+         * that they include all of the columns. We provide them with a list of all of the columns left to include.
+         */
+        List<ColumnIdentifier> missingPrimaryKeyColumns =
+            Lists.newArrayList(filter(transform(table.primaryKeyColumns(), c -> c.name), c -> !primaryKeyColumns.contains(c)));
+
+        if (!missingPrimaryKeyColumns.isEmpty())
+        {
+            throw ire("Cannot create materialized view '%s' without primary key columns %s from base table '%s'",
+                      viewName, join(", ", transform(missingPrimaryKeyColumns, ColumnIdentifier::toString)), tableName);
+        }
+
+        Set<ColumnIdentifier> regularBaseTableColumnsInViewPrimaryKey = new HashSet<>(primaryKeyColumns);
+        transform(table.primaryKeyColumns(), c -> c.name).forEach(regularBaseTableColumnsInViewPrimaryKey::remove);
+        if (regularBaseTableColumnsInViewPrimaryKey.size() > 1)
+        {
+            throw ire("Cannot include more than one non-primary key column in materialized view primary key (got %s)",
+                      join(", ", transform(regularBaseTableColumnsInViewPrimaryKey, ColumnIdentifier::toString)));
+        }
+
+        /*
+         * Process WHERE clause
+         */
+
+        if (whereClause.containsCustomExpressions())
+            throw ire("WHERE clause for materialized view '%s' cannot contain custom index expressions", viewName);
+
+        StatementRestrictions restrictions =
+            new StatementRestrictions(StatementType.SELECT,
+                                      table,
+                                      whereClause,
+                                      VariableSpecifications.empty(),
+                                      false,
+                                      false,
+                                      true,
+                                      true);
+
+        List<ColumnIdentifier> nonRestrictedPrimaryKeyColumns =
+            Lists.newArrayList(filter(primaryKeyColumns, name -> !restrictions.isRestricted(table.getColumn(name))));
+
+        if (!nonRestrictedPrimaryKeyColumns.isEmpty())
+        {
+            throw ire("Primary key columns %s must be restricted with 'IS NOT NULL' or otherwise",
+                      join(", ", transform(nonRestrictedPrimaryKeyColumns, ColumnIdentifier::toString)));
+        }
+
+        // See CASSANDRA-13798
+        Set<ColumnMetadata> restrictedNonPrimaryKeyColumns = restrictions.nonPKRestrictedColumns(false);
+        if (!restrictedNonPrimaryKeyColumns.isEmpty() && !Boolean.getBoolean("cassandra.mv.allow_filtering_nonkey_columns_unsafe"))
+        {
+            throw ire("Non-primary key columns can only be restricted with 'IS NOT NULL' (got: %s restricted illegally)",
+                      join(",", transform(restrictedNonPrimaryKeyColumns, ColumnMetadata::toString)));
+        }
+
+        /*
+         * Validate WITH params
+         */
+
+        attrs.validate();
+
+        if (attrs.hasOption(TableParams.Option.DEFAULT_TIME_TO_LIVE))
+        {
+            throw ire("Cannot set default_time_to_live for a materialized view. " +
+                      "Data in a materialized view always expire at the same time than " +
+                      "the corresponding data in the parent table.");
+        }
+
+        /*
+         * Build the thing
+         */
+
+        TableMetadata.Builder builder = TableMetadata.builder(keyspaceName, viewName);
+
+        if (attrs.hasProperty(TableAttributes.ID))
+            builder.id(attrs.getId());
+
+        builder.params(attrs.asNewTableParams())
+               .kind(TableMetadata.Kind.VIEW);
+
+        partitionKeyColumns.forEach(name -> builder.addPartitionKeyColumn(name, getType(table, name)));
+        clusteringColumns.forEach(name -> builder.addClusteringColumn(name, getType(table, name)));
+
+        selectedColumns.stream()
+                       .filter(name -> !primaryKeyColumns.contains(name))
+                       .forEach(name -> builder.addRegularColumn(name, getType(table, name)));
+
+        ViewMetadata view = new ViewMetadata(table.id, table.name, rawColumns.isEmpty(), whereClause, builder.build());
+        view.metadata.validate();
+
+        return schema.withAddedOrUpdated(keyspace.withSwapped(keyspace.views.with(view)));
+    }
+
+    SchemaChange schemaChangeEvent(KeyspacesDiff diff)
+    {
+        return new SchemaChange(Change.CREATED, Target.TABLE, keyspaceName, viewName);
+    }
+
+    public void authorize(ClientState client)
+    {
+        client.ensureTablePermission(keyspaceName, tableName, Permission.ALTER);
+    }
+
+    private AbstractType<?> getType(TableMetadata table, ColumnIdentifier name)
+    {
+        AbstractType<?> type = table.getColumn(name).type;
+        boolean reverse = !clusteringOrder.getOrDefault(name, true);
+
+        if (type.isReversed() && !reverse)
+            return ((ReversedType) type).baseType;
+        else if (!type.isReversed() && reverse)
+            return ReversedType.getInstance(type);
+        else
+            return type;
+    }
+
+    @Override
+    Set<String> clientWarnings(KeyspacesDiff diff)
+    {
+        return ImmutableSet.of("Materialized views are experimental and are not recommended for production use.");
+    }
+
+    @Override
+    public AuditLogContext getAuditLogContext()
+    {
+        return new AuditLogContext(AuditLogEntryType.CREATE_VIEW, keyspaceName, viewName);
+    }
+
+    public final static class Raw extends CQLStatement.Raw
+    {
+        private final QualifiedName tableName;
+        private final QualifiedName viewName;
+        private final boolean ifNotExists;
+
+        private final List<RawSelector> rawColumns;
+        private final List<ColumnIdentifier> clusteringColumns = new ArrayList<>();
+        private List<ColumnIdentifier> partitionKeyColumns;
+
+        private final WhereClause whereClause;
+
+        private final LinkedHashMap<ColumnIdentifier, Boolean> clusteringOrder = new LinkedHashMap<>();
+        public final TableAttributes attrs = new TableAttributes();
+
+        public Raw(QualifiedName tableName, QualifiedName viewName, List<RawSelector> rawColumns, WhereClause whereClause, boolean ifNotExists)
+        {
+            this.tableName = tableName;
+            this.viewName = viewName;
+            this.rawColumns = rawColumns;
+            this.whereClause = whereClause;
+            this.ifNotExists = ifNotExists;
+        }
+
+        public CreateViewStatement prepare(ClientState state)
+        {
+            String keyspaceName = viewName.hasKeyspace() ? viewName.getKeyspace() : state.getKeyspace();
+
+            if (tableName.hasKeyspace() && !keyspaceName.equals(tableName.getKeyspace()))
+                throw ire("Cannot create a materialized view on a table in a different keyspace");
+
+            if (!bindVariables.isEmpty())
+                throw ire("Bind variables are not allowed in CREATE MATERIALIZED VIEW statements");
+
+            if (null == partitionKeyColumns)
+                throw ire("No PRIMARY KEY specifed for view '%s' (exactly one required)", viewName);
+
+            return new CreateViewStatement(keyspaceName,
+                                           tableName.getName(),
+                                           viewName.getName(),
+
+                                           rawColumns,
+                                           partitionKeyColumns,
+                                           clusteringColumns,
+
+                                           whereClause,
+
+                                           clusteringOrder,
+                                           attrs,
+
+                                           ifNotExists);
+        }
+
+        public void setPartitionKeyColumns(List<ColumnIdentifier> columns)
+        {
+            partitionKeyColumns = columns;
+        }
+
+        public void markClusteringColumn(ColumnIdentifier column)
+        {
+            clusteringColumns.add(column);
+        }
+
+        public void extendClusteringOrder(ColumnIdentifier column, boolean ascending)
+        {
+            if (null != clusteringOrder.put(column, ascending))
+                throw ire("Duplicate column '%s' in CLUSTERING ORDER BY clause for view '%s'", column, viewName);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/cql3/statements/schema/DropAggregateStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/schema/DropAggregateStatement.java b/src/java/org/apache/cassandra/cql3/statements/schema/DropAggregateStatement.java
new file mode 100644
index 0000000..564f267
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/statements/schema/DropAggregateStatement.java
@@ -0,0 +1,174 @@
+/*
+ * 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.statements.schema;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.function.Predicate;
+import java.util.stream.Stream;
+
+import org.apache.cassandra.audit.AuditLogContext;
+import org.apache.cassandra.audit.AuditLogEntryType;
+import org.apache.cassandra.auth.FunctionResource;
+import org.apache.cassandra.auth.Permission;
+import org.apache.cassandra.cql3.CQL3Type;
+import org.apache.cassandra.cql3.CQLStatement;
+import org.apache.cassandra.cql3.functions.Function;
+import org.apache.cassandra.cql3.functions.FunctionName;
+import org.apache.cassandra.cql3.functions.UDAggregate;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.schema.*;
+import org.apache.cassandra.schema.Keyspaces.KeyspacesDiff;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.transport.Event.SchemaChange;
+import org.apache.cassandra.transport.Event.SchemaChange.Change;
+
+import static java.lang.String.format;
+import static java.lang.String.join;
+import static java.util.stream.Collectors.toList;
+
+import static com.google.common.collect.Iterables.transform;
+
+public final class DropAggregateStatement extends AlterSchemaStatement
+{
+    private final String aggregateName;
+    private final List<CQL3Type.Raw> arguments;
+    private final boolean argumentsSpeficied;
+    private final boolean ifExists;
+
+    public DropAggregateStatement(String keyspaceName,
+                                  String aggregateName,
+                                  List<CQL3Type.Raw> arguments,
+                                  boolean argumentsSpeficied,
+                                  boolean ifExists)
+    {
+        super(keyspaceName);
+        this.aggregateName = aggregateName;
+        this.arguments = arguments;
+        this.argumentsSpeficied = argumentsSpeficied;
+        this.ifExists = ifExists;
+    }
+
+    public Keyspaces apply(Keyspaces schema)
+    {
+        String name =
+            argumentsSpeficied
+          ? format("%s.%s(%s)", keyspaceName, aggregateName, join(", ", transform(arguments, CQL3Type.Raw::toString)))
+          : format("%s.%s", keyspaceName, aggregateName);
+
+        KeyspaceMetadata keyspace = schema.getNullable(keyspaceName);
+        if (null == keyspace)
+        {
+            if (ifExists)
+                return schema;
+
+            throw ire("Aggregate '%s' doesn't exist", name);
+        }
+
+        Collection<Function> aggregates = keyspace.functions.get(new FunctionName(keyspaceName, aggregateName));
+        if (aggregates.size() > 1 && !argumentsSpeficied)
+        {
+            throw ire("'DROP AGGREGATE %s' matches multiple function definitions; " +
+                      "specify the argument types by issuing a statement like " +
+                      "'DROP AGGREGATE %s (type, type, ...)'. You can use cqlsh " +
+                      "'DESCRIBE AGGREGATE %s' command to find all overloads",
+                      aggregateName, aggregateName, aggregateName);
+        }
+
+        arguments.stream()
+                 .filter(CQL3Type.Raw::isFrozen)
+                 .findFirst()
+                 .ifPresent(t -> { throw ire("Argument '%s' cannot be frozen; remove frozen<> modifier from '%s'", t, t); });
+
+        List<AbstractType<?>> argumentTypes = prepareArgumentTypes(keyspace.types);
+
+        Predicate<Function> filter = Functions.Filter.UDA;
+        if (argumentsSpeficied)
+            filter = filter.and(f -> Functions.typesMatch(f.argTypes(), argumentTypes));
+
+        Function aggregate = aggregates.stream().filter(filter).findAny().orElse(null);
+        if (null == aggregate)
+        {
+            if (ifExists)
+                return schema;
+
+            throw ire("Aggregate '%s' doesn't exist", name);
+        }
+
+        return schema.withAddedOrUpdated(keyspace.withSwapped(keyspace.functions.without(aggregate)));
+    }
+
+    SchemaChange schemaChangeEvent(KeyspacesDiff diff)
+    {
+        Functions dropped = diff.altered.get(0).udas.dropped;
+        assert dropped.size() == 1;
+        return SchemaChange.forAggregate(Change.DROPPED, (UDAggregate) dropped.iterator().next());
+    }
+
+    public void authorize(ClientState client)
+    {
+        KeyspaceMetadata keyspace = Schema.instance.getKeyspaceMetadata(keyspaceName);
+        if (null == keyspace)
+            return;
+
+        Stream<Function> functions = keyspace.functions.get(new FunctionName(keyspaceName, aggregateName)).stream();
+        if (argumentsSpeficied)
+            functions = functions.filter(f -> Functions.typesMatch(f.argTypes(), prepareArgumentTypes(keyspace.types)));
+
+        functions.forEach(f -> client.ensurePermission(Permission.DROP, FunctionResource.function(f)));
+    }
+
+    @Override
+    public AuditLogContext getAuditLogContext()
+    {
+        return new AuditLogContext(AuditLogEntryType.DROP_AGGREGATE, keyspaceName, aggregateName);
+    }
+
+    private List<AbstractType<?>> prepareArgumentTypes(Types types)
+    {
+        return arguments.stream()
+                        .map(t -> t.prepare(keyspaceName, types))
+                        .map(CQL3Type::getType)
+                        .collect(toList());
+    }
+
+    public static final class Raw extends CQLStatement.Raw
+    {
+        private final FunctionName name;
+        private final List<CQL3Type.Raw> arguments;
+        private final boolean argumentsSpecified;
+        private final boolean ifExists;
+
+        public Raw(FunctionName name,
+                   List<CQL3Type.Raw> arguments,
+                   boolean argumentsSpecified,
+                   boolean ifExists)
+        {
+            this.name = name;
+            this.arguments = arguments;
+            this.argumentsSpecified = argumentsSpecified;
+            this.ifExists = ifExists;
+        }
+
+        public DropAggregateStatement prepare(ClientState state)
+        {
+            String keyspaceName = name.hasKeyspace() ? name.keyspace : state.getKeyspace();
+            return new DropAggregateStatement(keyspaceName, name.name, arguments, argumentsSpecified, ifExists);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/cql3/statements/schema/DropFunctionStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/schema/DropFunctionStatement.java b/src/java/org/apache/cassandra/cql3/statements/schema/DropFunctionStatement.java
new file mode 100644
index 0000000..9433833
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/statements/schema/DropFunctionStatement.java
@@ -0,0 +1,182 @@
+/*
+ * 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.statements.schema;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.function.Predicate;
+import java.util.stream.Stream;
+
+import org.apache.cassandra.audit.AuditLogContext;
+import org.apache.cassandra.audit.AuditLogEntryType;
+import org.apache.cassandra.auth.FunctionResource;
+import org.apache.cassandra.auth.Permission;
+import org.apache.cassandra.cql3.CQL3Type;
+import org.apache.cassandra.cql3.CQLStatement;
+import org.apache.cassandra.cql3.functions.*;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.schema.*;
+import org.apache.cassandra.schema.Keyspaces.KeyspacesDiff;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.transport.Event.SchemaChange;
+import org.apache.cassandra.transport.Event.SchemaChange.Change;
+
+import static java.lang.String.format;
+import static java.lang.String.join;
+import static java.util.stream.Collectors.joining;
+import static java.util.stream.Collectors.toList;
+
+import static com.google.common.collect.Iterables.transform;
+
+public final class DropFunctionStatement extends AlterSchemaStatement
+{
+    private final String functionName;
+    private final Collection<CQL3Type.Raw> arguments;
+    private final boolean argumentsSpeficied;
+    private final boolean ifExists;
+
+    public DropFunctionStatement(String keyspaceName,
+                                 String functionName,
+                                 Collection<CQL3Type.Raw> arguments,
+                                 boolean argumentsSpeficied,
+                                 boolean ifExists)
+    {
+        super(keyspaceName);
+        this.functionName = functionName;
+        this.arguments = arguments;
+        this.argumentsSpeficied = argumentsSpeficied;
+        this.ifExists = ifExists;
+    }
+
+    public Keyspaces apply(Keyspaces schema)
+    {
+        String name =
+            argumentsSpeficied
+          ? format("%s.%s(%s)", keyspaceName, functionName, join(", ", transform(arguments, CQL3Type.Raw::toString)))
+          : format("%s.%s", keyspaceName, functionName);
+
+        KeyspaceMetadata keyspace = schema.getNullable(keyspaceName);
+        if (null == keyspace)
+        {
+            if (ifExists)
+                return schema;
+
+            throw ire("Function '%s' doesn't exist", name);
+        }
+
+        Collection<Function> functions = keyspace.functions.get(new FunctionName(keyspaceName, functionName));
+        if (functions.size() > 1 && !argumentsSpeficied)
+        {
+            throw ire("'DROP FUNCTION %s' matches multiple function definitions; " +
+                      "specify the argument types by issuing a statement like " +
+                      "'DROP FUNCTION %s (type, type, ...)'. You can use cqlsh " +
+                      "'DESCRIBE FUNCTION %s' command to find all overloads",
+                      functionName, functionName, functionName);
+        }
+
+        arguments.stream()
+                 .filter(CQL3Type.Raw::isFrozen)
+                 .findFirst()
+                 .ifPresent(t -> { throw ire("Argument '%s' cannot be frozen; remove frozen<> modifier from '%s'", t, t); });
+
+        List<AbstractType<?>> argumentTypes = prepareArgumentTypes(keyspace.types);
+
+        Predicate<Function> filter = Functions.Filter.UDF;
+        if (argumentsSpeficied)
+            filter = filter.and(f -> Functions.typesMatch(f.argTypes(), argumentTypes));
+
+        Function function = functions.stream().filter(filter).findAny().orElse(null);
+        if (null == function)
+        {
+            if (ifExists)
+                return schema;
+
+            throw ire("Function '%s' doesn't exist", name);
+        }
+
+        String dependentAggregates =
+            keyspace.functions
+                    .aggregatesUsingFunction(function)
+                    .map(a -> a.name().toString())
+                    .collect(joining(", "));
+
+        if (!dependentAggregates.isEmpty())
+            throw ire("Function '%s' is still referenced by aggregates %s", name, dependentAggregates);
+
+        return schema.withAddedOrUpdated(keyspace.withSwapped(keyspace.functions.without(function)));
+    }
+
+    SchemaChange schemaChangeEvent(KeyspacesDiff diff)
+    {
+        Functions dropped = diff.altered.get(0).udfs.dropped;
+        assert dropped.size() == 1;
+        return SchemaChange.forFunction(Change.DROPPED, (UDFunction) dropped.iterator().next());
+    }
+
+    public void authorize(ClientState client)
+    {
+        KeyspaceMetadata keyspace = Schema.instance.getKeyspaceMetadata(keyspaceName);
+        if (null == keyspace)
+            return;
+
+        Stream<Function> functions = keyspace.functions.get(new FunctionName(keyspaceName, functionName)).stream();
+        if (argumentsSpeficied)
+            functions = functions.filter(f -> Functions.typesMatch(f.argTypes(), prepareArgumentTypes(keyspace.types)));
+
+        functions.forEach(f -> client.ensurePermission(Permission.DROP, FunctionResource.function(f)));
+    }
+
+    @Override
+    public AuditLogContext getAuditLogContext()
+    {
+        return new AuditLogContext(AuditLogEntryType.DROP_FUNCTION, keyspaceName, functionName);
+    }
+
+    private List<AbstractType<?>> prepareArgumentTypes(Types types)
+    {
+        return arguments.stream()
+                        .map(t -> t.prepare(keyspaceName, types))
+                        .map(CQL3Type::getType)
+                        .collect(toList());
+    }
+
+    public static final class Raw extends CQLStatement.Raw
+    {
+        private final FunctionName name;
+        private final List<CQL3Type.Raw> arguments;
+        private final boolean argumentsSpecified;
+        private final boolean ifExists;
+
+        public Raw(FunctionName name,
+                   List<CQL3Type.Raw> arguments,
+                   boolean argumentsSpecified,
+                   boolean ifExists)
+        {
+            this.name = name;
+            this.arguments = arguments;
+            this.argumentsSpecified = argumentsSpecified;
+            this.ifExists = ifExists;
+        }
+
+        public DropFunctionStatement prepare(ClientState state)
+        {
+            String keyspaceName = name.hasKeyspace() ? name.keyspace : state.getKeyspace();
+            return new DropFunctionStatement(keyspaceName, name.name, arguments, argumentsSpecified, ifExists);
+        }
+    }
+}


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


[05/13] cassandra git commit: Make all DDL statements idempotent and not dependent on global state

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/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 f6545b0..40f1b84 100644
--- a/src/java/org/apache/cassandra/db/view/View.java
+++ b/src/java/org/apache/cassandra/db/view/View.java
@@ -25,7 +25,7 @@ import javax.annotation.Nullable;
 import com.google.common.collect.Iterables;
 
 import org.apache.cassandra.cql3.*;
-import org.apache.cassandra.cql3.statements.ParsedStatement;
+import org.apache.cassandra.cql3.selection.RawSelector;
 import org.apache.cassandra.cql3.statements.SelectStatement;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.rows.*;
@@ -34,7 +34,6 @@ import org.apache.cassandra.schema.KeyspaceMetadata;
 import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.schema.TableMetadataRef;
 import org.apache.cassandra.schema.ViewMetadata;
-import org.apache.cassandra.service.ClientState;
 import org.apache.cassandra.utils.FBUtilities;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -56,19 +55,13 @@ public class View
     public volatile List<ColumnMetadata> baseNonPKColumnsInViewPK;
     private ViewBuilder builder;
 
-    // Only the raw statement can be final, because the statement cannot always be prepared when the MV is initialized.
-    // For example, during startup, this view will be initialized as part of the Keyspace.open() work; preparing a statement
-    // also requires the keyspace to be open, so this results in double-initialization problems.
-    private final SelectStatement.RawStatement rawSelect;
     private SelectStatement select;
     private ReadQuery query;
 
-    public View(ViewMetadata definition,
-                ColumnFamilyStore baseCfs)
+    public View(ViewMetadata definition, ColumnFamilyStore baseCfs)
     {
         this.baseCfs = baseCfs;
-        this.name = definition.name;
-        this.rawSelect = definition.select;
+        this.name = definition.name();
 
         updateDefinition(definition);
     }
@@ -160,31 +153,52 @@ public class View
      * Returns the SelectStatement used to populate and filter this view.  Internal users should access the select
      * statement this way to ensure it has been prepared.
      */
-    public SelectStatement getSelectStatement()
+    SelectStatement getSelectStatement()
     {
-        if (select == null)
+        if (null == select)
         {
-            ClientState state = ClientState.forInternalCalls();
-            state.setKeyspace(baseCfs.keyspace.getName());
-            rawSelect.prepareKeyspace(state);
-            ParsedStatement.Prepared prepared = rawSelect.prepare(true);
-            select = (SelectStatement) prepared.statement;
+            SelectStatement.Parameters parameters =
+                new SelectStatement.Parameters(Collections.emptyMap(),
+                                               Collections.emptyList(),
+                                               false,
+                                               true,
+                                               false);
+
+            SelectStatement.RawStatement rawSelect =
+                new SelectStatement.RawStatement(new QualifiedName(baseCfs.keyspace.getName(), baseCfs.name),
+                                                 parameters,
+                                                 selectClause(),
+                                                 definition.whereClause,
+                                                 null,
+                                                 null);
+
+            rawSelect.setBindVariables(Collections.emptyList());
+
+            select = rawSelect.prepare(true);
         }
 
         return select;
     }
 
+    private List<RawSelector> selectClause()
+    {
+        return definition.metadata
+                         .columns()
+                         .stream()
+                         .map(c -> c.name.toString())
+                         .map(ColumnMetadata.Raw::forQuoted)
+                         .map(c -> new RawSelector(c, null))
+                         .collect(Collectors.toList());
+    }
+
     /**
      * Returns the ReadQuery used to filter this view.  Internal users should access the query this way to ensure it
      * has been prepared.
      */
-    public ReadQuery getReadQuery()
+    ReadQuery getReadQuery()
     {
         if (query == null)
-        {
             query = getSelectStatement().getQuery(QueryOptions.forInternalCalls(Collections.emptyList()), FBUtilities.nowInSeconds());
-            logger.trace("View query: {}", rawSelect);
-        }
 
         return query;
     }
@@ -216,63 +230,13 @@ public class View
         return (view == null) ? null : Schema.instance.getTableMetadataRef(view.baseTableId);
     }
 
+    // TODO: REMOVE
     public static Iterable<ViewMetadata> findAll(String keyspace, String baseTable)
     {
         KeyspaceMetadata ksm = Schema.instance.getKeyspaceMetadata(keyspace);
         return Iterables.filter(ksm.views, view -> view.baseTableName.equals(baseTable));
     }
 
-    /**
-     * Builds the string text for a materialized view's SELECT statement.
-     */
-    public static String buildSelectStatement(String cfName, Collection<ColumnMetadata> includedColumns, String whereClause)
-    {
-         StringBuilder rawSelect = new StringBuilder("SELECT ");
-        if (includedColumns == null || includedColumns.isEmpty())
-            rawSelect.append("*");
-        else
-            rawSelect.append(includedColumns.stream().map(id -> id.name.toCQLString()).collect(Collectors.joining(", ")));
-        rawSelect.append(" FROM \"").append(cfName).append("\" WHERE ") .append(whereClause).append(" ALLOW FILTERING");
-        return rawSelect.toString();
-    }
-
-    public static String relationsToWhereClause(List<Relation> whereClause)
-    {
-        List<String> expressions = new ArrayList<>(whereClause.size());
-        for (Relation rel : whereClause)
-        {
-            StringBuilder sb = new StringBuilder();
-
-            if (rel.isMultiColumn())
-            {
-                sb.append(((MultiColumnRelation) rel).getEntities().stream()
-                        .map(ColumnMetadata.Raw::toString)
-                        .collect(Collectors.joining(", ", "(", ")")));
-            }
-            else
-            {
-                sb.append(((SingleColumnRelation) rel).getEntity());
-            }
-
-            sb.append(" ").append(rel.operator()).append(" ");
-
-            if (rel.isIN())
-            {
-                sb.append(rel.getInValues().stream()
-                        .map(Term.Raw::getText)
-                        .collect(Collectors.joining(", ", "(", ")")));
-            }
-            else
-            {
-                sb.append(rel.getValue().getText());
-            }
-
-            expressions.add(sb.toString());
-        }
-
-        return expressions.stream().collect(Collectors.joining(" AND "));
-    }
-
     public boolean hasSamePrimaryKeyColumnsAsBaseTable()
     {
         return baseNonPKColumnsInViewPK.isEmpty();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/db/view/ViewManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/view/ViewManager.java b/src/java/org/apache/cassandra/db/view/ViewManager.java
index 8d12349..0d565ae 100644
--- a/src/java/org/apache/cassandra/db/view/ViewManager.java
+++ b/src/java/org/apache/cassandra/db/view/ViewManager.java
@@ -101,7 +101,7 @@ public class ViewManager
         Map<String, ViewMetadata> newViewsByName = Maps.newHashMapWithExpectedSize(views.size());
         for (ViewMetadata definition : views)
         {
-            newViewsByName.put(definition.name, definition);
+            newViewsByName.put(definition.name(), definition);
         }
 
         for (String viewName : viewsByName.keySet())
@@ -147,14 +147,14 @@ public class ViewManager
         if (!keyspace.hasColumnFamilyStore(definition.baseTableId))
         {
             logger.warn("Not adding view {} because the base table {} is unknown",
-                        definition.name,
+                        definition.name(),
                         definition.baseTableId);
             return;
         }
 
         View view = new View(definition, keyspace.getColumnFamilyStore(definition.baseTableId));
         forTable(view.getDefinition().baseTableId).add(view);
-        viewsByName.put(definition.name, view);
+        viewsByName.put(definition.name(), view);
     }
 
     public void removeView(String name)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/hadoop/cql3/CqlBulkRecordWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hadoop/cql3/CqlBulkRecordWriter.java b/src/java/org/apache/cassandra/hadoop/cql3/CqlBulkRecordWriter.java
index 204d9ee..4bbb861 100644
--- a/src/java/org/apache/cassandra/hadoop/cql3/CqlBulkRecordWriter.java
+++ b/src/java/org/apache/cassandra/hadoop/cql3/CqlBulkRecordWriter.java
@@ -31,8 +31,8 @@ import com.google.common.net.HostAndPort;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.cassandra.cql3.statements.CreateTableStatement;
 import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.cql3.statements.schema.CreateTableStatement;
 import org.apache.cassandra.dht.IPartitioner;
 import org.apache.cassandra.dht.Murmur3Partitioner;
 import org.apache.cassandra.exceptions.InvalidRequestException;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/index/SecondaryIndexManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/SecondaryIndexManager.java b/src/java/org/apache/cassandra/index/SecondaryIndexManager.java
index 70aebbd..d765fac 100644
--- a/src/java/org/apache/cassandra/index/SecondaryIndexManager.java
+++ b/src/java/org/apache/cassandra/index/SecondaryIndexManager.java
@@ -50,7 +50,7 @@ import org.apache.cassandra.concurrent.JMXEnabledThreadPoolExecutor;
 import org.apache.cassandra.concurrent.NamedThreadFactory;
 import org.apache.cassandra.concurrent.StageManager;
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.cql3.statements.IndexTarget;
+import org.apache.cassandra.cql3.statements.schema.IndexTarget;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.compaction.CompactionManager;
 import org.apache.cassandra.db.filter.RowFilter;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/index/TargetParser.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/TargetParser.java b/src/java/org/apache/cassandra/index/TargetParser.java
index bc679f2..9ada4c6 100644
--- a/src/java/org/apache/cassandra/index/TargetParser.java
+++ b/src/java/org/apache/cassandra/index/TargetParser.java
@@ -25,7 +25,7 @@ import org.apache.commons.lang3.StringUtils;
 import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.cql3.ColumnIdentifier;
-import org.apache.cassandra.cql3.statements.IndexTarget;
+import org.apache.cassandra.cql3.statements.schema.IndexTarget;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.schema.IndexMetadata;
 import org.apache.cassandra.utils.Pair;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/index/internal/CassandraIndex.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/internal/CassandraIndex.java b/src/java/org/apache/cassandra/index/internal/CassandraIndex.java
index fb0d629..76b7543 100644
--- a/src/java/org/apache/cassandra/index/internal/CassandraIndex.java
+++ b/src/java/org/apache/cassandra/index/internal/CassandraIndex.java
@@ -37,7 +37,7 @@ import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.schema.TableMetadataRef;
 import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.cql3.Operator;
-import org.apache.cassandra.cql3.statements.IndexTarget;
+import org.apache.cassandra.cql3.statements.schema.IndexTarget;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.compaction.CompactionManager;
 import org.apache.cassandra.db.filter.RowFilter;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/index/sasi/SASIIndex.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/sasi/SASIIndex.java b/src/java/org/apache/cassandra/index/sasi/SASIIndex.java
index 76e5801..3ffcb6e 100644
--- a/src/java/org/apache/cassandra/index/sasi/SASIIndex.java
+++ b/src/java/org/apache/cassandra/index/sasi/SASIIndex.java
@@ -25,7 +25,7 @@ import com.googlecode.concurrenttrees.common.Iterables;
 
 import org.apache.cassandra.config.*;
 import org.apache.cassandra.cql3.Operator;
-import org.apache.cassandra.cql3.statements.IndexTarget;
+import org.apache.cassandra.cql3.statements.schema.IndexTarget;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.compaction.CompactionManager;
 import org.apache.cassandra.db.compaction.OperationType;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java b/src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java
index 22296e8..0c2cf28 100644
--- a/src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java
@@ -31,16 +31,16 @@ import java.util.stream.Collectors;
 
 import com.datastax.driver.core.ProtocolVersion;
 import com.datastax.driver.core.TypeCodec;
+import org.apache.cassandra.cql3.statements.schema.CreateTableStatement;
+import org.apache.cassandra.cql3.statements.schema.CreateTypeStatement;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.cql3.ColumnSpecification;
 import org.apache.cassandra.cql3.QueryOptions;
 import org.apache.cassandra.cql3.QueryProcessor;
 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.ModificationStatement;
-import org.apache.cassandra.cql3.statements.ParsedStatement;
 import org.apache.cassandra.cql3.statements.UpdateStatement;
 import org.apache.cassandra.db.Clustering;
 import org.apache.cassandra.db.DecoratedKey;
@@ -58,14 +58,12 @@ import org.apache.cassandra.schema.KeyspaceParams;
 import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.schema.SchemaConstants;
 import org.apache.cassandra.schema.SchemaKeyspace;
-import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.schema.TableMetadataRef;
 import org.apache.cassandra.schema.Tables;
 import org.apache.cassandra.schema.Types;
 import org.apache.cassandra.schema.Views;
 import org.apache.cassandra.service.ClientState;
 import org.apache.cassandra.utils.ByteBufferUtil;
-import org.apache.cassandra.utils.Pair;
 
 /**
  * Utility to write SSTables.
@@ -348,8 +346,8 @@ public class CQLSSTableWriter implements Closeable
 
         protected SSTableFormat.Type formatType = null;
 
-        private CreateTableStatement.RawStatement schemaStatement;
-        private final List<CreateTypeStatement> typeStatements;
+        private CreateTableStatement.Raw schemaStatement;
+        private final List<CreateTypeStatement.Raw> typeStatements;
         private ModificationStatement.Parsed insertStatement;
         private IPartitioner partitioner;
 
@@ -398,7 +396,7 @@ public class CQLSSTableWriter implements Closeable
 
         public Builder withType(String typeDefinition) throws SyntaxException
         {
-            typeStatements.add(QueryProcessor.parseStatement(typeDefinition, CreateTypeStatement.class, "CREATE TYPE"));
+            typeStatements.add(QueryProcessor.parseStatement(typeDefinition, CreateTypeStatement.Raw.class, "CREATE TYPE"));
             return this;
         }
 
@@ -418,7 +416,7 @@ public class CQLSSTableWriter implements Closeable
          */
         public Builder forTable(String schema)
         {
-            this.schemaStatement = QueryProcessor.parseStatement(schema, CreateTableStatement.RawStatement.class, "CREATE TABLE");
+            this.schemaStatement = QueryProcessor.parseStatement(schema, CreateTableStatement.Raw.class, "CREATE TABLE");
             return this;
         }
 
@@ -531,10 +529,9 @@ public class CQLSSTableWriter implements Closeable
                                                                  Functions.none()));
                 }
 
-
                 KeyspaceMetadata ksm = Schema.instance.getKeyspaceMetadata(keyspaceName);
 
-                TableMetadata tableMetadata = ksm.tables.getNullable(schemaStatement.columnFamily());
+                TableMetadata tableMetadata = ksm.tables.getNullable(schemaStatement.table());
                 if (tableMetadata == null)
                 {
                     Types types = createTypes(keyspaceName);
@@ -542,24 +539,24 @@ public class CQLSSTableWriter implements Closeable
                     Schema.instance.load(ksm.withSwapped(ksm.tables.with(tableMetadata)).withSwapped(types));
                 }
 
-                Pair<UpdateStatement, List<ColumnSpecification>> preparedInsert = prepareInsert();
+                UpdateStatement preparedInsert = prepareInsert();
 
                 TableMetadataRef ref = TableMetadataRef.forOfflineTools(tableMetadata);
                 AbstractSSTableSimpleWriter writer = sorted
-                                                     ? new SSTableSimpleWriter(directory, ref, preparedInsert.left.updatedColumns())
-                                                     : new SSTableSimpleUnsortedWriter(directory, ref, preparedInsert.left.updatedColumns(), bufferSizeInMB);
+                                                   ? new SSTableSimpleWriter(directory, ref, preparedInsert.updatedColumns())
+                                                   : new SSTableSimpleUnsortedWriter(directory, ref, preparedInsert.updatedColumns(), bufferSizeInMB);
 
                 if (formatType != null)
                     writer.setSSTableFormatType(formatType);
 
-                return new CQLSSTableWriter(writer, preparedInsert.left, preparedInsert.right);
+                return new CQLSSTableWriter(writer, preparedInsert, preparedInsert.getBindVariables());
             }
         }
 
         private Types createTypes(String keyspace)
         {
             Types.RawBuilder builder = Types.rawBuilder(keyspace);
-            for (CreateTypeStatement st : typeStatements)
+            for (CreateTypeStatement.Raw st : typeStatements)
                 st.addToRawBuilder(builder);
             return builder.build();
         }
@@ -571,10 +568,11 @@ public class CQLSSTableWriter implements Closeable
          */
         private TableMetadata createTable(Types types)
         {
-            CreateTableStatement statement = (CreateTableStatement) schemaStatement.prepare(types).statement;
+            ClientState state = ClientState.forInternalCalls();
+            CreateTableStatement statement = schemaStatement.prepare(state);
             statement.validate(ClientState.forInternalCalls());
 
-            TableMetadata.Builder builder = statement.builder();
+            TableMetadata.Builder builder = statement.builder(types);
             if (partitioner != null)
                 builder.partitioner(partitioner);
 
@@ -586,20 +584,20 @@ public class CQLSSTableWriter implements Closeable
          *
          * @return prepared Insert statement and it's bound names
          */
-        private Pair<UpdateStatement, List<ColumnSpecification>> prepareInsert()
+        private UpdateStatement prepareInsert()
         {
-            ParsedStatement.Prepared cqlStatement = insertStatement.prepare();
-            UpdateStatement insert = (UpdateStatement) cqlStatement.statement;
-            insert.validate(ClientState.forInternalCalls());
+            ClientState state = ClientState.forInternalCalls();
+            UpdateStatement insert = (UpdateStatement) insertStatement.prepare(state);
+            insert.validate(state);
 
             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())
+            if (insert.getBindVariables().isEmpty())
                 throw new IllegalArgumentException("Provided insert statement has no bind variables");
 
-            return Pair.create(insert, cqlStatement.boundNames);
+            return insert;
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/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 4097715..90c0146 100644
--- a/src/java/org/apache/cassandra/repair/RepairRunnable.java
+++ b/src/java/org/apache/cassandra/repair/RepairRunnable.java
@@ -64,6 +64,7 @@ import org.apache.cassandra.metrics.StorageMetrics;
 import org.apache.cassandra.locator.InetAddressAndPort;
 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.streaming.PreviewKind;
@@ -715,7 +716,7 @@ public class RepairRunnable extends WrappedRunnable implements ProgressEventNoti
 
                 String format = "select event_id, source, source_port, activity from %s.%s where session_id = ? and event_id > ? and event_id < ?;";
                 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());
 
                 ByteBuffer sessionIdBytes = ByteBufferUtil.bytes(sessionId);
                 InetAddressAndPort source = FBUtilities.getBroadcastAddressAndPort();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/repair/SystemDistributedKeyspace.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/repair/SystemDistributedKeyspace.java b/src/java/org/apache/cassandra/repair/SystemDistributedKeyspace.java
index 19d83db..a85a1e5 100644
--- a/src/java/org/apache/cassandra/repair/SystemDistributedKeyspace.java
+++ b/src/java/org/apache/cassandra/repair/SystemDistributedKeyspace.java
@@ -39,7 +39,7 @@ import org.slf4j.LoggerFactory;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.cql3.QueryProcessor;
 import org.apache.cassandra.cql3.UntypedResultSet;
-import org.apache.cassandra.cql3.statements.CreateTableStatement;
+import org.apache.cassandra.cql3.statements.schema.CreateTableStatement;
 import org.apache.cassandra.db.ConsistencyLevel;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.dht.Range;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/schema/ColumnMetadata.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/schema/ColumnMetadata.java b/src/java/org/apache/cassandra/schema/ColumnMetadata.java
index 0380b35..b6e743b 100644
--- a/src/java/org/apache/cassandra/schema/ColumnMetadata.java
+++ b/src/java/org/apache/cassandra/schema/ColumnMetadata.java
@@ -23,7 +23,6 @@ import java.util.function.Predicate;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.MoreObjects;
-import com.google.common.base.Objects;
 import com.google.common.collect.Collections2;
 
 import org.apache.cassandra.cql3.*;
@@ -263,12 +262,29 @@ public final class ColumnMetadata extends ColumnSpecification implements Selecta
 
         ColumnMetadata cd = (ColumnMetadata) o;
 
-        return Objects.equal(ksName, cd.ksName)
-            && Objects.equal(cfName, cd.cfName)
-            && Objects.equal(name, cd.name)
-            && Objects.equal(type, cd.type)
-            && Objects.equal(kind, cd.kind)
-            && Objects.equal(position, cd.position);
+        return equalsWithoutType(cd) && type.equals(cd.type);
+    }
+
+    private boolean equalsWithoutType(ColumnMetadata other)
+    {
+        return name.equals(other.name)
+            && kind == other.kind
+            && position == other.position
+            && ksName.equals(other.ksName)
+            && cfName.equals(other.cfName);
+    }
+
+    Optional<Difference> compare(ColumnMetadata other)
+    {
+        if (!equalsWithoutType(other))
+            return Optional.of(Difference.SHALLOW);
+
+        if (type.equals(other.type))
+            return Optional.empty();
+
+        return type.asCQL3Type().toString().equals(other.type.asCQL3Type().toString())
+             ? Optional.of(Difference.DEEP)
+             : Optional.of(Difference.SHALLOW);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/schema/CompressionParams.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/schema/CompressionParams.java b/src/java/org/apache/cassandra/schema/CompressionParams.java
index b96334b..d644c56 100644
--- a/src/java/org/apache/cassandra/schema/CompressionParams.java
+++ b/src/java/org/apache/cassandra/schema/CompressionParams.java
@@ -26,6 +26,7 @@ import java.util.Map;
 import java.util.concurrent.ThreadLocalRandom;
 
 import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Objects;
 import com.google.common.collect.ImmutableMap;
 
 import org.apache.commons.lang3.builder.EqualsBuilder;
@@ -84,6 +85,7 @@ public final class CompressionParams
     private final double minCompressRatio;  // In configuration we store min ratio, the input parameter.
     private final ImmutableMap<String, String> otherOptions; // Unrecognized options, can be used by the compressor
 
+    // TODO: deprecated, should now be carefully removed. Doesn't affect schema code as it isn't included in equals() and hashCode()
     private volatile double crcCheckChance = 1.0;
 
     public static CompressionParams fromMap(Map<String, String> opts)
@@ -548,20 +550,17 @@ public final class CompressionParams
     public boolean equals(Object obj)
     {
         if (obj == this)
-        {
             return true;
-        }
-        else if (obj == null || obj.getClass() != getClass())
-        {
+
+        if (!(obj instanceof CompressionParams))
             return false;
-        }
 
         CompressionParams cp = (CompressionParams) obj;
-        return new EqualsBuilder()
-            .append(sstableCompressor, cp.sstableCompressor)
-            .append(chunkLength(), cp.chunkLength())
-            .append(otherOptions, cp.otherOptions)
-            .isEquals();
+
+        return Objects.equal(sstableCompressor, cp.sstableCompressor)
+            && chunkLength == cp.chunkLength
+            && otherOptions.equals(cp.otherOptions)
+            && minCompressRatio == cp.minCompressRatio;
     }
 
     @Override
@@ -569,8 +568,9 @@ public final class CompressionParams
     {
         return new HashCodeBuilder(29, 1597)
             .append(sstableCompressor)
-            .append(chunkLength())
+            .append(chunkLength)
             .append(otherOptions)
+            .append(minCompressRatio)
             .toHashCode();
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/schema/Diff.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/schema/Diff.java b/src/java/org/apache/cassandra/schema/Diff.java
new file mode 100644
index 0000000..36c0687
--- /dev/null
+++ b/src/java/org/apache/cassandra/schema/Diff.java
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.schema;
+
+import com.google.common.collect.ImmutableCollection;
+import com.google.common.collect.Iterables;
+
+public class Diff<T extends Iterable, S>
+{
+    public final T created;
+    public final T dropped;
+    public final ImmutableCollection<Altered<S>> altered;
+
+    Diff(T created, T dropped, ImmutableCollection<Altered<S>> altered)
+    {
+        this.created = created;
+        this.dropped = dropped;
+        this.altered = altered;
+    }
+
+    boolean isEmpty()
+    {
+        return Iterables.isEmpty(created) && Iterables.isEmpty(dropped) && Iterables.isEmpty(altered);
+    }
+
+    Iterable<Altered<S>> altered(Difference kind)
+    {
+        return Iterables.filter(altered, a -> a.kind == kind);
+    }
+
+    public static final class Altered<T>
+    {
+        public final T before;
+        public final T after;
+        public final Difference kind;
+
+        Altered(T before, T after, Difference kind)
+        {
+            this.before = before;
+            this.after = after;
+            this.kind = kind;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/schema/Difference.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/schema/Difference.java b/src/java/org/apache/cassandra/schema/Difference.java
new file mode 100644
index 0000000..4f1aea9
--- /dev/null
+++ b/src/java/org/apache/cassandra/schema/Difference.java
@@ -0,0 +1,38 @@
+/*
+ * 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.schema;
+
+public enum Difference
+{
+    /**
+     * Two schema objects are considered to differ DEEP-ly if one or more of their nested schema objects differ.
+     *
+     * For example, if a table T has a column c of type U, where U is a user defined type, then upon altering U table
+     * T0 (before alter) will differ DEEP-ly from table T1 (after alter).
+     */
+    DEEP,
+
+    /**
+     *
+     * Two schema objects are considered to differ DEEP-ly if their direct structure is altered.
+     *
+     * For example, if a table T is altered to add a new column, a different compaction strategy, or a new description,
+     * then it will differ SHALLOW-ly from the original.
+     */
+    SHALLOW
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/schema/Functions.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/schema/Functions.java b/src/java/org/apache/cassandra/schema/Functions.java
index 8e3a3f1..2a0111c 100644
--- a/src/java/org/apache/cassandra/schema/Functions.java
+++ b/src/java/org/apache/cassandra/schema/Functions.java
@@ -17,19 +17,20 @@
  */
 package org.apache.cassandra.schema;
 
+import java.nio.ByteBuffer;
 import java.util.*;
-import java.util.stream.Collectors;
+import java.util.function.Predicate;
 import java.util.stream.Stream;
 
-import com.google.common.collect.ImmutableMultimap;
-import com.google.common.collect.MapDifference;
-import com.google.common.collect.Maps;
+import com.google.common.collect.*;
 
 import org.apache.cassandra.cql3.functions.*;
 import org.apache.cassandra.db.marshal.AbstractType;
-import org.apache.cassandra.utils.Pair;
+import org.apache.cassandra.db.marshal.UserType;
 
-import static com.google.common.collect.Iterables.filter;
+import static java.util.stream.Collectors.toList;
+
+import static com.google.common.collect.Iterables.any;
 
 /**
  * An immutable container for a keyspace's UDAs and UDFs (and, in case of {@link org.apache.cassandra.db.SystemKeyspace},
@@ -37,6 +38,21 @@ import static com.google.common.collect.Iterables.filter;
  */
 public final class Functions implements Iterable<Function>
 {
+    public enum Filter implements Predicate<Function>
+    {
+        ALL, UDF, UDA;
+
+        public boolean test(Function function)
+        {
+            switch (this)
+            {
+                case UDF: return function instanceof UDFunction;
+                case UDA: return function instanceof UDAggregate;
+                default:  return true;
+            }
+        }
+    }
+
     private final ImmutableMultimap<FunctionName, Function> functions;
 
     private Functions(Builder builder)
@@ -69,12 +85,17 @@ public final class Functions implements Iterable<Function>
         return functions.values().stream();
     }
 
+    public int size()
+    {
+        return functions.size();
+    }
+
     /**
      * @return a stream of keyspace's UDFs
      */
     public Stream<UDFunction> udfs()
     {
-        return stream().filter(f -> f instanceof UDFunction).map(f -> (UDFunction) f);
+        return stream().filter(Filter.UDF).map(f -> (UDFunction) f);
     }
 
     /**
@@ -82,38 +103,32 @@ public final class Functions implements Iterable<Function>
      */
     public Stream<UDAggregate> udas()
     {
-        return stream().filter(f -> f instanceof UDAggregate).map(f -> (UDAggregate) f);
+        return stream().filter(Filter.UDA).map(f -> (UDAggregate) f);
     }
 
-    MapDifference<Pair<FunctionName, List<String>>, UDFunction> udfsDiff(Functions other)
+    public Iterable<Function> referencingUserType(ByteBuffer name)
     {
-        Map<Pair<FunctionName, List<String>>, UDFunction> before = new HashMap<>();
-        udfs().forEach(f -> before.put(Pair.create(f.name(), f.argumentsList()), f));
-
-        Map<Pair<FunctionName, List<String>>, UDFunction> after = new HashMap<>();
-        other.udfs().forEach(f -> after.put(Pair.create(f.name(), f.argumentsList()), f));
-
-        return Maps.difference(before, after);
+        return Iterables.filter(this, f -> f.referencesUserType(name));
     }
 
-    MapDifference<Pair<FunctionName, List<String>>, UDAggregate> udasDiff(Functions other)
+    public Functions withUpdatedUserType(UserType udt)
     {
-        Map<Pair<FunctionName, List<String>>, UDAggregate> before = new HashMap<>();
-        udas().forEach(f -> before.put(Pair.create(f.name(), f.argumentsList()), f));
+        if (!any(this, f -> f.referencesUserType(udt.name)))
+            return this;
 
-        Map<Pair<FunctionName, List<String>>, UDAggregate> after = new HashMap<>();
-        other.udas().forEach(f -> after.put(Pair.create(f.name(), f.argumentsList()), f));
+        Collection<UDFunction>  udfs = udfs().map(f -> f.withUpdatedUserType(udt)).collect(toList());
+        Collection<UDAggregate> udas = udas().map(f -> f.withUpdatedUserType(udfs, udt)).collect(toList());
 
-        return Maps.difference(before, after);
+        return builder().add(udfs).add(udas).build();
     }
 
     /**
-     * @return a collection of aggregates that use the provided function as either a state or a final function
+     * @return a stream of aggregates that use the provided function as either a state or a final function
      * @param function the referree function
      */
-    public Collection<UDAggregate> aggregatesUsingFunction(Function function)
+    public Stream<UDAggregate> aggregatesUsingFunction(Function function)
     {
-        return udas().filter(uda -> uda.hasReferenceTo(function)).collect(Collectors.toList());
+        return udas().filter(uda -> uda.hasReferenceTo(function));
     }
 
     /**
@@ -127,6 +142,11 @@ public final class Functions implements Iterable<Function>
         return functions.get(name);
     }
 
+    public Optional<Function> find(FunctionName name, List<AbstractType<?>> argTypes)
+    {
+        return find(name, argTypes, Filter.ALL);
+    }
+
     /**
      * Find the function with the specified name
      *
@@ -134,13 +154,18 @@ public final class Functions implements Iterable<Function>
      * @param argTypes function argument types
      * @return an empty {@link Optional} if the function name is not found; a non-empty optional of {@link Function} otherwise
      */
-    public Optional<Function> find(FunctionName name, List<AbstractType<?>> argTypes)
+    public Optional<Function> find(FunctionName name, List<AbstractType<?>> argTypes, Filter filter)
     {
         return get(name).stream()
-                        .filter(fun -> typesMatch(fun.argTypes(), argTypes))
+                        .filter(filter.and(fun -> typesMatch(fun.argTypes(), argTypes)))
                         .findAny();
     }
 
+    public boolean isEmpty()
+    {
+        return functions.isEmpty();
+    }
+
     /*
      * We need to compare the CQL3 representation of the type because comparing
      * the AbstractType will fail for example if a UDT has been changed.
@@ -154,7 +179,7 @@ public final class Functions implements Iterable<Function>
      * or
      *    ALTER TYPE foo RENAME ...
      */
-    public static boolean typesMatch(AbstractType<?> t1, AbstractType<?> t2)
+    private static boolean typesMatch(AbstractType<?> t1, AbstractType<?> t2)
     {
         return t1.freeze().asCQL3Type().toString().equals(t2.freeze().asCQL3Type().toString());
     }
@@ -184,6 +209,13 @@ public final class Functions implements Iterable<Function>
         return h;
     }
 
+    public Functions filter(Predicate<Function> predicate)
+    {
+        Builder builder = builder();
+        stream().filter(predicate).forEach(builder::add);
+        return builder.build();
+    }
+
     /**
      * Create a Functions instance with the provided function added
      */
@@ -203,7 +235,19 @@ public final class Functions implements Iterable<Function>
         Function fun =
             find(name, argTypes).orElseThrow(() -> new IllegalStateException(String.format("Function %s doesn't exists", name)));
 
-        return builder().add(filter(this, f -> f != fun)).build();
+        return without(fun);
+    }
+
+    public Functions without(Function function)
+    {
+        return builder().add(Iterables.filter(this, f -> f != function)).build();
+    }
+
+    public Functions withAddedOrUpdated(Function function)
+    {
+        return builder().add(Iterables.filter(this, f -> !(f.name().equals(function.name()) && Functions.typesMatch(f.argTypes(), function.argTypes()))))
+                        .add(function)
+                        .build();
     }
 
     @Override
@@ -252,10 +296,52 @@ public final class Functions implements Iterable<Function>
             return this;
         }
 
-        public  Builder add(Iterable<? extends Function> funs)
+        public Builder add(Iterable<? extends Function> funs)
         {
             funs.forEach(this::add);
             return this;
         }
     }
+
+    @SuppressWarnings("unchecked")
+    static FunctionsDiff<UDFunction> udfsDiff(Functions before, Functions after)
+    {
+        return (FunctionsDiff<UDFunction>) FunctionsDiff.diff(before, after, Filter.UDF);
+    }
+
+    @SuppressWarnings("unchecked")
+    static FunctionsDiff<UDAggregate> udasDiff(Functions before, Functions after)
+    {
+        return (FunctionsDiff<UDAggregate>) FunctionsDiff.diff(before, after, Filter.UDA);
+    }
+
+    public static final class FunctionsDiff<T extends Function> extends Diff<Functions, T>
+    {
+        static final FunctionsDiff NONE = new FunctionsDiff<>(Functions.none(), Functions.none(), ImmutableList.of());
+
+        private FunctionsDiff(Functions created, Functions dropped, ImmutableCollection<Altered<T>> altered)
+        {
+            super(created, dropped, altered);
+        }
+
+        private static FunctionsDiff diff(Functions before, Functions after, Filter filter)
+        {
+            if (before == after)
+                return NONE;
+
+            Functions created = after.filter(filter.and(k -> !before.find(k.name(), k.argTypes(), filter).isPresent()));
+            Functions dropped = before.filter(filter.and(k -> !after.find(k.name(), k.argTypes(), filter).isPresent()));
+
+            ImmutableList.Builder<Altered<Function>> altered = ImmutableList.builder();
+            before.stream().filter(filter).forEach(functionBefore ->
+            {
+                after.find(functionBefore.name(), functionBefore.argTypes(), filter).ifPresent(functionAfter ->
+                {
+                    functionBefore.compare(functionAfter).ifPresent(kind -> altered.add(new Altered<>(functionBefore, functionAfter, kind)));
+                });
+            });
+
+            return new FunctionsDiff<>(created, dropped, altered.build());
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/schema/IndexMetadata.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/schema/IndexMetadata.java b/src/java/org/apache/cassandra/schema/IndexMetadata.java
index 190871a..3020793 100644
--- a/src/java/org/apache/cassandra/schema/IndexMetadata.java
+++ b/src/java/org/apache/cassandra/schema/IndexMetadata.java
@@ -32,7 +32,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.cql3.ColumnIdentifier;
-import org.apache.cassandra.cql3.statements.IndexTarget;
+import org.apache.cassandra.cql3.statements.schema.IndexTarget;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.exceptions.UnknownIndexException;
 import org.apache.cassandra.index.Index;
@@ -98,12 +98,14 @@ public final class IndexMetadata
         return name != null && !name.isEmpty() && PATTERN_WORD_CHARS.matcher(name).matches();
     }
 
-    public static String getDefaultIndexName(String cfName, String root)
+    public static String generateDefaultIndexName(String table, ColumnIdentifier column)
     {
-        if (root == null)
-            return PATTERN_NON_WORD_CHAR.matcher(cfName + "_" + "idx").replaceAll("");
-        else
-            return PATTERN_NON_WORD_CHAR.matcher(cfName + "_" + root + "_idx").replaceAll("");
+        return PATTERN_NON_WORD_CHAR.matcher(table + "_" + column.toString() + "_idx").replaceAll("");
+    }
+
+    public static String generateDefaultIndexName(String table)
+    {
+        return PATTERN_NON_WORD_CHAR.matcher(table + "_" + "idx").replaceAll("");
     }
 
     public void validate(TableMetadata table)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/schema/Indexes.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/schema/Indexes.java b/src/java/org/apache/cassandra/schema/Indexes.java
index 6122197..a83be4b 100644
--- a/src/java/org/apache/cassandra/schema/Indexes.java
+++ b/src/java/org/apache/cassandra/schema/Indexes.java
@@ -23,8 +23,6 @@ import java.util.stream.Stream;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Sets;
 
-import org.apache.cassandra.exceptions.ConfigurationException;
-
 import static java.lang.String.format;
 
 import static com.google.common.collect.Iterables.filter;
@@ -169,20 +167,6 @@ public final class Indexes implements Iterable<IndexMetadata>
 
     public void validate(TableMetadata table)
     {
-        /*
-         * Index name check is duplicated in Keyspaces, for the time being.
-         * The reason for this is that schema altering statements are not calling
-         * Keyspaces.validate() as of yet. TODO: remove this once they do (on CASSANDRA-9425 completion)
-         */
-        Set<String> indexNames = Sets.newHashSetWithExpectedSize(indexesByName.size());
-        for (IndexMetadata index : indexesByName.values())
-        {
-            if (indexNames.contains(index.name))
-                throw new ConfigurationException(format("Duplicate index name %s for table %s", index.name, table));
-
-            indexNames.add(index.name);
-        }
-
         indexesByName.values().forEach(i -> i.validate(table));
     }
 
@@ -198,20 +182,6 @@ public final class Indexes implements Iterable<IndexMetadata>
         return indexesByName.values().toString();
     }
 
-    public static String getAvailableIndexName(String ksName, String cfName, String indexNameRoot)
-    {
-
-        KeyspaceMetadata ksm = Schema.instance.getKeyspaceMetadata(ksName);
-        Set<String> existingNames = ksm == null ? new HashSet<>() : ksm.existingIndexNames(null);
-        String baseName = IndexMetadata.getDefaultIndexName(cfName, indexNameRoot);
-        String acceptedName = baseName;
-        int i = 0;
-        while (existingNames.contains(acceptedName))
-            acceptedName = baseName + '_' + (++i);
-
-        return acceptedName;
-    }
-
     public static final class Builder
     {
         final ImmutableMap.Builder<String, IndexMetadata> indexesByName = new ImmutableMap.Builder<>();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/schema/KeyspaceMetadata.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/schema/KeyspaceMetadata.java b/src/java/org/apache/cassandra/schema/KeyspaceMetadata.java
index 5a72d2c..aacd962 100644
--- a/src/java/org/apache/cassandra/schema/KeyspaceMetadata.java
+++ b/src/java/org/apache/cassandra/schema/KeyspaceMetadata.java
@@ -27,10 +27,22 @@ import com.google.common.base.MoreObjects;
 import com.google.common.base.Objects;
 import com.google.common.collect.Iterables;
 
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.cql3.functions.UDAggregate;
+import org.apache.cassandra.cql3.functions.UDFunction;
+import org.apache.cassandra.db.marshal.UserType;
 import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.locator.AbstractReplicationStrategy;
+import org.apache.cassandra.schema.Functions.FunctionsDiff;
+import org.apache.cassandra.schema.Tables.TablesDiff;
+import org.apache.cassandra.schema.Types.TypesDiff;
+import org.apache.cassandra.schema.Views.ViewsDiff;
+import org.apache.cassandra.service.StorageService;
 
 import static java.lang.String.format;
 
+import static com.google.common.collect.Iterables.any;
+
 /**
  * An immutable representation of keyspace metadata (name, params, tables, types, and functions).
  */
@@ -110,9 +122,24 @@ public final class KeyspaceMetadata
         return kind == Kind.VIRTUAL;
     }
 
+    /**
+     * Returns a new KeyspaceMetadata with all instances of old UDT replaced with the updated version.
+     * Replaces all instances in tables, views, types, and functions.
+     */
+    public KeyspaceMetadata withUpdatedUserType(UserType udt)
+    {
+        return new KeyspaceMetadata(name,
+                                    kind,
+                                    params,
+                                    tables.withUpdatedUserType(udt),
+                                    views.withUpdatedUserTypes(udt),
+                                    types.withUpdatedUserType(udt),
+                                    functions.withUpdatedUserType(udt));
+    }
+
     public Iterable<TableMetadata> tablesAndViews()
     {
-        return Iterables.concat(tables, views.metadatas());
+        return Iterables.concat(tables, views.allTableMetadata());
     }
 
     @Nullable
@@ -124,14 +151,34 @@ public final class KeyspaceMetadata
              : view.metadata;
     }
 
-    public Set<String> existingIndexNames(String cfToExclude)
+    public boolean hasTable(String tableName)
     {
-        Set<String> indexNames = new HashSet<>();
-        for (TableMetadata table : tables)
-            if (cfToExclude == null || !table.name.equals(cfToExclude))
-                for (IndexMetadata index : table.indexes)
-                    indexNames.add(index.name);
-        return indexNames;
+        return tables.get(tableName).isPresent();
+    }
+
+    public boolean hasView(String viewName)
+    {
+        return views.get(viewName).isPresent();
+    }
+
+    public boolean hasIndex(String indexName)
+    {
+        return any(tables, t -> t.indexes.has(indexName));
+    }
+
+    public String findAvailableIndexName(String baseName)
+    {
+        if (!hasIndex(baseName))
+            return baseName;
+
+        int i = 1;
+        do
+        {
+            String name = baseName + '_' + i++;
+            if (!hasIndex(name))
+                return name;
+        }
+        while (true);
     }
 
     public Optional<TableMetadata> findIndexedTable(String indexName)
@@ -209,4 +256,77 @@ public final class KeyspaceMetadata
             }
         }
     }
+
+    public AbstractReplicationStrategy createReplicationStrategy()
+    {
+        return AbstractReplicationStrategy.createReplicationStrategy(name,
+                                                                     params.replication.klass,
+                                                                     StorageService.instance.getTokenMetadata(),
+                                                                     DatabaseDescriptor.getEndpointSnitch(),
+                                                                     params.replication.options);
+    }
+
+    static Optional<KeyspaceDiff> diff(KeyspaceMetadata before, KeyspaceMetadata after)
+    {
+        return KeyspaceDiff.diff(before, after);
+    }
+
+    public static final class KeyspaceDiff
+    {
+        public final KeyspaceMetadata before;
+        public final KeyspaceMetadata after;
+
+        public final TablesDiff tables;
+        public final ViewsDiff views;
+        public final TypesDiff types;
+
+        public final FunctionsDiff<UDFunction> udfs;
+        public final FunctionsDiff<UDAggregate> udas;
+
+        private KeyspaceDiff(KeyspaceMetadata before,
+                             KeyspaceMetadata after,
+                             TablesDiff tables,
+                             ViewsDiff views,
+                             TypesDiff types,
+                             FunctionsDiff<UDFunction> udfs,
+                             FunctionsDiff<UDAggregate> udas)
+        {
+            this.before = before;
+            this.after = after;
+            this.tables = tables;
+            this.views = views;
+            this.types = types;
+            this.udfs = udfs;
+            this.udas = udas;
+        }
+
+        private static Optional<KeyspaceDiff> diff(KeyspaceMetadata before, KeyspaceMetadata after)
+        {
+            if (before == after)
+                return Optional.empty();
+
+            if (!before.name.equals(after.name))
+            {
+                String msg = String.format("Attempting to diff two keyspaces with different names ('%s' and '%s')", before.name, after.name);
+                throw new IllegalArgumentException(msg);
+            }
+
+            TablesDiff tables = Tables.diff(before.tables, after.tables);
+            ViewsDiff views = Views.diff(before.views, after.views);
+            TypesDiff types = Types.diff(before.types, after.types);
+
+            @SuppressWarnings("unchecked") FunctionsDiff<UDFunction>  udfs = FunctionsDiff.NONE;
+            @SuppressWarnings("unchecked") FunctionsDiff<UDAggregate> udas = FunctionsDiff.NONE;
+            if (before.functions != after.functions)
+            {
+                udfs = Functions.udfsDiff(before.functions, after.functions);
+                udas = Functions.udasDiff(before.functions, after.functions);
+            }
+
+            if (before.params.equals(after.params) && tables.isEmpty() && views.isEmpty() && types.isEmpty() && udfs.isEmpty() && udas.isEmpty())
+                return Optional.empty();
+
+            return Optional.of(new KeyspaceDiff(before, after, tables, views, types, udfs, udas));
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/schema/KeyspaceParams.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/schema/KeyspaceParams.java b/src/java/org/apache/cassandra/schema/KeyspaceParams.java
index 1deaa29..68ac5e4 100644
--- a/src/java/org/apache/cassandra/schema/KeyspaceParams.java
+++ b/src/java/org/apache/cassandra/schema/KeyspaceParams.java
@@ -31,8 +31,8 @@ public final class KeyspaceParams
     public static final boolean DEFAULT_DURABLE_WRITES = true;
 
     /**
-     * This determines durable writes for the {@link org.apache.cassandra.config.SchemaConstants#SCHEMA_KEYSPACE_NAME}
-     * and {@link org.apache.cassandra.config.SchemaConstants#SYSTEM_KEYSPACE_NAME} keyspaces,
+     * This determines durable writes for the {@link org.apache.cassandra.schema.SchemaConstants#SCHEMA_KEYSPACE_NAME}
+     * and {@link org.apache.cassandra.schema.SchemaConstants#SYSTEM_KEYSPACE_NAME} keyspaces,
      * the only reason it is not final is for commitlog unit tests. It should only be changed for testing purposes.
      */
     @VisibleForTesting

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/schema/Keyspaces.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/schema/Keyspaces.java b/src/java/org/apache/cassandra/schema/Keyspaces.java
index 1692f88..1938d93 100644
--- a/src/java/org/apache/cassandra/schema/Keyspaces.java
+++ b/src/java/org/apache/cassandra/schema/Keyspaces.java
@@ -18,18 +18,21 @@
 package org.apache.cassandra.schema;
 
 import java.util.Iterator;
+import java.util.Optional;
 import java.util.Set;
 import java.util.function.Predicate;
 import java.util.stream.Stream;
 
 import javax.annotation.Nullable;
 
-import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.MapDifference;
-import com.google.common.collect.Maps;
+import com.google.common.collect.*;
+
+import org.apache.cassandra.schema.KeyspaceMetadata.KeyspaceDiff;
 
 public final class Keyspaces implements Iterable<KeyspaceMetadata>
 {
+    private static final Keyspaces NONE = builder().build();
+
     private final ImmutableMap<String, KeyspaceMetadata> keyspaces;
     private final ImmutableMap<TableId, TableMetadata> tables;
 
@@ -46,7 +49,7 @@ public final class Keyspaces implements Iterable<KeyspaceMetadata>
 
     public static Keyspaces none()
     {
-        return builder().build();
+        return NONE;
     }
 
     public static Keyspaces of(KeyspaceMetadata... keyspaces)
@@ -69,18 +72,39 @@ public final class Keyspaces implements Iterable<KeyspaceMetadata>
         return keyspaces.keySet();
     }
 
+    /**
+     * Get the keyspace with the specified name
+     *
+     * @param name a non-qualified keyspace name
+     * @return an empty {@link Optional} if the table name is not found; a non-empty optional of {@link KeyspaceMetadata} otherwise
+     */
+    public Optional<KeyspaceMetadata> get(String name)
+    {
+        return Optional.ofNullable(keyspaces.get(name));
+    }
+
     @Nullable
     public KeyspaceMetadata getNullable(String name)
     {
         return keyspaces.get(name);
     }
 
+    public boolean containsKeyspace(String name)
+    {
+        return keyspaces.containsKey(name);
+    }
+
     @Nullable
     public TableMetadata getTableOrViewNullable(TableId id)
     {
         return tables.get(id);
     }
 
+    public boolean isEmpty()
+    {
+        return keyspaces.isEmpty();
+    }
+
     public Keyspaces filter(Predicate<KeyspaceMetadata> predicate)
     {
         Builder builder = builder();
@@ -97,19 +121,19 @@ public final class Keyspaces implements Iterable<KeyspaceMetadata>
         if (keyspace == null)
             throw new IllegalStateException(String.format("Keyspace %s doesn't exists", name));
 
-        return builder().add(filter(k -> k != keyspace)).build();
+        return filter(k -> k != keyspace);
     }
 
     public Keyspaces withAddedOrUpdated(KeyspaceMetadata keyspace)
     {
-        return builder().add(filter(k -> !k.name.equals(keyspace.name)))
+        return builder().add(Iterables.filter(this, k -> !k.name.equals(keyspace.name)))
                         .add(keyspace)
                         .build();
     }
 
-    MapDifference<String, KeyspaceMetadata> diff(Keyspaces other)
+    public void validate()
     {
-        return Maps.difference(keyspaces, other.keyspaces);
+        keyspaces.values().forEach(KeyspaceMetadata::validate);
     }
 
     @Override
@@ -167,4 +191,49 @@ public final class Keyspaces implements Iterable<KeyspaceMetadata>
             return this;
         }
     }
+
+    static KeyspacesDiff diff(Keyspaces before, Keyspaces after)
+    {
+        return KeyspacesDiff.diff(before, after);
+    }
+
+    public static final class KeyspacesDiff
+    {
+        static final KeyspacesDiff NONE = new KeyspacesDiff(Keyspaces.none(), Keyspaces.none(), ImmutableList.of());
+
+        public final Keyspaces created;
+        public final Keyspaces dropped;
+        public final ImmutableList<KeyspaceDiff> altered;
+
+        private KeyspacesDiff(Keyspaces created, Keyspaces dropped, ImmutableList<KeyspaceDiff> altered)
+        {
+            this.created = created;
+            this.dropped = dropped;
+            this.altered = altered;
+        }
+
+        private static KeyspacesDiff diff(Keyspaces before, Keyspaces after)
+        {
+            if (before == after)
+                return NONE;
+
+            Keyspaces created = after.filter(k -> !before.containsKeyspace(k.name));
+            Keyspaces dropped = before.filter(k -> !after.containsKeyspace(k.name));
+
+            ImmutableList.Builder<KeyspaceDiff> altered = ImmutableList.builder();
+            before.forEach(keyspaceBefore ->
+            {
+                KeyspaceMetadata keyspaceAfter = after.getNullable(keyspaceBefore.name);
+                if (null != keyspaceAfter)
+                    KeyspaceMetadata.diff(keyspaceBefore, keyspaceAfter).ifPresent(altered::add);
+            });
+
+            return new KeyspacesDiff(created, dropped, altered.build());
+        }
+
+        public boolean isEmpty()
+        {
+            return created.isEmpty() && dropped.isEmpty() && altered.isEmpty();
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/schema/MigrationManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/schema/MigrationManager.java b/src/java/org/apache/cassandra/schema/MigrationManager.java
index c8881e5..ac95054 100644
--- a/src/java/org/apache/cassandra/schema/MigrationManager.java
+++ b/src/java/org/apache/cassandra/schema/MigrationManager.java
@@ -23,15 +23,14 @@ import java.util.concurrent.*;
 import java.lang.management.ManagementFactory;
 import java.lang.management.RuntimeMXBean;
 
+import com.google.common.util.concurrent.Futures;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
 import org.apache.cassandra.concurrent.ScheduledExecutors;
 import org.apache.cassandra.concurrent.Stage;
 import org.apache.cassandra.concurrent.StageManager;
-import org.apache.cassandra.cql3.functions.UDAggregate;
-import org.apache.cassandra.cql3.functions.UDFunction;
 import org.apache.cassandra.db.*;
-import org.apache.cassandra.db.marshal.UserType;
 import org.apache.cassandra.exceptions.AlreadyExistsException;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.gms.*;
@@ -41,9 +40,8 @@ import org.apache.cassandra.io.util.DataOutputPlus;
 import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.net.MessageOut;
 import org.apache.cassandra.net.MessagingService;
-import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.schema.Keyspaces.KeyspacesDiff;
 import org.apache.cassandra.utils.FBUtilities;
-import org.apache.cassandra.utils.WrappedRunnable;
 
 public class MigrationManager
 {
@@ -150,6 +148,14 @@ public class MigrationManager
                 && !Gossiper.instance.isGossipOnlyMember(endpoint);
     }
 
+    private static boolean shouldPushSchemaTo(InetAddressAndPort endpoint)
+    {
+        // only push schema to nodes with known and equal versions
+        return !endpoint.equals(FBUtilities.getBroadcastAddressAndPort())
+               && MessagingService.instance().knowsVersion(endpoint)
+               && MessagingService.instance().getRawVersion(endpoint) == MessagingService.current_version;
+    }
+
     public static boolean isReadyForBootstrap()
     {
         return MigrationTask.getInflightTasks().isEmpty();
@@ -194,21 +200,16 @@ public class MigrationManager
         announce(SchemaKeyspace.makeCreateKeyspaceMutation(ksm, timestamp), announceLocally);
     }
 
-    public static void announceNewTable(TableMetadata cfm) throws ConfigurationException
-    {
-        announceNewTable(cfm, false);
-    }
-
-    public static void announceNewTable(TableMetadata cfm, boolean announceLocally)
+    public static void announceNewTable(TableMetadata cfm)
     {
-        announceNewTable(cfm, announceLocally, true);
+        announceNewTable(cfm, true, FBUtilities.timestampMicros());
     }
 
     /**
      * Announces the table even if the definition is already know locally.
      * This should generally be avoided but is used internally when we want to force the most up to date version of
      * a system table schema (Note that we don't know if the schema we force _is_ the most recent version or not, we
-     * just rely on idempotency to basically ignore that announce if it's not. That's why we can't use announceUpdateColumnFamily,
+     * just rely on idempotency to basically ignore that announce if it's not. That's why we can't use announceTableUpdate
      * it would for instance delete new columns if this is not called with the most up-to-date version)
      *
      * Note that this is only safe for system tables where we know the id is fixed and will be the same whatever version
@@ -216,15 +217,10 @@ public class MigrationManager
      */
     public static void forceAnnounceNewTable(TableMetadata cfm)
     {
-        announceNewTable(cfm, false, false, 0);
-    }
-
-    private static void announceNewTable(TableMetadata cfm, boolean announceLocally, boolean throwOnDuplicate)
-    {
-        announceNewTable(cfm, announceLocally, throwOnDuplicate, FBUtilities.timestampMicros());
+        announceNewTable(cfm, false, 0);
     }
 
-    private static void announceNewTable(TableMetadata cfm, boolean announceLocally, boolean throwOnDuplicate, long timestamp)
+    private static void announceNewTable(TableMetadata cfm, boolean throwOnDuplicate, long timestamp)
     {
         cfm.validate();
 
@@ -236,49 +232,10 @@ public class MigrationManager
             throw new AlreadyExistsException(cfm.keyspace, cfm.name);
 
         logger.info("Create new table: {}", cfm);
-        announce(SchemaKeyspace.makeCreateTableMutation(ksm, cfm, timestamp), announceLocally);
-    }
-
-    public static void announceNewView(ViewMetadata view, boolean announceLocally) throws ConfigurationException
-    {
-        view.metadata.validate();
-
-        KeyspaceMetadata ksm = Schema.instance.getKeyspaceMetadata(view.keyspace);
-        if (ksm == null)
-            throw new ConfigurationException(String.format("Cannot add table '%s' to non existing keyspace '%s'.", view.name, view.keyspace));
-        else if (ksm.getTableOrViewNullable(view.name) != null)
-            throw new AlreadyExistsException(view.keyspace, view.name);
-
-        logger.info("Create new view: {}", view);
-        announce(SchemaKeyspace.makeCreateViewMutation(ksm, view, FBUtilities.timestampMicros()), announceLocally);
-    }
-
-    public static void announceNewType(UserType newType, boolean announceLocally)
-    {
-        KeyspaceMetadata ksm = Schema.instance.getKeyspaceMetadata(newType.keyspace);
-        announce(SchemaKeyspace.makeCreateTypeMutation(ksm, newType, FBUtilities.timestampMicros()), announceLocally);
-    }
-
-    public static void announceNewFunction(UDFunction udf, boolean announceLocally)
-    {
-        logger.info("Create scalar function '{}'", udf.name());
-        KeyspaceMetadata ksm = Schema.instance.getKeyspaceMetadata(udf.name().keyspace);
-        announce(SchemaKeyspace.makeCreateFunctionMutation(ksm, udf, FBUtilities.timestampMicros()), announceLocally);
-    }
-
-    public static void announceNewAggregate(UDAggregate udf, boolean announceLocally)
-    {
-        logger.info("Create aggregate function '{}'", udf.name());
-        KeyspaceMetadata ksm = Schema.instance.getKeyspaceMetadata(udf.name().keyspace);
-        announce(SchemaKeyspace.makeCreateAggregateMutation(ksm, udf, FBUtilities.timestampMicros()), announceLocally);
+        announce(SchemaKeyspace.makeCreateTableMutation(ksm, cfm, timestamp), false);
     }
 
-    public static void announceKeyspaceUpdate(KeyspaceMetadata ksm) throws ConfigurationException
-    {
-        announceKeyspaceUpdate(ksm, false);
-    }
-
-    public static void announceKeyspaceUpdate(KeyspaceMetadata ksm, boolean announceLocally) throws ConfigurationException
+    static void announceKeyspaceUpdate(KeyspaceMetadata ksm)
     {
         ksm.validate();
 
@@ -287,20 +244,15 @@ public class MigrationManager
             throw new ConfigurationException(String.format("Cannot update non existing keyspace '%s'.", ksm.name));
 
         logger.info("Update Keyspace '{}' From {} To {}", ksm.name, oldKsm, ksm);
-        announce(SchemaKeyspace.makeCreateKeyspaceMutation(ksm.name, ksm.params, FBUtilities.timestampMicros()), announceLocally);
+        announce(SchemaKeyspace.makeCreateKeyspaceMutation(ksm.name, ksm.params, FBUtilities.timestampMicros()), false);
     }
 
-    public static void announceTableUpdate(TableMetadata tm) throws ConfigurationException
+    public static void announceTableUpdate(TableMetadata tm)
     {
         announceTableUpdate(tm, false);
     }
 
-    public static void announceTableUpdate(TableMetadata updated, boolean announceLocally) throws ConfigurationException
-    {
-        announceTableUpdate(updated, null, announceLocally);
-    }
-
-    public static void announceTableUpdate(TableMetadata updated, Collection<ViewMetadata> views, boolean announceLocally) throws ConfigurationException
+    public static void announceTableUpdate(TableMetadata updated, boolean announceLocally)
     {
         updated.validate();
 
@@ -309,69 +261,27 @@ public class MigrationManager
             throw new ConfigurationException(String.format("Cannot update non existing table '%s' in keyspace '%s'.", updated.name, updated.keyspace));
         KeyspaceMetadata ksm = Schema.instance.getKeyspaceMetadata(current.keyspace);
 
-        current.validateCompatibility(updated);
+        updated.validateCompatibility(current);
 
         long timestamp = FBUtilities.timestampMicros();
 
         logger.info("Update table '{}/{}' From {} To {}", current.keyspace, current.name, current, updated);
         Mutation.SimpleBuilder builder = SchemaKeyspace.makeUpdateTableMutation(ksm, current, updated, timestamp);
 
-        if (views != null)
-            views.forEach(view -> addViewUpdateToMutationBuilder(view, builder));
-
-        announce(builder, announceLocally);
-    }
-
-    public static void announceViewUpdate(ViewMetadata view, boolean announceLocally) throws ConfigurationException
-    {
-        KeyspaceMetadata ksm = Schema.instance.getKeyspaceMetadata(view.keyspace);
-        long timestamp = FBUtilities.timestampMicros();
-        Mutation.SimpleBuilder builder = SchemaKeyspace.makeCreateKeyspaceMutation(ksm.name, ksm.params, timestamp);
-        addViewUpdateToMutationBuilder(view, builder);
         announce(builder, announceLocally);
     }
 
-    private static void addViewUpdateToMutationBuilder(ViewMetadata view, Mutation.SimpleBuilder builder)
-    {
-        view.metadata.validate();
-
-        ViewMetadata oldView = Schema.instance.getView(view.keyspace, view.name);
-        if (oldView == null)
-            throw new ConfigurationException(String.format("Cannot update non existing materialized view '%s' in keyspace '%s'.", view.name, view.keyspace));
-
-        oldView.metadata.validateCompatibility(view.metadata);
-
-        logger.info("Update view '{}/{}' From {} To {}", view.keyspace, view.name, oldView, view);
-        SchemaKeyspace.makeUpdateViewMutation(builder, oldView, view);
-    }
-
-    public static void announceTypeUpdate(UserType updatedType, boolean announceLocally)
-    {
-        logger.info("Update type '{}.{}' to {}", updatedType.keyspace, updatedType.getNameAsString(), updatedType);
-        announceNewType(updatedType, announceLocally);
-    }
-
-    public static void announceKeyspaceDrop(String ksName) throws ConfigurationException
-    {
-        announceKeyspaceDrop(ksName, false);
-    }
-
-    public static void announceKeyspaceDrop(String ksName, boolean announceLocally) throws ConfigurationException
+    static void announceKeyspaceDrop(String ksName)
     {
         KeyspaceMetadata oldKsm = Schema.instance.getKeyspaceMetadata(ksName);
         if (oldKsm == null)
             throw new ConfigurationException(String.format("Cannot drop non existing keyspace '%s'.", ksName));
 
         logger.info("Drop Keyspace '{}'", oldKsm.name);
-        announce(SchemaKeyspace.makeDropKeyspaceMutation(oldKsm, FBUtilities.timestampMicros()), announceLocally);
-    }
-
-    public static void announceTableDrop(String ksName, String cfName) throws ConfigurationException
-    {
-        announceTableDrop(ksName, cfName, false);
+        announce(SchemaKeyspace.makeDropKeyspaceMutation(oldKsm, FBUtilities.timestampMicros()), false);
     }
 
-    public static void announceTableDrop(String ksName, String cfName, boolean announceLocally) throws ConfigurationException
+    public static void announceTableDrop(String ksName, String cfName, boolean announceLocally)
     {
         TableMetadata tm = Schema.instance.getTableMetadata(ksName, cfName);
         if (tm == null)
@@ -382,37 +292,6 @@ public class MigrationManager
         announce(SchemaKeyspace.makeDropTableMutation(ksm, tm, FBUtilities.timestampMicros()), announceLocally);
     }
 
-    public static void announceViewDrop(String ksName, String viewName, boolean announceLocally) throws ConfigurationException
-    {
-        ViewMetadata view = Schema.instance.getView(ksName, viewName);
-        if (view == null)
-            throw new ConfigurationException(String.format("Cannot drop non existing materialized view '%s' in keyspace '%s'.", viewName, ksName));
-        KeyspaceMetadata ksm = Schema.instance.getKeyspaceMetadata(ksName);
-
-        logger.info("Drop table '{}/{}'", view.keyspace, view.name);
-        announce(SchemaKeyspace.makeDropViewMutation(ksm, view, FBUtilities.timestampMicros()), announceLocally);
-    }
-
-    public static void announceTypeDrop(UserType droppedType, boolean announceLocally)
-    {
-        KeyspaceMetadata ksm = Schema.instance.getKeyspaceMetadata(droppedType.keyspace);
-        announce(SchemaKeyspace.dropTypeFromSchemaMutation(ksm, droppedType, FBUtilities.timestampMicros()), announceLocally);
-    }
-
-    public static void announceFunctionDrop(UDFunction udf, boolean announceLocally)
-    {
-        logger.info("Drop scalar function overload '{}' args '{}'", udf.name(), udf.argTypes());
-        KeyspaceMetadata ksm = Schema.instance.getKeyspaceMetadata(udf.name().keyspace);
-        announce(SchemaKeyspace.makeDropFunctionMutation(ksm, udf, FBUtilities.timestampMicros()), announceLocally);
-    }
-
-    public static void announceAggregateDrop(UDAggregate udf, boolean announceLocally)
-    {
-        logger.info("Drop aggregate function overload '{}' args '{}'", udf.name(), udf.argTypes());
-        KeyspaceMetadata ksm = Schema.instance.getKeyspaceMetadata(udf.name().keyspace);
-        announce(SchemaKeyspace.makeDropAggregateMutation(ksm, udf, FBUtilities.timestampMicros()), announceLocally);
-    }
-
     /**
      * actively announce a new version to active hosts via rpc
      * @param schema The schema mutation to be applied
@@ -424,7 +303,7 @@ public class MigrationManager
         if (announceLocally)
             Schema.instance.merge(mutations);
         else
-            FBUtilities.waitOnFuture(announce(mutations));
+            announce(mutations);
     }
 
     private static void pushSchemaMutation(InetAddressAndPort endpoint, Collection<Mutation> schema)
@@ -436,38 +315,36 @@ public class MigrationManager
     }
 
     // Returns a future on the local application of the schema
-    private static Future<?> announce(final Collection<Mutation> schema)
+    private static void announce(Collection<Mutation> schema)
     {
-        Future<?> f = StageManager.getStage(Stage.MIGRATION).submit(new WrappedRunnable()
-        {
-            protected void runMayThrow() throws ConfigurationException
-            {
-                Schema.instance.mergeAndAnnounceVersion(schema);
-            }
-        });
+        Future<?> f = StageManager.getStage(Stage.MIGRATION).submit(() -> Schema.instance.mergeAndAnnounceVersion(schema));
 
         for (InetAddressAndPort endpoint : Gossiper.instance.getLiveMembers())
-        {
-            // only push schema to nodes with known and equal versions
-            if (!endpoint.equals(FBUtilities.getBroadcastAddressAndPort()) &&
-                    MessagingService.instance().knowsVersion(endpoint) &&
-                    MessagingService.instance().getRawVersion(endpoint) == MessagingService.current_version)
+            if (shouldPushSchemaTo(endpoint))
                 pushSchemaMutation(endpoint, schema);
-        }
 
-        return f;
+        FBUtilities.waitOnFuture(f);
     }
 
-    /**
-     * Announce my version passively over gossip.
-     * Used to notify nodes as they arrive in the cluster.
-     *
-     * @param version The schema version to announce
-     */
-    static void passiveAnnounce(UUID version)
+    public static KeyspacesDiff announce(SchemaTransformation transformation, boolean locally)
     {
-        Gossiper.instance.addLocalApplicationState(ApplicationState.SCHEMA, StorageService.instance.valueFactory.schema(version));
-        logger.debug("Gossiping my schema version {}", version);
+        long now = FBUtilities.timestampMicros();
+
+        Future<Schema.TransformationResult> future =
+            StageManager.getStage(Stage.MIGRATION).submit(() -> Schema.instance.transform(transformation, locally, now));
+
+        Schema.TransformationResult result = Futures.getUnchecked(future);
+        if (!result.success)
+            throw result.exception;
+
+        if (locally || result.diff.isEmpty())
+            return result.diff;
+
+        for (InetAddressAndPort endpoint : Gossiper.instance.getLiveMembers())
+            if (shouldPushSchemaTo(endpoint))
+                pushSchemaMutation(endpoint, result.mutations);
+
+        return result.diff;
     }
 
     /**


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


[11/13] cassandra git commit: Make all DDL statements idempotent and not dependent on global state

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/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
deleted file mode 100644
index 260c8fd..0000000
--- a/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java
+++ /dev/null
@@ -1,317 +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.statements;
-
-import java.util.*;
-import java.util.stream.Collectors;
-
-import com.google.common.collect.Iterables;
-
-import org.apache.cassandra.audit.AuditLogEntryType;
-import org.apache.cassandra.auth.Permission;
-import org.apache.cassandra.cql3.*;
-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.view.View;
-import org.apache.cassandra.exceptions.*;
-import org.apache.cassandra.schema.ColumnMetadata;
-import org.apache.cassandra.schema.DroppedColumn;
-import org.apache.cassandra.schema.IndexMetadata;
-import org.apache.cassandra.schema.Indexes;
-import org.apache.cassandra.schema.MigrationManager;
-import org.apache.cassandra.schema.Schema;
-import org.apache.cassandra.schema.TableMetadata;
-import org.apache.cassandra.schema.TableParams;
-import org.apache.cassandra.schema.ViewMetadata;
-import org.apache.cassandra.service.ClientState;
-import org.apache.cassandra.service.QueryState;
-import org.apache.cassandra.transport.Event;
-
-public class AlterTableStatement extends SchemaAlteringStatement
-{
-    public enum Type
-    {
-        ADD, ALTER, DROP, OPTS, RENAME
-    }
-
-    public final Type oType;
-    private final TableAttributes attrs;
-    private final Map<ColumnMetadata.Raw, ColumnMetadata.Raw> renames;
-    private final List<AlterTableStatementColumn> colNameList;
-    private final Long deleteTimestamp;
-
-    public AlterTableStatement(CFName name,
-                               Type type,
-                               List<AlterTableStatementColumn> colDataList,
-                               TableAttributes attrs,
-                               Map<ColumnMetadata.Raw, ColumnMetadata.Raw> renames,
-                               Long deleteTimestamp)
-    {
-        super(name);
-        this.oType = type;
-        this.colNameList = colDataList;
-        this.attrs = attrs;
-        this.renames = renames;
-        this.deleteTimestamp = deleteTimestamp;
-    }
-
-    public void checkAccess(ClientState state) throws UnauthorizedException, InvalidRequestException
-    {
-        state.hasColumnFamilyAccess(keyspace(), columnFamily(), Permission.ALTER);
-    }
-
-    public void validate(ClientState state)
-    {
-        // validated in announceMigration()
-    }
-
-    public Event.SchemaChange announceMigration(QueryState queryState, boolean isLocalOnly) throws RequestValidationException
-    {
-        TableMetadata current = Schema.instance.validateTable(keyspace(), columnFamily());
-        if (current.isView())
-            throw new InvalidRequestException("Cannot use ALTER TABLE on Materialized View");
-
-        if (current.isVirtual())
-            throw new InvalidRequestException("Cannot alter virtual tables");
-
-        TableMetadata.Builder builder = current.unbuild();
-
-        ColumnIdentifier columnName = null;
-        ColumnMetadata def = null;
-        CQL3Type.Raw dataType = null;
-        boolean isStatic = false;
-        CQL3Type validator = null;
-
-        List<ViewMetadata> viewUpdates = new ArrayList<>();
-        Iterable<ViewMetadata> views = View.findAll(keyspace(), columnFamily());
-
-        switch (oType)
-        {
-            case ALTER:
-                throw new InvalidRequestException("Altering of types is not allowed");
-            case ADD:
-                if (current.isDense())
-                    throw new InvalidRequestException("Cannot add new column to a COMPACT STORAGE table");
-
-                for (AlterTableStatementColumn colData : colNameList)
-                {
-                    columnName = colData.getColumnName().getIdentifier(current);
-                    def = builder.getColumn(columnName);
-                    dataType = colData.getColumnType();
-                    assert dataType != null;
-                    isStatic = colData.getStaticType();
-                    validator = dataType.prepare(keyspace());
-
-
-                    if (isStatic)
-                    {
-                        if (!current.isCompound())
-                            throw new InvalidRequestException("Static columns are not allowed in COMPACT STORAGE tables");
-                        if (current.clusteringColumns().isEmpty())
-                            throw new InvalidRequestException("Static columns are only useful (and thus allowed) if the table has at least one clustering column");
-                    }
-
-                    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));
-                        }
-                    }
-
-                    // Cannot re-add a dropped counter column. See #7831.
-                    if (current.isCounter() && current.getDroppedColumn(columnName.bytes) != null)
-                        throw new InvalidRequestException(String.format("Cannot re-add previously dropped counter column %s", columnName));
-
-                    AbstractType<?> type = validator.getType();
-                    if (type.isCollection() && type.isMultiCell())
-                    {
-                        if (!current.isCompound())
-                            throw new InvalidRequestException("Cannot use non-frozen collections in COMPACT STORAGE tables");
-                        if (current.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).
-                        DroppedColumn dropped = current.droppedColumns.get(columnName.bytes);
-                        if (dropped != null && dropped.column.type instanceof CollectionType
-                            && dropped.column.type.isMultiCell() && !type.isCompatibleWith(dropped.column.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.column.type.asCQL3Type());
-                            throw new InvalidRequestException(message);
-                        }
-                    }
-
-                    builder.addColumn(isStatic
-                                    ? ColumnMetadata.staticColumn(current, columnName.bytes, type)
-                                    : ColumnMetadata.regularColumn(current, 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 (ViewMetadata view : views)
-                            if (view.includeAllColumns)
-                                viewUpdates.add(view.withAddedRegularColumn(ColumnMetadata.regularColumn(view.metadata, columnName.bytes, type)));
-
-                }
-                break;
-            case DROP:
-                if (!current.isCQLTable())
-                    throw new InvalidRequestException("Cannot drop columns from a non-CQL3 table");
-
-                for (AlterTableStatementColumn colData : colNameList)
-                {
-                    columnName = colData.getColumnName().getIdentifier(current);
-                    def = builder.getColumn(columnName);
-
-                    if (def == null)
-                        throw new InvalidRequestException(String.format("Column %s was not found in table %s", columnName, columnFamily()));
-
-                    switch (def.kind)
-                    {
-                         case PARTITION_KEY:
-                         case CLUSTERING:
-                              throw new InvalidRequestException(String.format("Cannot drop PRIMARY KEY part %s", columnName));
-                         case REGULAR:
-                         case STATIC:
-                             builder.removeRegularOrStaticColumn(def.name);
-                             builder.recordColumnDrop(def, 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 = current.indexes;
-                    if (!allIndexes.isEmpty())
-                    {
-                        ColumnFamilyStore store = Keyspace.openAndGetStore(current);
-                        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 %s with materialized views.",
-                                                                        columnName.toString(),
-                                                                        columnFamily()));
-                }
-                break;
-            case OPTS:
-                if (attrs == null)
-                    throw new InvalidRequestException("ALTER TABLE WITH invoked, but no parameters found");
-                attrs.validate();
-
-                TableParams params = attrs.asAlteredTableParams(current.params);
-
-                if (!Iterables.isEmpty(views) && params.gcGraceSeconds == 0)
-                {
-                    throw new InvalidRequestException("Cannot alter gc_grace_seconds of the base table of a " +
-                                                      "materialized view to 0, since this value is used to TTL " +
-                                                      "undelivered updates. Setting gc_grace_seconds too low might " +
-                                                      "cause undelivered updates to expire " +
-                                                      "before being replayed.");
-                }
-
-                if (current.isCounter() && params.defaultTimeToLive > 0)
-                    throw new InvalidRequestException("Cannot set default_time_to_live on a table with counters");
-
-                builder.params(params);
-
-                break;
-            case RENAME:
-                for (Map.Entry<ColumnMetadata.Raw, ColumnMetadata.Raw> entry : renames.entrySet())
-                {
-                    ColumnIdentifier from = entry.getKey().getIdentifier(current);
-                    ColumnIdentifier to = entry.getValue().getIdentifier(current);
-
-                    def = current.getColumn(from);
-                    if (def == null)
-                        throw new InvalidRequestException(String.format("Cannot rename unknown column %s in table %s", from, current.name));
-
-                    if (current.getColumn(to) != null)
-                        throw new InvalidRequestException(String.format("Cannot rename column %s to %s in table %s; another column of that name already exist", from, to, current.name));
-
-                    if (!def.isPrimaryKeyColumn())
-                        throw new InvalidRequestException(String.format("Cannot rename non PRIMARY KEY part %s", from));
-
-                    if (!current.indexes.isEmpty())
-                    {
-                        ColumnFamilyStore store = Keyspace.openAndGetStore(current);
-                        Set<IndexMetadata> dependentIndexes = store.indexManager.getDependentIndexes(def);
-                        if (!dependentIndexes.isEmpty())
-                            throw new InvalidRequestException(String.format("Cannot rename column %s because it has " +
-                                                                            "dependent secondary indexes (%s)",
-                                                                            from,
-                                                                            dependentIndexes.stream()
-                                                                                            .map(i -> i.name)
-                                                                                            .collect(Collectors.joining(","))));
-                    }
-
-                    builder.renamePrimaryKeyColumn(from, to);
-
-                    // If the view includes a renamed column, it must be renamed in the view table and the definition.
-                    for (ViewMetadata view : views)
-                    {
-                        if (!view.includes(from))
-                            continue;
-
-                        ColumnIdentifier viewFrom = entry.getKey().getIdentifier(view.metadata);
-                        ColumnIdentifier viewTo = entry.getValue().getIdentifier(view.metadata);
-                        viewUpdates.add(view.renamePrimaryKeyColumn(viewFrom, viewTo));
-                    }
-                }
-                break;
-        }
-
-        MigrationManager.announceTableUpdate(builder.build(), viewUpdates, isLocalOnly);
-        return new Event.SchemaChange(Event.SchemaChange.Change.UPDATED, Event.SchemaChange.Target.TABLE, keyspace(), columnFamily());
-    }
-
-    @Override
-    public String toString()
-    {
-        return String.format("AlterTableStatement(name=%s, type=%s)",
-                             cfName,
-                             oType);
-    }
-
-    @Override
-    public AuditLogContext getAuditLogContext()
-    {
-        return new AuditLogContext(AuditLogEntryType.ALTER_TABLE, keyspace(), cfName.getColumnFamily());
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/cql3/statements/AlterTableStatementColumn.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/AlterTableStatementColumn.java b/src/java/org/apache/cassandra/cql3/statements/AlterTableStatementColumn.java
deleted file mode 100644
index 97bf024..0000000
--- a/src/java/org/apache/cassandra/cql3/statements/AlterTableStatementColumn.java
+++ /dev/null
@@ -1,74 +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.statements;
-
-import org.apache.cassandra.schema.ColumnMetadata;
-import org.apache.cassandra.cql3.CQL3Type;
-import org.apache.commons.lang3.builder.ToStringBuilder;
-import org.apache.commons.lang3.builder.ToStringStyle;
-
-/**
- * Stores a column name and optionally type for an Alter Table statement definition.
- *
- * This is used by AlterTableStatement to store the added, altered or dropped columns.
- */
-public class AlterTableStatementColumn
-{
-    private final CQL3Type.Raw dataType;
-    private final ColumnMetadata.Raw colName;
-    private final Boolean isStatic;
-
-    public AlterTableStatementColumn(ColumnMetadata.Raw colName, CQL3Type.Raw dataType, boolean isStatic)
-    {
-        assert colName != null;
-        this.dataType = dataType; // will be null when dropping columns, and never null otherwise (for ADD and ALTER).
-        this.colName = colName;
-        this.isStatic = isStatic;
-    }
-
-    public AlterTableStatementColumn(ColumnMetadata.Raw colName, CQL3Type.Raw dataType)
-    {
-        this(colName, dataType, false);
-    }
-
-    public AlterTableStatementColumn(ColumnMetadata.Raw colName)
-    {
-        this(colName, null, false);
-    }
-
-    public CQL3Type.Raw getColumnType()
-    {
-        return dataType;
-    }
-
-    public ColumnMetadata.Raw getColumnName()
-    {
-        return colName;
-    }
-
-    public Boolean getStaticType()
-    {
-        return isStatic;
-    }
-    
-    @Override
-    public String toString()
-    {
-        return ToStringBuilder.reflectionToString(this, ToStringStyle.SHORT_PREFIX_STYLE);
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/cql3/statements/AlterTypeStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/AlterTypeStatement.java b/src/java/org/apache/cassandra/cql3/statements/AlterTypeStatement.java
deleted file mode 100644
index 3249af2..0000000
--- a/src/java/org/apache/cassandra/cql3/statements/AlterTypeStatement.java
+++ /dev/null
@@ -1,195 +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.statements;
-
-import java.util.*;
-
-import org.apache.cassandra.audit.AuditLogEntryType;
-import org.apache.cassandra.auth.Permission;
-import org.apache.cassandra.cql3.*;
-import org.apache.cassandra.db.marshal.*;
-import org.apache.cassandra.exceptions.*;
-import org.apache.cassandra.schema.KeyspaceMetadata;
-import org.apache.cassandra.schema.MigrationManager;
-import org.apache.cassandra.schema.Schema;
-import org.apache.cassandra.service.ClientState;
-import org.apache.cassandra.service.QueryState;
-import org.apache.cassandra.transport.Event;
-import org.apache.commons.lang3.builder.ToStringBuilder;
-import org.apache.commons.lang3.builder.ToStringStyle;
-
-public abstract class AlterTypeStatement extends SchemaAlteringStatement
-{
-    protected final UTName name;
-
-    protected AlterTypeStatement(UTName name)
-    {
-        this.name = name;
-    }
-
-    @Override
-    public void prepareKeyspace(ClientState state) throws InvalidRequestException
-    {
-        if (!name.hasKeyspace())
-            name.setKeyspace(state.getKeyspace());
-
-        if (name.getKeyspace() == null)
-            throw new InvalidRequestException("You need to be logged in a keyspace or use a fully qualified user type name");
-    }
-
-    protected abstract UserType makeUpdatedType(UserType toUpdate, KeyspaceMetadata ksm) throws InvalidRequestException;
-
-    public static AlterTypeStatement addition(UTName name, FieldIdentifier fieldName, CQL3Type.Raw type)
-    {
-        return new Add(name, fieldName, type);
-    }
-
-    public static AlterTypeStatement alter(UTName name, FieldIdentifier fieldName, CQL3Type.Raw type)
-    {
-        throw new InvalidRequestException("Altering of types is not allowed");
-    }
-
-    public static AlterTypeStatement renames(UTName name, Map<FieldIdentifier, FieldIdentifier> renames)
-    {
-        return new Renames(name, renames);
-    }
-
-    public void checkAccess(ClientState state) throws UnauthorizedException, InvalidRequestException
-    {
-        state.hasKeyspaceAccess(keyspace(), Permission.ALTER);
-    }
-
-    public void validate(ClientState state) throws RequestValidationException
-    {
-        // Validation is left to announceMigration as it's easier to do it while constructing the updated type.
-        // It doesn't really change anything anyway.
-    }
-
-    @Override
-    public String keyspace()
-    {
-        return name.getKeyspace();
-    }
-
-    public Event.SchemaChange announceMigration(QueryState queryState, boolean isLocalOnly) throws InvalidRequestException, ConfigurationException
-    {
-        KeyspaceMetadata ksm = Schema.instance.getKeyspaceMetadata(name.getKeyspace());
-        if (ksm == null)
-            throw new InvalidRequestException(String.format("Cannot alter type in unknown keyspace %s", name.getKeyspace()));
-
-        UserType toUpdate =
-            ksm.types.get(name.getUserTypeName())
-                     .orElseThrow(() -> new InvalidRequestException(String.format("No user type named %s exists.", name)));
-
-        UserType updated = makeUpdatedType(toUpdate, ksm);
-
-        // Now, we need to announce the type update to basically change it for new tables using this type,
-        // but we also need to find all existing user types and CF using it and change them.
-        MigrationManager.announceTypeUpdate(updated, isLocalOnly);
-
-        return new Event.SchemaChange(Event.SchemaChange.Change.UPDATED, Event.SchemaChange.Target.TYPE, keyspace(), name.getStringTypeName());
-    }
-
-    protected void checkTypeNotUsedByAggregate(KeyspaceMetadata ksm)
-    {
-        ksm.functions.udas().filter(aggregate -> aggregate.initialCondition() != null && aggregate.stateType().referencesUserType(name.getStringTypeName()))
-                     .findAny()
-                     .ifPresent((aggregate) -> {
-                         throw new InvalidRequestException(String.format("Cannot alter user type %s as it is still used as an INITCOND by aggregate %s", name, aggregate));
-                     });
-    }
-
-    private static class Add extends AlterTypeStatement
-    {
-        private final FieldIdentifier fieldName;
-        private final CQL3Type.Raw type;
-
-        public Add(UTName name, FieldIdentifier fieldName, CQL3Type.Raw type)
-        {
-            super(name);
-            this.fieldName = fieldName;
-            this.type = type;
-        }
-
-        protected UserType makeUpdatedType(UserType toUpdate, KeyspaceMetadata ksm) throws InvalidRequestException
-        {
-            if (toUpdate.fieldPosition(fieldName) >= 0)
-                throw new InvalidRequestException(String.format("Cannot add new field %s to type %s: a field of the same name already exists", fieldName, name));
-
-            List<FieldIdentifier> newNames = new ArrayList<>(toUpdate.size() + 1);
-            newNames.addAll(toUpdate.fieldNames());
-            newNames.add(fieldName);
-
-            AbstractType<?> addType = type.prepare(keyspace()).getType();
-            if (addType.referencesUserType(toUpdate.getNameAsString()))
-                throw new InvalidRequestException(String.format("Cannot add new field %s of type %s to type %s as this would create a circular reference", fieldName, type, name));
-
-            List<AbstractType<?>> newTypes = new ArrayList<>(toUpdate.size() + 1);
-            newTypes.addAll(toUpdate.fieldTypes());
-            newTypes.add(addType);
-
-            return new UserType(toUpdate.keyspace, toUpdate.name, newNames, newTypes, toUpdate.isMultiCell());
-        }
-    }
-
-    private static class Renames extends AlterTypeStatement
-    {
-        private final Map<FieldIdentifier, FieldIdentifier> renames;
-
-        public Renames(UTName name, Map<FieldIdentifier, FieldIdentifier> renames)
-        {
-            super(name);
-            this.renames = renames;
-        }
-
-        protected UserType makeUpdatedType(UserType toUpdate, KeyspaceMetadata ksm) throws InvalidRequestException
-        {
-            checkTypeNotUsedByAggregate(ksm);
-
-            List<FieldIdentifier> newNames = new ArrayList<>(toUpdate.fieldNames());
-            List<AbstractType<?>> newTypes = new ArrayList<>(toUpdate.fieldTypes());
-
-            for (Map.Entry<FieldIdentifier, FieldIdentifier> entry : renames.entrySet())
-            {
-                FieldIdentifier from = entry.getKey();
-                FieldIdentifier to = entry.getValue();
-                int idx = toUpdate.fieldPosition(from);
-                if (idx < 0)
-                    throw new InvalidRequestException(String.format("Unknown field %s in type %s", from, name));
-                newNames.set(idx, to);
-            }
-
-            UserType updated = new UserType(toUpdate.keyspace, toUpdate.name, newNames, newTypes, toUpdate.isMultiCell());
-            CreateTypeStatement.checkForDuplicateNames(updated);
-            return updated;
-        }
-
-    }
-    
-    @Override
-    public String toString()
-    {
-        return ToStringBuilder.reflectionToString(this, ToStringStyle.SHORT_PREFIX_STYLE);
-    }
-
-    @Override
-    public AuditLogContext getAuditLogContext()
-    {
-        return new AuditLogContext(AuditLogEntryType.ALTER_TYPE, keyspace(), name.getStringTypeName());
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/cql3/statements/AlterViewStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/AlterViewStatement.java b/src/java/org/apache/cassandra/cql3/statements/AlterViewStatement.java
deleted file mode 100644
index 938908c..0000000
--- a/src/java/org/apache/cassandra/cql3/statements/AlterViewStatement.java
+++ /dev/null
@@ -1,103 +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.statements;
-
-import org.apache.cassandra.audit.AuditLogEntryType;
-import org.apache.cassandra.auth.Permission;
-import org.apache.cassandra.cql3.CFName;
-import org.apache.cassandra.db.view.View;
-import org.apache.cassandra.exceptions.InvalidRequestException;
-import org.apache.cassandra.exceptions.RequestValidationException;
-import org.apache.cassandra.exceptions.UnauthorizedException;
-import org.apache.cassandra.schema.MigrationManager;
-import org.apache.cassandra.schema.Schema;
-import org.apache.cassandra.schema.TableMetadata;
-import org.apache.cassandra.schema.TableMetadataRef;
-import org.apache.cassandra.schema.TableParams;
-import org.apache.cassandra.schema.ViewMetadata;
-import org.apache.cassandra.service.ClientState;
-import org.apache.cassandra.service.QueryState;
-import org.apache.cassandra.transport.Event;
-
-public class AlterViewStatement extends SchemaAlteringStatement
-{
-    private final TableAttributes attrs;
-
-    public AlterViewStatement(CFName name, TableAttributes attrs)
-    {
-        super(name);
-        this.attrs = attrs;
-    }
-
-    public void checkAccess(ClientState state) throws UnauthorizedException, InvalidRequestException
-    {
-        TableMetadataRef baseTable = View.findBaseTable(keyspace(), columnFamily());
-        if (baseTable != null)
-            state.hasColumnFamilyAccess(keyspace(), baseTable.name, Permission.ALTER);
-    }
-
-    public void validate(ClientState state)
-    {
-        // validated in announceMigration()
-    }
-
-    public Event.SchemaChange announceMigration(QueryState queryState, boolean isLocalOnly) throws RequestValidationException
-    {
-        TableMetadata meta = Schema.instance.validateTable(keyspace(), columnFamily());
-        if (!meta.isView())
-            throw new InvalidRequestException("Cannot use ALTER MATERIALIZED VIEW on Table");
-
-        ViewMetadata current = Schema.instance.getView(keyspace(), columnFamily());
-
-        if (attrs == null)
-            throw new InvalidRequestException("ALTER MATERIALIZED VIEW WITH invoked, but no parameters found");
-
-        attrs.validate();
-
-        TableParams params = attrs.asAlteredTableParams(current.metadata.params);
-        if (params.gcGraceSeconds == 0)
-        {
-            throw new InvalidRequestException("Cannot alter gc_grace_seconds of a materialized view to 0, since this " +
-                                              "value is used to TTL undelivered updates. Setting gc_grace_seconds too " +
-                                              "low might cause undelivered updates to expire before being replayed.");
-        }
-
-        if (params.defaultTimeToLive > 0)
-        {
-            throw new InvalidRequestException("Cannot set or alter default_time_to_live for a materialized view. " +
-                                              "Data in a materialized view always expire at the same time than " +
-                                              "the corresponding data in the parent table.");
-        }
-
-        ViewMetadata updated = current.copy(current.metadata.unbuild().params(params).build());
-
-        MigrationManager.announceViewUpdate(updated, isLocalOnly);
-        return new Event.SchemaChange(Event.SchemaChange.Change.UPDATED, Event.SchemaChange.Target.TABLE, keyspace(), columnFamily());
-    }
-
-    public String toString()
-    {
-        return String.format("AlterViewStatement(name=%s)", cfName);
-    }
-
-    @Override
-    public AuditLogContext getAuditLogContext()
-    {
-        return new AuditLogContext(AuditLogEntryType.ALTER_VIEW, keyspace(), cfName.getColumnFamily());
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/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 0283009..a8cbaa7 100644
--- a/src/java/org/apache/cassandra/cql3/statements/AuthenticationStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/AuthenticationStatement.java
@@ -28,17 +28,11 @@ import org.apache.cassandra.service.ClientState;
 import org.apache.cassandra.service.QueryState;
 import org.apache.cassandra.transport.messages.ResultMessage;
 
-public abstract class AuthenticationStatement extends ParsedStatement implements CQLStatement
+public abstract class AuthenticationStatement extends CQLStatement.Raw implements CQLStatement
 {
-    @Override
-    public Prepared prepare()
+    public AuthenticationStatement prepare(ClientState state)
     {
-        return new Prepared(this);
-    }
-
-    public int getBoundTerms()
-    {
-        return 0;
+        return this;
     }
 
     public ResultMessage execute(QueryState state, QueryOptions options, long queryStartNanoTime)
@@ -49,9 +43,9 @@ public abstract class AuthenticationStatement extends ParsedStatement implements
 
     public abstract ResultMessage execute(ClientState state) throws RequestExecutionException, RequestValidationException;
 
-    public ResultMessage executeInternal(QueryState state, QueryOptions options)
+    public ResultMessage executeLocally(QueryState state, QueryOptions options)
     {
-        // executeInternal is for local query only, thus altering users doesn't make sense and is not supported
+        // executeLocally is for local query only, thus altering users doesn't make sense and is not supported
         throw new UnsupportedOperationException();
     }
 
@@ -59,7 +53,7 @@ public abstract class AuthenticationStatement extends ParsedStatement implements
     {
         try
         {
-            state.ensureHasPermission(required, resource);
+            state.ensurePermission(required, resource);
         }
         catch (UnauthorizedException e)
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/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 8c6f93b..46285c6 100644
--- a/src/java/org/apache/cassandra/cql3/statements/AuthorizationStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/AuthorizationStatement.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.cql3.statements;
 
-
 import org.apache.cassandra.auth.DataResource;
 import org.apache.cassandra.auth.IResource;
 import org.apache.cassandra.cql3.CQLStatement;
@@ -31,17 +30,11 @@ import org.apache.cassandra.transport.messages.ResultMessage;
 import org.apache.commons.lang3.builder.ToStringBuilder;
 import org.apache.commons.lang3.builder.ToStringStyle;
 
-public abstract class AuthorizationStatement extends ParsedStatement implements CQLStatement
+public abstract class AuthorizationStatement extends CQLStatement.Raw implements CQLStatement
 {
-    @Override
-    public Prepared prepare()
-    {
-        return new Prepared(this);
-    }
-
-    public int getBoundTerms()
+    public AuthorizationStatement prepare(ClientState state)
     {
-        return 0;
+        return this;
     }
 
     public ResultMessage execute(QueryState state, QueryOptions options, long queryStartNanoTime)
@@ -52,9 +45,9 @@ public abstract class AuthorizationStatement extends ParsedStatement implements
 
     public abstract ResultMessage execute(ClientState state) throws RequestValidationException, RequestExecutionException;
 
-    public ResultMessage executeInternal(QueryState state, QueryOptions options)
+    public ResultMessage executeLocally(QueryState state, QueryOptions options)
     {
-        // executeInternal is for local query only, thus altering permission doesn't make sense and is not supported
+        // executeLocally is for local query only, thus altering permission doesn't make sense and is not supported
         throw new UnsupportedOperationException();
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/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 a71c799..9ed150c 100644
--- a/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
@@ -27,6 +27,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.slf4j.helpers.MessageFormatter;
 
+import org.apache.cassandra.audit.AuditLogContext;
 import org.apache.cassandra.audit.AuditLogEntryType;
 import org.apache.cassandra.schema.TableId;
 import org.apache.cassandra.schema.TableMetadata;
@@ -45,6 +46,8 @@ import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.NoSpamLogger;
 import org.apache.cassandra.utils.Pair;
 
+import static java.util.function.Predicate.isEqual;
+
 import static org.apache.cassandra.cql3.statements.RequestValidations.checkFalse;
 
 /**
@@ -57,8 +60,8 @@ public class BatchStatement implements CQLStatement
         LOGGED, UNLOGGED, COUNTER
     }
 
-    private final int boundTerms;
     public final Type type;
+    private final VariableSpecifications bindVariables;
     private final List<ModificationStatement> statements;
 
     // Columns modified for each table (keyed by the table ID)
@@ -93,10 +96,10 @@ public class BatchStatement implements CQLStatement
      * @param statements the list of statements in the batch
      * @param attrs      additional attributes for statement (CL, timestamp, timeToLive)
      */
-    public BatchStatement(int boundTerms, Type type, List<ModificationStatement> statements, Attributes attrs)
+    public BatchStatement(Type type, VariableSpecifications bindVariables, List<ModificationStatement> statements, Attributes attrs)
     {
-        this.boundTerms = boundTerms;
         this.type = type;
+        this.bindVariables = bindVariables;
         this.statements = statements;
         this.attrs = attrs;
 
@@ -128,6 +131,26 @@ public class BatchStatement implements CQLStatement
         this.updatesVirtualTables = updatesVirtualTables;
     }
 
+    @Override
+    public List<ColumnSpecification> getBindVariables()
+    {
+        return bindVariables.getBindVariables();
+    }
+
+    @Override
+    public short[] getPartitionKeyBindVariableIndexes()
+    {
+        boolean affectsMultipleTables =
+            !statements.isEmpty() && statements.stream().map(s -> s.metadata().id).allMatch(isEqual(statements.get(0).metadata().id));
+
+        // Use the TableMetadata of the first statement for partition key bind indexes.  If the statements affect
+        // multiple tables, we won't send partition key bind indexes.
+        return (affectsMultipleTables || statements.isEmpty())
+             ? null
+             : bindVariables.getPartitionKeyBindVariableIndexes(statements.get(0).metadata());
+    }
+
+    @Override
     public Iterable<org.apache.cassandra.cql3.functions.Function> getFunctions()
     {
         List<org.apache.cassandra.cql3.functions.Function> functions = new ArrayList<>();
@@ -136,15 +159,10 @@ public class BatchStatement implements CQLStatement
         return functions;
     }
 
-    public int getBoundTerms()
-    {
-        return boundTerms;
-    }
-
-    public void checkAccess(ClientState state) throws InvalidRequestException, UnauthorizedException
+    public void authorize(ClientState state) throws InvalidRequestException, UnauthorizedException
     {
         for (ModificationStatement statement : statements)
-            statement.checkAccess(state);
+            statement.authorize(state);
     }
 
     // Validates a prepared batch statement without validating its nested statements.
@@ -506,7 +524,7 @@ public class BatchStatement implements CQLStatement
         return hasConditions;
     }
 
-    public ResultMessage executeInternal(QueryState queryState, QueryOptions options) throws RequestValidationException, RequestExecutionException
+    public ResultMessage executeLocally(QueryState queryState, QueryOptions options) throws RequestValidationException, RequestExecutionException
     {
         BatchQueryOptions batchOptions = BatchQueryOptions.withoutPerStatementVariables(options);
 
@@ -544,7 +562,7 @@ public class BatchStatement implements CQLStatement
         return String.format("BatchStatement(type=%s, statements=%s)", type, statements);
     }
 
-    public static class Parsed extends CFStatement
+    public static class Parsed extends QualifiedStatement
     {
         private final Type type;
         private final Attributes.Raw attrs;
@@ -559,48 +577,24 @@ public class BatchStatement implements CQLStatement
         }
 
         @Override
-        public void prepareKeyspace(ClientState state) throws InvalidRequestException
+        public void setKeyspace(ClientState state) throws InvalidRequestException
         {
             for (ModificationStatement.Parsed statement : parsedStatements)
-                statement.prepareKeyspace(state);
+                statement.setKeyspace(state);
         }
 
-        public ParsedStatement.Prepared prepare() throws InvalidRequestException
+        public BatchStatement prepare(ClientState state)
         {
-            VariableSpecifications boundNames = getBoundVariables();
-
-            String firstKS = null;
-            String firstCF = null;
-            boolean haveMultipleCFs = false;
-
             List<ModificationStatement> statements = new ArrayList<>(parsedStatements.size());
-            for (ModificationStatement.Parsed parsed : parsedStatements)
-            {
-                if (firstKS == null)
-                {
-                    firstKS = parsed.keyspace();
-                    firstCF = parsed.columnFamily();
-                }
-                else if (!haveMultipleCFs)
-                {
-                    haveMultipleCFs = !firstKS.equals(parsed.keyspace()) || !firstCF.equals(parsed.columnFamily());
-                }
-
-                statements.add(parsed.prepare(boundNames));
-            }
+            parsedStatements.forEach(s -> statements.add(s.prepare(bindVariables)));
 
             Attributes prepAttrs = attrs.prepare("[batch]", "[batch]");
-            prepAttrs.collectMarkerSpecification(boundNames);
+            prepAttrs.collectMarkerSpecification(bindVariables);
 
-            BatchStatement batchStatement = new BatchStatement(boundNames.size(), type, statements, prepAttrs);
+            BatchStatement batchStatement = new BatchStatement(type, bindVariables, statements, prepAttrs);
             batchStatement.validate();
 
-            // Use the TableMetadata of the first statement for partition key bind indexes.  If the statements affect
-            // multiple tables, we won't send partition key bind indexes.
-            short[] partitionKeyBindIndexes = (haveMultipleCFs || batchStatement.statements.isEmpty())? null
-                                                              : boundNames.getPartitionKeyBindIndexes(batchStatement.statements.get(0).metadata());
-
-            return new ParsedStatement.Prepared(batchStatement, boundNames, partitionKeyBindIndexes);
+            return batchStatement;
         }
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/cql3/statements/CFProperties.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/CFProperties.java b/src/java/org/apache/cassandra/cql3/statements/CFProperties.java
deleted file mode 100644
index 3f40c0d..0000000
--- a/src/java/org/apache/cassandra/cql3/statements/CFProperties.java
+++ /dev/null
@@ -1,64 +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.statements;
-
-import java.util.LinkedHashMap;
-import java.util.Map;
-
-import org.apache.cassandra.cql3.ColumnIdentifier;
-import org.apache.cassandra.db.marshal.AbstractType;
-import org.apache.cassandra.db.marshal.ReversedType;
-import org.apache.commons.lang3.builder.ToStringBuilder;
-import org.apache.commons.lang3.builder.ToStringStyle;
-
-public class CFProperties
-{
-    public final TableAttributes properties = new TableAttributes();
-    final Map<ColumnIdentifier, Boolean> definedOrdering = new LinkedHashMap<>(); // Insertion ordering is important
-    boolean useCompactStorage = false;
-
-    public void validate()
-    {
-        properties.validate();
-    }
-
-    public void setOrdering(ColumnIdentifier alias, boolean reversed)
-    {
-        definedOrdering.put(alias, reversed);
-    }
-
-    public void setCompactStorage()
-    {
-        useCompactStorage = true;
-    }
-
-    public AbstractType getReversableType(ColumnIdentifier targetIdentifier, AbstractType<?> type)
-    {
-        if (!definedOrdering.containsKey(targetIdentifier))
-        {
-            return type;
-        }
-        return definedOrdering.get(targetIdentifier) ? ReversedType.getInstance(type) : type;
-    }
-    
-    @Override
-    public String toString()
-    {
-        return ToStringBuilder.reflectionToString(this, ToStringStyle.SHORT_PREFIX_STYLE);
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/src/java/org/apache/cassandra/cql3/statements/CFStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/CFStatement.java b/src/java/org/apache/cassandra/cql3/statements/CFStatement.java
deleted file mode 100644
index 7cc8a99..0000000
--- a/src/java/org/apache/cassandra/cql3/statements/CFStatement.java
+++ /dev/null
@@ -1,74 +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.statements;
-
-import org.apache.cassandra.cql3.CFName;
-import org.apache.cassandra.service.ClientState;
-import org.apache.cassandra.exceptions.InvalidRequestException;
-import org.apache.commons.lang3.builder.ToStringBuilder;
-import org.apache.commons.lang3.builder.ToStringStyle;
-
-/**
- * Abstract class for statements that apply on a given column family.
- */
-public abstract class CFStatement extends ParsedStatement
-{
-    protected final CFName cfName;
-
-    protected CFStatement(CFName cfName)
-    {
-        this.cfName = cfName;
-    }
-
-    public void prepareKeyspace(ClientState state) throws InvalidRequestException
-    {
-        if (!cfName.hasKeyspace())
-        {
-            // XXX: We explicitly only want to call state.getKeyspace() in this case, as we don't want to throw
-            // if not logged in any keyspace but a keyspace is explicitly set on the statement. So don't move
-            // the call outside the 'if' or replace the method by 'prepareKeyspace(state.getKeyspace())'
-            cfName.setKeyspace(state.getKeyspace(), true);
-        }
-    }
-
-    // Only for internal calls, use the version with ClientState for user queries. In particular, the
-    // version with ClientState throws an exception if the statement does not have keyspace set *and*
-    // ClientState has no keyspace.
-    public void prepareKeyspace(String keyspace)
-    {
-        if (!cfName.hasKeyspace())
-            cfName.setKeyspace(keyspace, true);
-    }
-
-    public String keyspace()
-    {
-        assert cfName.hasKeyspace() : "The statement hasn't been prepared correctly";
-        return cfName.getKeyspace();
-    }
-
-    public String columnFamily()
-    {
-        return cfName.getColumnFamily();
-    }
-    
-    @Override
-    public String toString()
-    {
-        return ToStringBuilder.reflectionToString(this, ToStringStyle.SHORT_PREFIX_STYLE);
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/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
deleted file mode 100644
index e428087..0000000
--- a/src/java/org/apache/cassandra/cql3/statements/CreateAggregateStatement.java
+++ /dev/null
@@ -1,281 +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.statements;
-
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.Objects;
-import java.util.List;
-
-import org.apache.cassandra.audit.AuditLogEntryType;
-import org.apache.cassandra.auth.*;
-import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.schema.KeyspaceMetadata;
-import org.apache.cassandra.schema.Schema;
-import org.apache.cassandra.cql3.*;
-import org.apache.cassandra.cql3.functions.*;
-import org.apache.cassandra.db.marshal.AbstractType;
-import org.apache.cassandra.exceptions.*;
-import org.apache.cassandra.serializers.MarshalException;
-import org.apache.cassandra.service.ClientState;
-import org.apache.cassandra.schema.MigrationManager;
-import org.apache.cassandra.service.QueryState;
-import org.apache.cassandra.transport.Event;
-import org.apache.cassandra.transport.ProtocolVersion;
-import org.apache.commons.lang3.builder.ToStringBuilder;
-import org.apache.commons.lang3.builder.ToStringStyle;
-
-/**
- * A {@code CREATE AGGREGATE} statement parsed from a CQL query.
- */
-public final class CreateAggregateStatement extends SchemaAlteringStatement
-{
-    private final boolean orReplace;
-    private final boolean ifNotExists;
-    private FunctionName functionName;
-    private FunctionName stateFunc;
-    private FunctionName finalFunc;
-    private final CQL3Type.Raw stateTypeRaw;
-
-    private final List<CQL3Type.Raw> argRawTypes;
-    private final Term.Raw ival;
-
-    private List<AbstractType<?>> argTypes;
-    private AbstractType<?> returnType;
-    private ScalarFunction stateFunction;
-    private ScalarFunction finalFunction;
-    private ByteBuffer initcond;
-
-    public CreateAggregateStatement(FunctionName functionName,
-                                    List<CQL3Type.Raw> argRawTypes,
-                                    String stateFunc,
-                                    CQL3Type.Raw stateType,
-                                    String finalFunc,
-                                    Term.Raw ival,
-                                    boolean orReplace,
-                                    boolean ifNotExists)
-    {
-        this.functionName = functionName;
-        this.argRawTypes = argRawTypes;
-        this.stateFunc = new FunctionName(functionName.keyspace, stateFunc);
-        this.finalFunc = finalFunc != null ? new FunctionName(functionName.keyspace, finalFunc) : null;
-        this.stateTypeRaw = stateType;
-        this.ival = ival;
-        this.orReplace = orReplace;
-        this.ifNotExists = ifNotExists;
-    }
-
-    public Prepared prepare()
-    {
-        argTypes = new ArrayList<>(argRawTypes.size());
-        for (CQL3Type.Raw rawType : argRawTypes)
-            argTypes.add(prepareType("arguments", rawType));
-
-        AbstractType<?> stateType = prepareType("state type", stateTypeRaw);
-
-        List<AbstractType<?>> stateArgs = stateArguments(stateType, argTypes);
-
-        Function f = Schema.instance.findFunction(stateFunc, stateArgs).orElse(null);
-        if (!(f instanceof ScalarFunction))
-            throw new InvalidRequestException("State function " + stateFuncSig(stateFunc, stateTypeRaw, argRawTypes) + " does not exist or is not a scalar function");
-        stateFunction = (ScalarFunction)f;
-
-        AbstractType<?> stateReturnType = stateFunction.returnType();
-        if (!stateReturnType.equals(stateType))
-            throw new InvalidRequestException("State function " + stateFuncSig(stateFunction.name(), stateTypeRaw, argRawTypes) + " return type must be the same as the first argument type - check STYPE, argument and return types");
-
-        if (finalFunc != null)
-        {
-            List<AbstractType<?>> finalArgs = Collections.<AbstractType<?>>singletonList(stateType);
-            f = Schema.instance.findFunction(finalFunc, finalArgs).orElse(null);
-            if (!(f instanceof ScalarFunction))
-                throw new InvalidRequestException("Final function " + finalFunc + '(' + stateTypeRaw + ") does not exist or is not a scalar function");
-            finalFunction = (ScalarFunction) f;
-            returnType = finalFunction.returnType();
-        }
-        else
-        {
-            returnType = stateReturnType;
-        }
-
-        if (ival != null)
-        {
-            initcond = Terms.asBytes(functionName.keyspace, ival.toString(), stateType);
-
-            if (initcond != null)
-            {
-                try
-                {
-                    stateType.validate(initcond);
-                }
-                catch (MarshalException e)
-                {
-                    throw new InvalidRequestException(String.format("Invalid value for INITCOND of type %s%s", stateType.asCQL3Type(),
-                                                                    e.getMessage() == null ? "" : String.format(" (%s)", e.getMessage())));
-                }
-            }
-
-            // Sanity check that converts the initcond to a CQL literal and parse it back to avoid getting in CASSANDRA-11064.
-            String initcondAsCql = stateType.asCQL3Type().toCQLLiteral(initcond, ProtocolVersion.CURRENT);
-            assert Objects.equals(initcond, Terms.asBytes(functionName.keyspace, initcondAsCql, stateType));
-
-            if (Constants.NULL_LITERAL != ival && UDHelper.isNullOrEmpty(stateType, initcond))
-                throw new InvalidRequestException("INITCOND must not be empty for all types except TEXT, ASCII, BLOB");
-        }
-
-        return super.prepare();
-    }
-
-    private AbstractType<?> prepareType(String typeName, CQL3Type.Raw rawType)
-    {
-        if (rawType.isFrozen())
-            throw new InvalidRequestException(String.format("The function %s should not be frozen; remove the frozen<> modifier", typeName));
-
-        // UDT are not supported non frozen but we do not allow the frozen keyword for argument. So for the moment we
-        // freeze them here
-        if (!rawType.canBeNonFrozen())
-            rawType.freeze();
-
-        AbstractType<?> type = rawType.prepare(functionName.keyspace).getType();
-        return type;
-    }
-
-    public void prepareKeyspace(ClientState state) throws InvalidRequestException
-    {
-        if (!functionName.hasKeyspace() && state.getRawKeyspace() != null)
-            functionName = new FunctionName(state.getKeyspace(), functionName.name);
-
-        if (!functionName.hasKeyspace())
-            throw new InvalidRequestException("Functions must be fully qualified with a keyspace name if a keyspace is not set for the session");
-
-        Schema.validateKeyspaceNotSystem(functionName.keyspace);
-
-        stateFunc = new FunctionName(functionName.keyspace, stateFunc.name);
-        if (finalFunc != null)
-            finalFunc = new FunctionName(functionName.keyspace, finalFunc.name);
-    }
-
-    protected void grantPermissionsToCreator(QueryState state)
-    {
-        try
-        {
-            IResource resource = FunctionResource.function(functionName.keyspace, functionName.name, argTypes);
-            DatabaseDescriptor.getAuthorizer().grant(AuthenticatedUser.SYSTEM_USER,
-                                                     resource.applicablePermissions(),
-                                                     resource,
-                                                     RoleResource.role(state.getClientState().getUser().getName()));
-        }
-        catch (RequestExecutionException e)
-        {
-            throw new RuntimeException(e);
-        }
-    }
-
-    public void checkAccess(ClientState state) throws UnauthorizedException, InvalidRequestException
-    {
-        if (Schema.instance.findFunction(functionName, argTypes).isPresent() && orReplace)
-            state.ensureHasPermission(Permission.ALTER, FunctionResource.function(functionName.keyspace,
-                                                                                  functionName.name,
-                                                                                  argTypes));
-        else
-            state.ensureHasPermission(Permission.CREATE, FunctionResource.keyspace(functionName.keyspace));
-
-        state.ensureHasPermission(Permission.EXECUTE, stateFunction);
-
-        if (finalFunction != null)
-            state.ensureHasPermission(Permission.EXECUTE, finalFunction);
-    }
-
-    public void validate(ClientState state) throws InvalidRequestException
-    {
-        if (ifNotExists && orReplace)
-            throw new InvalidRequestException("Cannot use both 'OR REPLACE' and 'IF NOT EXISTS' directives");
-
-
-        KeyspaceMetadata ksm = Schema.instance.getKeyspaceMetadata(functionName.keyspace);
-        if (null == ksm)
-            throw new InvalidRequestException(String.format("Cannot add aggregate '%s' to non existing keyspace '%s'.", functionName.name, functionName.keyspace));
-        if (ksm.isVirtual())
-            throw new InvalidRequestException("Cannot create aggregates in virtual keyspaces");
-    }
-
-    public Event.SchemaChange announceMigration(QueryState queryState, boolean isLocalOnly) throws RequestValidationException
-    {
-        Function old = Schema.instance.findFunction(functionName, argTypes).orElse(null);
-        boolean replaced = old != null;
-        if (replaced)
-        {
-            if (ifNotExists)
-                return null;
-            if (!orReplace)
-                throw new InvalidRequestException(String.format("Function %s already exists", old));
-            if (!(old instanceof AggregateFunction))
-                throw new InvalidRequestException(String.format("Aggregate %s can only replace an aggregate", old));
-
-            // Means we're replacing the function. We still need to validate that 1) it's not a native function and 2) that the return type
-            // matches (or that could break existing code badly)
-            if (old.isNative())
-                throw new InvalidRequestException(String.format("Cannot replace native aggregate %s", old));
-            if (!old.returnType().isValueCompatibleWith(returnType))
-                throw new InvalidRequestException(String.format("Cannot replace aggregate %s, the new return type %s is not compatible with the return type %s of existing function",
-                                                                functionName, returnType.asCQL3Type(), old.returnType().asCQL3Type()));
-        }
-
-        if (!stateFunction.isCalledOnNullInput() && initcond == null)
-            throw new InvalidRequestException(String.format("Cannot create aggregate %s without INITCOND because state function %s does not accept 'null' arguments", functionName, stateFunc));
-
-        UDAggregate udAggregate = new UDAggregate(functionName, argTypes, returnType, stateFunction, finalFunction, initcond);
-
-        MigrationManager.announceNewAggregate(udAggregate, isLocalOnly);
-
-        return new Event.SchemaChange(replaced ? Event.SchemaChange.Change.UPDATED : Event.SchemaChange.Change.CREATED,
-                                      Event.SchemaChange.Target.AGGREGATE,
-                                      udAggregate.name().keyspace, udAggregate.name().name, AbstractType.asCQLTypeStringList(udAggregate.argTypes()));
-    }
-
-    private static String stateFuncSig(FunctionName stateFuncName, CQL3Type.Raw stateTypeRaw, List<CQL3Type.Raw> argRawTypes)
-    {
-        StringBuilder sb = new StringBuilder();
-        sb.append(stateFuncName.toString()).append('(').append(stateTypeRaw);
-        for (CQL3Type.Raw argRawType : argRawTypes)
-            sb.append(", ").append(argRawType);
-        sb.append(')');
-        return sb.toString();
-    }
-
-    private static List<AbstractType<?>> stateArguments(AbstractType<?> stateType, List<AbstractType<?>> argTypes)
-    {
-        List<AbstractType<?>> r = new ArrayList<>(argTypes.size() + 1);
-        r.add(stateType);
-        r.addAll(argTypes);
-        return r;
-    }
-    
-    @Override
-    public String toString()
-    {
-        return ToStringBuilder.reflectionToString(this, ToStringStyle.SHORT_PREFIX_STYLE);
-    }
-
-    @Override
-    public AuditLogContext getAuditLogContext()
-    {
-        return new AuditLogContext(AuditLogEntryType.CREATE_AGGREGATE, keyspace(), functionName.name);
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/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
deleted file mode 100644
index c380991..0000000
--- a/src/java/org/apache/cassandra/cql3/statements/CreateFunctionStatement.java
+++ /dev/null
@@ -1,202 +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.statements;
-
-import java.util.ArrayList;
-import java.util.HashSet;
-import java.util.List;
-
-import org.apache.cassandra.audit.AuditLogEntryType;
-import org.apache.cassandra.auth.*;
-import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.schema.KeyspaceMetadata;
-import org.apache.cassandra.schema.Schema;
-import org.apache.cassandra.cql3.CQL3Type;
-import org.apache.cassandra.cql3.ColumnIdentifier;
-import org.apache.cassandra.cql3.functions.*;
-import org.apache.cassandra.db.marshal.AbstractType;
-import org.apache.cassandra.exceptions.*;
-import org.apache.cassandra.schema.Functions;
-import org.apache.cassandra.service.ClientState;
-import org.apache.cassandra.schema.MigrationManager;
-import org.apache.cassandra.service.QueryState;
-import org.apache.cassandra.transport.Event;
-import org.apache.commons.lang3.builder.ToStringBuilder;
-import org.apache.commons.lang3.builder.ToStringStyle;
-
-/**
- * A {@code CREATE FUNCTION} statement parsed from a CQL query.
- */
-public final class CreateFunctionStatement extends SchemaAlteringStatement
-{
-    private final boolean orReplace;
-    private final boolean ifNotExists;
-    private FunctionName functionName;
-    private final String language;
-    private final String body;
-
-    private final List<ColumnIdentifier> argNames;
-    private final List<CQL3Type.Raw> argRawTypes;
-    private final CQL3Type.Raw rawReturnType;
-    private final boolean calledOnNullInput;
-
-    private List<AbstractType<?>> argTypes;
-    private AbstractType<?> returnType;
-
-    public CreateFunctionStatement(FunctionName functionName,
-                                   String language,
-                                   String body,
-                                   List<ColumnIdentifier> argNames,
-                                   List<CQL3Type.Raw> argRawTypes,
-                                   CQL3Type.Raw rawReturnType,
-                                   boolean calledOnNullInput,
-                                   boolean orReplace,
-                                   boolean ifNotExists)
-    {
-        this.functionName = functionName;
-        this.language = language;
-        this.body = body;
-        this.argNames = argNames;
-        this.argRawTypes = argRawTypes;
-        this.rawReturnType = rawReturnType;
-        this.calledOnNullInput = calledOnNullInput;
-        this.orReplace = orReplace;
-        this.ifNotExists = ifNotExists;
-    }
-
-    public Prepared prepare() 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",
-                                                            functionName, argNames));
-
-        argTypes = new ArrayList<>(argRawTypes.size());
-        for (CQL3Type.Raw rawType : argRawTypes)
-            argTypes.add(prepareType("arguments", rawType));
-
-        returnType = prepareType("return type", rawReturnType);
-        return super.prepare();
-    }
-
-    public void prepareKeyspace(ClientState state) throws InvalidRequestException
-    {
-        if (!functionName.hasKeyspace() && state.getRawKeyspace() != null)
-            functionName = new FunctionName(state.getRawKeyspace(), functionName.name);
-
-        if (!functionName.hasKeyspace())
-            throw new InvalidRequestException("Functions must be fully qualified with a keyspace name if a keyspace is not set for the session");
-
-        Schema.validateKeyspaceNotSystem(functionName.keyspace);
-    }
-
-    protected void grantPermissionsToCreator(QueryState state)
-    {
-        try
-        {
-            IResource resource = FunctionResource.function(functionName.keyspace, functionName.name, argTypes);
-            DatabaseDescriptor.getAuthorizer().grant(AuthenticatedUser.SYSTEM_USER,
-                                                     resource.applicablePermissions(),
-                                                     resource,
-                                                     RoleResource.role(state.getClientState().getUser().getName()));
-        }
-        catch (RequestExecutionException e)
-        {
-            throw new RuntimeException(e);
-        }
-    }
-
-    public void checkAccess(ClientState state) throws UnauthorizedException, InvalidRequestException
-    {
-        if (Schema.instance.findFunction(functionName, argTypes).isPresent() && orReplace)
-            state.ensureHasPermission(Permission.ALTER, FunctionResource.function(functionName.keyspace,
-                                                                                  functionName.name,
-                                                                                  argTypes));
-        else
-            state.ensureHasPermission(Permission.CREATE, FunctionResource.keyspace(functionName.keyspace));
-    }
-
-    public void validate(ClientState state) throws InvalidRequestException
-    {
-        UDFunction.assertUdfsEnabled(language);
-
-        if (ifNotExists && orReplace)
-            throw new InvalidRequestException("Cannot use both 'OR REPLACE' and 'IF NOT EXISTS' directives");
-
-        KeyspaceMetadata ksm = Schema.instance.getKeyspaceMetadata(functionName.keyspace);
-        if (null == ksm)
-            throw new InvalidRequestException(String.format("Cannot add function '%s' to non existing keyspace '%s'.", functionName.name, functionName.keyspace));
-        if (ksm.isVirtual())
-            throw new InvalidRequestException("Cannot create functions in virtual keyspaces");
-    }
-
-    public Event.SchemaChange announceMigration(QueryState queryState, boolean isLocalOnly) throws RequestValidationException
-    {
-        Function old = Schema.instance.findFunction(functionName, argTypes).orElse(null);
-        boolean replaced = old != null;
-        if (replaced)
-        {
-            if (ifNotExists)
-                return null;
-            if (!orReplace)
-                throw new InvalidRequestException(String.format("Function %s already exists", old));
-            if (!(old instanceof ScalarFunction))
-                throw new InvalidRequestException(String.format("Function %s can only replace a function", old));
-            if (calledOnNullInput != ((ScalarFunction) old).isCalledOnNullInput())
-                throw new InvalidRequestException(String.format("Function %s can only be replaced with %s", old,
-                                                                calledOnNullInput ? "CALLED ON NULL INPUT" : "RETURNS NULL ON NULL INPUT"));
-
-            if (!Functions.typesMatch(old.returnType(), returnType))
-                throw new InvalidRequestException(String.format("Cannot replace function %s, the new return type %s is not compatible with the return type %s of existing function",
-                                                                functionName, returnType.asCQL3Type(), old.returnType().asCQL3Type()));
-        }
-
-        UDFunction udFunction = UDFunction.create(functionName, argNames, argTypes, returnType, calledOnNullInput, language, body);
-
-        MigrationManager.announceNewFunction(udFunction, isLocalOnly);
-
-        return new Event.SchemaChange(replaced ? Event.SchemaChange.Change.UPDATED : Event.SchemaChange.Change.CREATED,
-                                      Event.SchemaChange.Target.FUNCTION,
-                                      udFunction.name().keyspace, udFunction.name().name, AbstractType.asCQLTypeStringList(udFunction.argTypes()));
-    }
-
-    private AbstractType<?> prepareType(String typeName, CQL3Type.Raw rawType)
-    {
-        if (rawType.isFrozen())
-            throw new InvalidRequestException(String.format("The function %s should not be frozen; remove the frozen<> modifier", typeName));
-
-        // UDT are not supported non frozen but we do not allow the frozen keyword for argument. So for the moment we
-        // freeze them here
-        if (!rawType.canBeNonFrozen())
-            rawType.freeze();
-
-        AbstractType<?> type = rawType.prepare(functionName.keyspace).getType();
-        return type;
-    }
-    
-    @Override
-    public String toString()
-    {
-        return ToStringBuilder.reflectionToString(this, ToStringStyle.SHORT_PREFIX_STYLE);
-    }
-
-    @Override
-    public AuditLogContext getAuditLogContext()
-    {
-        return new AuditLogContext(AuditLogEntryType.CREATE_FUNCTION, functionName.keyspace, functionName.name);
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/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
deleted file mode 100644
index 778c4a3..0000000
--- a/src/java/org/apache/cassandra/cql3/statements/CreateIndexStatement.java
+++ /dev/null
@@ -1,275 +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.statements;
-
-import java.util.*;
-
-import com.google.common.base.Optional;
-import com.google.common.base.Strings;
-import com.google.common.collect.Iterables;
-import com.google.common.collect.Sets;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.cassandra.audit.AuditLogEntryType;
-import org.apache.cassandra.auth.Permission;
-import org.apache.cassandra.cql3.CFName;
-import org.apache.cassandra.cql3.ColumnIdentifier;
-import org.apache.cassandra.cql3.IndexName;
-import org.apache.cassandra.db.marshal.MapType;
-import org.apache.cassandra.exceptions.InvalidRequestException;
-import org.apache.cassandra.exceptions.RequestValidationException;
-import org.apache.cassandra.exceptions.UnauthorizedException;
-import org.apache.cassandra.schema.ColumnMetadata;
-import org.apache.cassandra.schema.IndexMetadata;
-import org.apache.cassandra.schema.Indexes;
-import org.apache.cassandra.schema.MigrationManager;
-import org.apache.cassandra.schema.Schema;
-import org.apache.cassandra.schema.TableMetadata;
-import org.apache.cassandra.service.ClientState;
-import org.apache.cassandra.service.QueryState;
-import org.apache.cassandra.transport.Event;
-import org.apache.commons.lang3.builder.ToStringBuilder;
-import org.apache.commons.lang3.builder.ToStringStyle;
-
-import static org.apache.cassandra.cql3.statements.RequestValidations.checkFalse;
-import static org.apache.cassandra.cql3.statements.RequestValidations.invalidRequest;
-
-/** A <code>CREATE INDEX</code> statement parsed from a CQL query. */
-public class CreateIndexStatement extends SchemaAlteringStatement
-{
-    private static final Logger logger = LoggerFactory.getLogger(CreateIndexStatement.class);
-
-    private final String indexName;
-    private final List<IndexTarget.Raw> rawTargets;
-    private final IndexPropDefs properties;
-    private final boolean ifNotExists;
-
-    public CreateIndexStatement(CFName name,
-                                IndexName indexName,
-                                List<IndexTarget.Raw> targets,
-                                IndexPropDefs properties,
-                                boolean ifNotExists)
-    {
-        super(name);
-        this.indexName = indexName.getIdx();
-        this.rawTargets = targets;
-        this.properties = properties;
-        this.ifNotExists = ifNotExists;
-    }
-
-    public void checkAccess(ClientState state) throws UnauthorizedException, InvalidRequestException
-    {
-        state.hasColumnFamilyAccess(keyspace(), columnFamily(), Permission.ALTER);
-    }
-
-    public void validate(ClientState state) throws RequestValidationException
-    {
-        TableMetadata table = Schema.instance.validateTable(keyspace(), columnFamily());
-
-        if (table.isVirtual())
-            throw new InvalidRequestException("Secondary indexes are not supported on virtual tables");
-
-        if (table.isCounter())
-            throw new InvalidRequestException("Secondary indexes are not supported on counter tables");
-
-        if (table.isView())
-            throw new InvalidRequestException("Secondary indexes are not supported on materialized views");
-
-        if (table.isCompactTable() && !table.isStaticCompactTable())
-            throw new InvalidRequestException("Secondary indexes are not supported on COMPACT STORAGE tables that have clustering columns");
-
-        List<IndexTarget> targets = new ArrayList<>(rawTargets.size());
-        for (IndexTarget.Raw rawTarget : rawTargets)
-            targets.add(rawTarget.prepare(table));
-
-        if (targets.isEmpty() && !properties.isCustom)
-            throw new InvalidRequestException("Only CUSTOM indexes can be created without specifying a target column");
-
-        if (targets.size() > 1)
-            validateTargetsForMultiColumnIndex(targets);
-
-        for (IndexTarget target : targets)
-        {
-            ColumnMetadata cd = table.getColumn(target.column);
-
-            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 (table.isCompactTable() && cd.isPrimaryKeyColumn())
-                throw new InvalidRequestException("Secondary indexes are not supported on PRIMARY KEY columns in COMPACT STORAGE tables");
-
-            if (cd.kind == ColumnMetadata.Kind.PARTITION_KEY && table.partitionKeyColumns().size() == 1)
-                throw new InvalidRequestException(String.format("Cannot create secondary index on partition key column %s", target.column));
-
-            boolean isMap = cd.type instanceof MapType;
-            boolean isFrozenCollection = cd.type.isCollection() && !cd.type.isMultiCell();
-            if (isFrozenCollection)
-            {
-                validateForFrozenCollection(target);
-            }
-            else
-            {
-                validateNotFullIndex(target);
-                validateIsSimpleIndexIfTargetColumnNotCollection(cd, target);
-                validateTargetColumnIsMapIfIndexInvolvesKeys(isMap, target);
-            }
-
-            checkFalse(cd.type.isUDT() && cd.type.isMultiCell(), "Secondary indexes are not supported on non-frozen UDTs");
-        }
-
-        if (!Strings.isNullOrEmpty(indexName))
-        {
-            if (Schema.instance.getKeyspaceMetadata(keyspace()).existingIndexNames(null).contains(indexName))
-            {
-                if (ifNotExists)
-                    return;
-                else
-                    throw new InvalidRequestException(String.format("Index %s already exists", indexName));
-            }
-        }
-
-        properties.validate();
-    }
-
-    private void validateForFrozenCollection(IndexTarget target) throws InvalidRequestException
-    {
-        if (target.type != IndexTarget.Type.FULL)
-            throw new InvalidRequestException(String.format("Cannot create %s() index on frozen column %s. " +
-                                                            "Frozen collections only support full() indexes",
-                                                            target.type, target.column));
-    }
-
-    private void validateNotFullIndex(IndexTarget target) throws InvalidRequestException
-    {
-        if (target.type == IndexTarget.Type.FULL)
-            throw new InvalidRequestException("full() indexes can only be created on frozen collections");
-    }
-
-    private void validateIsSimpleIndexIfTargetColumnNotCollection(ColumnMetadata cd, IndexTarget target) throws InvalidRequestException
-    {
-        if (!cd.type.isCollection() && target.type != IndexTarget.Type.SIMPLE)
-            throw new InvalidRequestException(String.format("Cannot create %s() index on %s. " +
-                                                            "Non-collection columns support only simple indexes",
-                                                            target.type.toString(), target.column));
-    }
-
-    private void validateTargetColumnIsMapIfIndexInvolvesKeys(boolean isMap, IndexTarget target) throws InvalidRequestException
-    {
-        if (target.type == IndexTarget.Type.KEYS || target.type == IndexTarget.Type.KEYS_AND_VALUES)
-        {
-            if (!isMap)
-                throw new InvalidRequestException(String.format("Cannot create index on %s of column %s with non-map type",
-                                                                target.type, target.column));
-        }
-    }
-
-    private void validateTargetsForMultiColumnIndex(List<IndexTarget> targets)
-    {
-        if (!properties.isCustom)
-            throw new InvalidRequestException("Only CUSTOM indexes support multiple columns");
-
-        Set<ColumnIdentifier> columns = Sets.newHashSetWithExpectedSize(targets.size());
-        for (IndexTarget target : targets)
-            if (!columns.add(target.column))
-                throw new InvalidRequestException("Duplicate column " + target.column + " in index target list");
-    }
-
-    public Event.SchemaChange announceMigration(QueryState queryState, boolean isLocalOnly) throws RequestValidationException
-    {
-        TableMetadata current = Schema.instance.getTableMetadata(keyspace(), columnFamily());
-        List<IndexTarget> targets = new ArrayList<>(rawTargets.size());
-        for (IndexTarget.Raw rawTarget : rawTargets)
-            targets.add(rawTarget.prepare(current));
-
-        String acceptedName = indexName;
-        if (Strings.isNullOrEmpty(acceptedName))
-        {
-            acceptedName = Indexes.getAvailableIndexName(keyspace(),
-                                                         columnFamily(),
-                                                         targets.size() == 1 ? targets.get(0).column.toString() : null);
-        }
-
-        if (Schema.instance.getKeyspaceMetadata(keyspace()).existingIndexNames(null).contains(acceptedName))
-        {
-            if (ifNotExists)
-                return null;
-            else
-                throw new InvalidRequestException(String.format("Index %s already exists", acceptedName));
-        }
-
-        IndexMetadata.Kind kind;
-        Map<String, String> indexOptions;
-        if (properties.isCustom)
-        {
-            kind = IndexMetadata.Kind.CUSTOM;
-            indexOptions = properties.getOptions();
-        }
-        else
-        {
-            indexOptions = Collections.emptyMap();
-            kind = current.isCompound() ? IndexMetadata.Kind.COMPOSITES : IndexMetadata.Kind.KEYS;
-        }
-
-        IndexMetadata index = IndexMetadata.fromIndexTargets(targets, acceptedName, kind, indexOptions);
-
-        // check to disallow creation of an index which duplicates an existing one in all but name
-        Optional<IndexMetadata> existingIndex = Iterables.tryFind(current.indexes, existing -> existing.equalsWithoutName(index));
-        if (existingIndex.isPresent())
-        {
-            if (ifNotExists)
-                return null;
-            else
-                throw new InvalidRequestException(String.format("Index %s is a duplicate of existing index %s",
-                                                                index.name,
-                                                                existingIndex.get().name));
-        }
-
-        TableMetadata updated =
-            current.unbuild()
-                   .indexes(current.indexes.with(index))
-                   .build();
-
-        logger.trace("Updating index definition for {}", indexName);
-
-        MigrationManager.announceTableUpdate(updated, isLocalOnly);
-
-        // Creating an index is akin to updating the CF
-        return new Event.SchemaChange(Event.SchemaChange.Change.UPDATED, Event.SchemaChange.Target.TABLE, keyspace(), columnFamily());
-    }
-    
-    @Override
-    public String toString()
-    {
-        return ToStringBuilder.reflectionToString(this, ToStringStyle.SHORT_PREFIX_STYLE);
-    }
-    @Override
-    public AuditLogContext getAuditLogContext()
-    {
-        return new AuditLogContext(AuditLogEntryType.CREATE_INDEX, keyspace(), indexName);
-    }
-}


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