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

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

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