You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by bl...@apache.org on 2017/01/18 14:59:34 UTC

cassandra git commit: Remove alter type support

Repository: cassandra
Updated Branches:
  refs/heads/cassandra-3.0 52bf7acb0 -> 5f66d48b3


Remove alter type support

patch by Carl Yeksigian; reviewed by Benjamin Lerer for CASSANDRA-12443


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

Branch: refs/heads/cassandra-3.0
Commit: 5f66d48b38d32d3768c78f16753ed4f2095bbede
Parents: 52bf7ac
Author: Carl Yeksigian <ca...@apache.org>
Authored: Wed Jan 18 15:56:35 2017 +0100
Committer: Benjamin Lerer <b....@gmail.com>
Committed: Wed Jan 18 15:59:10 2017 +0100

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 NEWS.txt                                        |  3 +
 doc/cql3/CQL.textile                            | 14 +---
 pylib/cqlshlib/cql3handling.py                  |  6 +-
 .../org/apache/cassandra/config/CFMetaData.java | 27 ++-----
 .../cql3/statements/AlterTableStatement.java    | 80 +-------------------
 .../cql3/statements/AlterTypeStatement.java     | 36 ++-------
 .../apache/cassandra/cql3/ViewSchemaTest.java   | 76 -------------------
 .../entities/FrozenCollectionsTest.java         | 42 ----------
 .../cql3/validation/entities/TypeTest.java      | 32 --------
 .../cql3/validation/entities/UserTypesTest.java |  1 -
 .../cql3/validation/operations/AlterTest.java   | 54 -------------
 .../cql3/validation/operations/UpdateTest.java  | 32 --------
 13 files changed, 23 insertions(+), 381 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/5f66d48b/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index e4afc3c..5a63213 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 3.0.11
+ * Remove ALTER TYPE support (CASSANDRA-12443)
  * Fix assertion for certain legacy range tombstone pattern (CASSANDRA-12203)
  * Set javac encoding to utf-8 (CASSANDRA-11077)
  * Replace empty strings with null values if they cannot be converted (CASSANDRA-12794)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/5f66d48b/NEWS.txt
----------------------------------------------------------------------
diff --git a/NEWS.txt b/NEWS.txt
index 3b8a333..4505574 100644
--- a/NEWS.txt
+++ b/NEWS.txt
@@ -18,6 +18,9 @@ using the provided 'sstableupgrade' tool.
 
 Upgrading
 ---------
+   - Support for alter types of already defined tables and of UDTs fields has been disabled.
+     If it is necessary to return a different type, please use casting instead. See
+     CASSANDRA-12443 for more details.
    - Nothing specific to this release, but please see previous versions upgrading section,
      especially if you are upgrading from 2.2.
    - Specifying the default_time_to_live option when creating or altering a

http://git-wip-us.apache.org/repos/asf/cassandra/blob/5f66d48b/doc/cql3/CQL.textile
----------------------------------------------------------------------
diff --git a/doc/cql3/CQL.textile b/doc/cql3/CQL.textile
index 2544878..1efa6d4 100644
--- a/doc/cql3/CQL.textile
+++ b/doc/cql3/CQL.textile
@@ -397,8 +397,7 @@ __Syntax:__
 bc(syntax).. 
 <alter-table-stmt> ::= ALTER (TABLE | COLUMNFAMILY) <tablename> <instruction>
 
-<instruction> ::= ALTER <identifier> TYPE <type>
-                | ADD   <identifier> <type>
+<instruction> ::= ADD   <identifier> <type>
                 | DROP  <identifier>
                 | WITH  <option> ( AND <option> )*
 p. 
@@ -406,7 +405,6 @@ __Sample:__
 
 bc(sample).. 
 ALTER TABLE addamsFamily
-ALTER lastKnownLocation TYPE uuid;
 
 ALTER TABLE addamsFamily
 ADD gravesite varchar;
@@ -415,10 +413,9 @@ ALTER TABLE addamsFamily
 WITH comment = 'A most excellent and useful column family'
  AND read_repair_chance = 0.2;
 p. 
-The @ALTER@ statement is used to manipulate table definitions. It allows for adding new columns, dropping existing ones, changing the type of existing columns, or updating the table options. As with table creation, @ALTER COLUMNFAMILY@ is allowed as an alias for @ALTER TABLE@.
+The @ALTER@ statement is used to manipulate table definitions. It allows for adding new columns, dropping existing ones, or updating the table options. As with table creation, @ALTER COLUMNFAMILY@ is allowed as an alias for @ALTER TABLE@.
 
 The @<tablename>@ is the table name optionally preceded by the keyspace name.  The @<instruction>@ defines the alteration to perform:
-* @ALTER@: Update the type of a given defined column. Note that the type of the "clustering columns":#createTablepartitionClustering cannot be modified as it induces the on-disk ordering of rows. Columns on which a "secondary index":#createIndexStmt is defined have the same restriction. Other columns are free from those restrictions (no validation of existing data is performed), but it is usually a bad idea to change the type to a non-compatible one, unless no data have been inserted for that column yet, as this could confuse CQL drivers/tools.
 * @ADD@: Adds a new column to the table. The @<identifier>@ for the new column must not conflict with an existing column. Moreover, columns cannot be added to tables defined with the @COMPACT STORAGE@ option.
 * @DROP@: Removes a column from the table. Dropped columns will immediately become unavailable in the queries and will not be included in compacted sstables in the future. If a column is readded, queries won't return values written before the column was last dropped. It is assumed that timestamps represent actual time, so if this is not your case, you should NOT readd previously dropped columns. Columns can't be dropped from tables defined with the @COMPACT STORAGE@ option.
 * @WITH@: Allows to update the options of the table. The "supported @<option>@":#createTableOptions (and syntax) are the same as for the @CREATE TABLE@ statement except that @COMPACT STORAGE@ is not supported. Note that setting any @compaction@ sub-options has the effect of erasing all previous @compaction@ options, so you  need to re-specify all the sub-options if you want to keep them. The same note applies to the set of @compression@ sub-options.
@@ -598,23 +595,18 @@ __Syntax:__
 bc(syntax).. 
 <alter-type-stmt> ::= ALTER TYPE <typename> <instruction>
 
-<instruction> ::= ALTER <field-name> TYPE <type>
-                | ADD <field-name> <type>
+<instruction> ::= ADD <field-name> <type>
                 | RENAME <field-name> TO <field-name> ( AND <field-name> TO <field-name> )*
 p. 
 __Sample:__
 
 bc(sample).. 
-ALTER TYPE address ALTER zip TYPE varint
-
 ALTER TYPE address ADD country text
 
 ALTER TYPE address RENAME zip TO zipcode AND street_name TO street
 p. 
 The @ALTER TYPE@ statement is used to manipulate type definitions. It allows for adding new fields, renaming existing fields, or changing the type of existing fields.
 
-When altering the type of a column, the new type must be compatible with the previous type.
-
 h3(#dropTypeStmt). DROP TYPE
 
 __Syntax:__

http://git-wip-us.apache.org/repos/asf/cassandra/blob/5f66d48b/pylib/cqlshlib/cql3handling.py
----------------------------------------------------------------------
diff --git a/pylib/cqlshlib/cql3handling.py b/pylib/cqlshlib/cql3handling.py
index 9008514..2a8b560 100644
--- a/pylib/cqlshlib/cql3handling.py
+++ b/pylib/cqlshlib/cql3handling.py
@@ -1277,8 +1277,7 @@ syntax_rules += r'''
 <alterTableStatement> ::= "ALTER" wat=( "COLUMNFAMILY" | "TABLE" ) cf=<columnFamilyName>
                                <alterInstructions>
                         ;
-<alterInstructions> ::= "ALTER" existcol=<cident> "TYPE" <storageType>
-                      | "ADD" newcol=<cident> <storageType> ("static")?
+<alterInstructions> ::= "ADD" newcol=<cident> <storageType> ("static")?
                       | "DROP" existcol=<cident>
                       | "WITH" <cfamProperty> ( "AND" <cfamProperty> )*
                       | "RENAME" existcol=<cident> "TO" newcol=<cident>
@@ -1288,8 +1287,7 @@ syntax_rules += r'''
 <alterUserTypeStatement> ::= "ALTER" "TYPE" ut=<userTypeName>
                                <alterTypeInstructions>
                              ;
-<alterTypeInstructions> ::= "ALTER" existcol=<cident> "TYPE" <storageType>
-                           | "ADD" newcol=<cident> <storageType>
+<alterTypeInstructions> ::= "ADD" newcol=<cident> <storageType>
                            | "RENAME" existcol=<cident> "TO" newcol=<cident>
                               ( "AND" existcol=<cident> "TO" newcol=<cident> )*
                            ;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/5f66d48b/src/java/org/apache/cassandra/config/CFMetaData.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/CFMetaData.java b/src/java/org/apache/cassandra/config/CFMetaData.java
index a702902..a3370dc 100644
--- a/src/java/org/apache/cassandra/config/CFMetaData.java
+++ b/src/java/org/apache/cassandra/config/CFMetaData.java
@@ -82,18 +82,17 @@ public final class CFMetaData
     private final boolean isSuper;
     private final boolean isCounter;
     private final boolean isView;
-
     private final boolean isIndex;
 
-    public volatile ClusteringComparator comparator;  // bytes, long, timeuuid, utf8, etc. This is built directly from clusteringColumns
+    public final ClusteringComparator comparator;  // bytes, long, timeuuid, utf8, etc. This is built directly from clusteringColumns
     public final IPartitioner partitioner;            // partitioner the table uses
+    private final AbstractType<?> keyValidator;
 
     private final Serializers serializers;
 
     // non-final, for now
     public volatile TableParams params = TableParams.DEFAULT;
 
-    private volatile AbstractType<?> keyValidator = BytesType.instance;
     private volatile Map<ByteBuffer, DroppedColumn> droppedColumns = new HashMap<>();
     private volatile Triggers triggers = Triggers.none();
     private volatile Indexes indexes = Indexes.none();
@@ -285,6 +284,11 @@ public final class CFMetaData
         this.partitionColumns = partitionColumns;
 
         this.serializers = new Serializers(this);
+
+        this.comparator = new ClusteringComparator(extractTypes(clusteringColumns));
+        List<AbstractType<?>> keyTypes = extractTypes(partitionKeyColumns);
+        this.keyValidator = keyTypes.size() == 1 ? keyTypes.get(0) : CompositeType.getInstance(keyTypes);
+
         rebuild();
     }
 
@@ -292,24 +296,16 @@ public final class CFMetaData
     // are kept because they are often useful in a different format.
     private void rebuild()
     {
-        this.comparator = new ClusteringComparator(extractTypes(clusteringColumns));
-
         Map<ByteBuffer, ColumnDefinition> newColumnMetadata = new HashMap<>();
         for (ColumnDefinition def : partitionKeyColumns)
             newColumnMetadata.put(def.name.bytes, def);
         for (ColumnDefinition def : clusteringColumns)
-        {
             newColumnMetadata.put(def.name.bytes, def);
-            def.type.checkComparable();
-        }
         for (ColumnDefinition def : partitionColumns)
             newColumnMetadata.put(def.name.bytes, def);
 
         this.columnMetadata = newColumnMetadata;
 
-        List<AbstractType<?>> keyTypes = extractTypes(partitionKeyColumns);
-        this.keyValidator = keyTypes.size() == 1 ? keyTypes.get(0) : CompositeType.getInstance(keyTypes);
-
         if (isCompactTable())
             this.compactValueColumn = CompactTables.getCompactValueColumn(partitionColumns, isSuper());
     }
@@ -762,8 +758,6 @@ public final class CFMetaData
 
         params = cfm.params;
 
-        keyValidator = cfm.keyValidator;
-
         if (!cfm.droppedColumns.isEmpty())
             droppedColumns = cfm.droppedColumns;
 
@@ -789,12 +783,8 @@ public final class CFMetaData
         if (!cfm.cfId.equals(cfId))
             throw new ConfigurationException(String.format("Column family ID mismatch (found %s; expected %s)",
                                                            cfm.cfId, cfId));
-
         if (!cfm.flags.equals(flags))
             throw new ConfigurationException("types do not match.");
-
-        if (!cfm.comparator.isCompatibleWith(comparator))
-            throw new ConfigurationException(String.format("Column family comparators do not match or are not compatible (found %s; expected %s).", cfm.comparator.toString(), comparator.toString()));
     }
 
 
@@ -931,12 +921,9 @@ public final class CFMetaData
         {
             case PARTITION_KEY:
                 partitionKeyColumns.set(def.position(), def);
-                List<AbstractType<?>> keyTypes = extractTypes(partitionKeyColumns);
-                keyValidator = keyTypes.size() == 1 ? keyTypes.get(0) : CompositeType.getInstance(keyTypes);
                 break;
             case CLUSTERING:
                 clusteringColumns.set(def.position(), def);
-                comparator = new ClusteringComparator(extractTypes(clusteringColumns));
                 break;
             case REGULAR:
             case STATIC:

http://git-wip-us.apache.org/repos/asf/cassandra/blob/5f66d48b/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java b/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java
index 6210d16..756bb96 100644
--- a/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java
@@ -111,6 +111,8 @@ public class AlterTableStatement extends SchemaAlteringStatement
 
         switch (oType)
         {
+            case ALTER:
+                throw new InvalidRequestException("Altering of types is not allowed");
             case ADD:
                 assert columnName != null;
                 if (cfm.isDense())
@@ -186,37 +188,6 @@ public class AlterTableStatement extends SchemaAlteringStatement
                 }
                 break;
 
-            case ALTER:
-                assert columnName != null;
-                if (def == null)
-                    throw new InvalidRequestException(String.format("Column %s was not found in table %s", columnName, columnFamily()));
-
-                AbstractType<?> validatorType = def.isReversedType() && !validator.getType().isReversed()
-                                                ? ReversedType.getInstance(validator.getType())
-                                                : validator.getType();
-                validateAlter(cfm, def, validatorType);
-                // In any case, we update the column definition
-                cfm.addOrReplaceColumnDefinition(def.withNewType(validatorType));
-
-                // We also have to validate the view types here. If we have a view which includes a column as part of
-                // the clustering key, we need to make sure that it is indeed compatible.
-                for (ViewDefinition view : views)
-                {
-                    if (!view.includes(columnName)) continue;
-                    ViewDefinition viewCopy = view.copy();
-                    ColumnDefinition viewDef = view.metadata.getColumnDefinition(columnName);
-                    AbstractType viewType = viewDef.isReversedType() && !validator.getType().isReversed()
-                                            ? ReversedType.getInstance(validator.getType())
-                                            : validator.getType();
-                    validateAlter(view.metadata, viewDef, viewType);
-                    viewCopy.metadata.addOrReplaceColumnDefinition(viewDef.withNewType(viewType));
-
-                    if (viewUpdates == null)
-                        viewUpdates = new ArrayList<>();
-                    viewUpdates.add(viewCopy);
-                }
-                break;
-
             case DROP:
                 assert columnName != null;
                 if (!cfm.isCQLTable())
@@ -336,53 +307,6 @@ public class AlterTableStatement extends SchemaAlteringStatement
         return new Event.SchemaChange(Event.SchemaChange.Change.UPDATED, Event.SchemaChange.Target.TABLE, keyspace(), columnFamily());
     }
 
-    private static void validateAlter(CFMetaData cfm, ColumnDefinition def, AbstractType<?> validatorType)
-    {
-        switch (def.kind)
-        {
-            case PARTITION_KEY:
-                if (validatorType instanceof CounterColumnType)
-                    throw new InvalidRequestException(String.format("counter type is not supported for PRIMARY KEY part %s", def.name));
-
-                AbstractType<?> currentType = cfm.getKeyValidatorAsClusteringComparator().subtype(def.position());
-                if (!validatorType.isValueCompatibleWith(currentType))
-                    throw new ConfigurationException(String.format("Cannot change %s from type %s to type %s: types are incompatible.",
-                                                                   def.name,
-                                                                   currentType.asCQL3Type(),
-                                                                   validatorType.asCQL3Type()));
-                break;
-            case CLUSTERING:
-                if (!cfm.isCQLTable())
-                    throw new InvalidRequestException(String.format("Cannot alter clustering column %s in a non-CQL3 table", def.name));
-
-                AbstractType<?> oldType = cfm.comparator.subtype(def.position());
-                // Note that CFMetaData.validateCompatibility already validate the change we're about to do. However, the error message it
-                // sends is a bit cryptic for a CQL3 user, so validating here for a sake of returning a better error message
-                // Do note that we need isCompatibleWith here, not just isValueCompatibleWith.
-                if (!validatorType.isCompatibleWith(oldType))
-                {
-                    throw new ConfigurationException(String.format("Cannot change %s from type %s to type %s: types are not order-compatible.",
-                                                                   def.name,
-                                                                   oldType.asCQL3Type(),
-                                                                   validatorType.asCQL3Type()));
-                }
-                break;
-            case REGULAR:
-            case STATIC:
-                // Thrift allows to change a column validator so CFMetaData.validateCompatibility will let it slide
-                // if we change to an incompatible type (contrarily to the comparator case). But we don't want to
-                // allow it for CQL3 (see #5882) so validating it explicitly here. We only care about value compatibility
-                // though since we won't compare values (except when there is an index, but that is validated by
-                // ColumnDefinition already).
-                if (!validatorType.isValueCompatibleWith(def.type))
-                    throw new ConfigurationException(String.format("Cannot change %s from type %s to type %s: types are incompatible.",
-                                                                   def.name,
-                                                                   def.type.asCQL3Type(),
-                                                                   validatorType.asCQL3Type()));
-                break;
-        }
-    }
-
     public String toString()
     {
         return String.format("AlterTableStatement(name=%s, type=%s, column=%s, validator=%s)",

http://git-wip-us.apache.org/repos/asf/cassandra/blob/5f66d48b/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
index 4cac3b3..f2e1578 100644
--- a/src/java/org/apache/cassandra/cql3/statements/AlterTypeStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/AlterTypeStatement.java
@@ -54,12 +54,12 @@ public abstract class AlterTypeStatement extends SchemaAlteringStatement
 
     public static AlterTypeStatement addition(UTName name, ColumnIdentifier fieldName, CQL3Type.Raw type)
     {
-        return new AddOrAlter(name, true, fieldName, type);
+        return new Add(name, fieldName, type);
     }
 
     public static AlterTypeStatement alter(UTName name, ColumnIdentifier fieldName, CQL3Type.Raw type)
     {
-        return new AddOrAlter(name, false, fieldName, type);
+        throw new InvalidRequestException("Altering of types is not allowed");
     }
 
     public static AlterTypeStatement renames(UTName name, Map<ColumnIdentifier, ColumnIdentifier> renames)
@@ -245,21 +245,19 @@ public abstract class AlterTypeStatement extends SchemaAlteringStatement
                      });
     }
 
-    private static class AddOrAlter extends AlterTypeStatement
+    private static class Add extends AlterTypeStatement
     {
-        private final boolean isAdd;
         private final ColumnIdentifier fieldName;
         private final CQL3Type.Raw type;
 
-        public AddOrAlter(UTName name, boolean isAdd, ColumnIdentifier fieldName, CQL3Type.Raw type)
+        public Add(UTName name, ColumnIdentifier fieldName, CQL3Type.Raw type)
         {
             super(name);
-            this.isAdd = isAdd;
             this.fieldName = fieldName;
             this.type = type;
         }
 
-        private UserType doAdd(UserType toUpdate) throws InvalidRequestException
+        protected UserType makeUpdatedType(UserType toUpdate, KeyspaceMetadata ksm) throws InvalidRequestException
         {
             if (getIdxOfField(toUpdate, 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));
@@ -278,30 +276,6 @@ public abstract class AlterTypeStatement extends SchemaAlteringStatement
 
             return new UserType(toUpdate.keyspace, toUpdate.name, newNames, newTypes);
         }
-
-        private UserType doAlter(UserType toUpdate, KeyspaceMetadata ksm) throws InvalidRequestException
-        {
-            checkTypeNotUsedByAggregate(ksm);
-
-            int idx = getIdxOfField(toUpdate, fieldName);
-            if (idx < 0)
-                throw new InvalidRequestException(String.format("Unknown field %s in type %s", fieldName, name));
-
-            AbstractType<?> previous = toUpdate.fieldType(idx);
-            if (!type.prepare(keyspace()).getType().isCompatibleWith(previous))
-                throw new InvalidRequestException(String.format("Type %s is incompatible with previous type %s of field %s in user type %s", type, previous.asCQL3Type(), fieldName, name));
-
-            List<ByteBuffer> newNames = new ArrayList<>(toUpdate.fieldNames());
-            List<AbstractType<?>> newTypes = new ArrayList<>(toUpdate.fieldTypes());
-            newTypes.set(idx, type.prepare(keyspace()).getType());
-
-            return new UserType(toUpdate.keyspace, toUpdate.name, newNames, newTypes);
-        }
-
-        protected UserType makeUpdatedType(UserType toUpdate, KeyspaceMetadata ksm) throws InvalidRequestException
-        {
-            return isAdd ? doAdd(toUpdate) : doAlter(toUpdate, ksm);
-        }
     }
 
     private static class Renames extends AlterTypeStatement

http://git-wip-us.apache.org/repos/asf/cassandra/blob/5f66d48b/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 b34f475..113fdf2 100644
--- a/test/unit/org/apache/cassandra/cql3/ViewSchemaTest.java
+++ b/test/unit/org/apache/cassandra/cql3/ViewSchemaTest.java
@@ -683,80 +683,4 @@ public class ViewSchemaTest extends CQLTester
             Assert.assertEquals("Cannot use DROP TABLE on Materialized View", e.getMessage());
         }
     }
-
-    @Test
-    public void testAlterTable() throws Throwable
-    {
-        createTable("CREATE TABLE %s (" +
-                    "a int," +
-                    "b text," +
-                    "PRIMARY KEY (a, b))");
-
-        executeNet(protocolVersion, "USE " + keyspace());
-
-        createView("mv1", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE a IS NOT NULL AND b IS NOT NULL PRIMARY KEY (b, a)");
-
-        alterTable("ALTER TABLE %s ALTER b TYPE blob");
-    }
-
-    @Test
-    public void testAlterReversedTypeBaseTable() throws Throwable
-    {
-        createTable("CREATE TABLE %s (" +
-                    "a int," +
-                    "b text," +
-                    "PRIMARY KEY (a, b))" +
-                    "WITH CLUSTERING ORDER BY (b DESC)");
-
-        executeNet(protocolVersion, "USE " + keyspace());
-
-        createView("mv1", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE a IS NOT NULL AND b IS NOT NULL PRIMARY KEY (a, b) WITH CLUSTERING ORDER BY (b ASC)");
-
-        alterTable("ALTER TABLE %s ALTER b TYPE blob");
-    }
-
-    @Test
-    public void testAlterReversedTypeViewTable() throws Throwable
-    {
-        createTable("CREATE TABLE %s (" +
-                    "a int," +
-                    "b text," +
-                    "PRIMARY KEY (a, b))");
-
-        executeNet(protocolVersion, "USE " + keyspace());
-
-        createView("mv1", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE a IS NOT NULL AND b IS NOT NULL PRIMARY KEY (a, b) WITH CLUSTERING ORDER BY (b DESC)");
-
-        alterTable("ALTER TABLE %s ALTER b TYPE blob");
-    }
-
-    @Test
-    public void testAlterClusteringViewTable() throws Throwable
-    {
-        createTable("CREATE TABLE %s (" +
-                    "a int," +
-                    "b text," +
-                    "PRIMARY KEY (a))");
-
-        executeNet(protocolVersion, "USE " + keyspace());
-
-        createView("mv1", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE a IS NOT NULL AND b IS NOT NULL PRIMARY KEY (a, b) WITH CLUSTERING ORDER BY (b DESC)");
-
-        alterTable("ALTER TABLE %s ALTER b TYPE blob");
-    }
-
-    @Test
-    public void testAlterViewTableValue() throws Throwable
-    {
-        createTable("CREATE TABLE %s (" +
-                    "a int," +
-                    "b int," +
-                    "PRIMARY KEY (a))");
-
-        executeNet(protocolVersion, "USE " + keyspace());
-
-        createView("mv1", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE a IS NOT NULL AND b IS NOT NULL PRIMARY KEY (a, b) WITH CLUSTERING ORDER BY (b DESC)");
-
-        assertInvalid("ALTER TABLE %s ALTER b TYPE blob");
-    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/5f66d48b/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 9df8ea0..f89163d 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/entities/FrozenCollectionsTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/FrozenCollectionsTest.java
@@ -509,20 +509,6 @@ public class FrozenCollectionsTest extends CQLTester
         }
     }
 
-    private void assertInvalidAlterWithMessage(String createTableStatement, String errorMessage) throws Throwable
-    {
-        try
-        {
-            alterTableMayThrow(createTableStatement);
-            Assert.fail("Expected CREATE TABLE statement to error: " + createTableStatement);
-        }
-        catch (InvalidRequestException | ConfigurationException ex)
-        {
-            Assert.assertTrue("Expected error message to contain '" + errorMessage + "', but got '" + ex.getMessage() + "'",
-                    ex.getMessage().contains(errorMessage));
-        }
-    }
-
     @Test
     public void testInvalidOperations() throws Throwable
     {
@@ -555,34 +541,6 @@ public class FrozenCollectionsTest extends CQLTester
                 "frozen<> is only allowed on collections, tuples, and user-defined types");
     }
 
-    @Test
-    public void testAltering() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a int, b frozen<list<int>>, c frozen<list<int>>, PRIMARY KEY (a, b))");
-
-        alterTable("ALTER TABLE %s ALTER c TYPE frozen<list<blob>>");
-
-        assertInvalidAlterWithMessage("ALTER TABLE %s ALTER b TYPE frozen<list<blob>>",
-                                      "types are not order-compatible");
-
-        assertInvalidAlterWithMessage("ALTER TABLE %s ALTER b TYPE list<int>",
-                                      "types are not order-compatible");
-
-        assertInvalidAlterWithMessage("ALTER TABLE %s ALTER c TYPE list<blob>",
-                                      "types are incompatible");
-
-        alterTable("ALTER TABLE %s DROP c");
-        alterTable("ALTER TABLE %s ADD c frozen<set<int>>");
-        assertInvalidAlterWithMessage("ALTER TABLE %s ALTER c TYPE frozen<set<blob>>",
-                                      "types are incompatible");
-
-        alterTable("ALTER TABLE %s DROP c");
-        alterTable("ALTER TABLE %s ADD c frozen<map<int, int>>");
-        assertInvalidAlterWithMessage("ALTER TABLE %s ALTER c TYPE frozen<map<blob, int>>",
-                                      "types are incompatible");
-        alterTable("ALTER TABLE %s ALTER c TYPE frozen<map<int, blob>>");
-    }
-
     private void assertInvalidIndexCreationWithMessage(String statement, String errorMessage) throws Throwable
     {
         try

http://git-wip-us.apache.org/repos/asf/cassandra/blob/5f66d48b/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 bcab807..60a0fdc 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/entities/TypeTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/TypeTest.java
@@ -67,36 +67,4 @@ public class TypeTest extends CQLTester
         UntypedResultSet results = execute("SELECT * FROM %s WHERE a=0 AND b < now()");
         assertEquals(1, results.size());
     }
-
-    @Test
-    // tests CASSANDRA-7797
-    public void testAlterReversedColumn() throws Throwable
-    {
-        createTable("CREATE TABLE IF NOT EXISTS %s (a int, b 'org.apache.cassandra.db.marshal.DateType', PRIMARY KEY (a, b)) WITH CLUSTERING ORDER BY (b DESC)");
-        alterTable("ALTER TABLE %s ALTER b TYPE 'org.apache.cassandra.db.marshal.ReversedType(org.apache.cassandra.db.marshal.TimestampType)'");
-    }
-
-    @Test
-    public void testIncompatibleReversedTypes() throws Throwable
-    {
-        createTable("CREATE TABLE IF NOT EXISTS %s (a int, b 'org.apache.cassandra.db.marshal.DateType', PRIMARY KEY (a, b)) WITH CLUSTERING ORDER BY (b DESC)");
-        try
-        {
-            alterTable("ALTER TABLE %s ALTER b TYPE 'org.apache.cassandra.db.marshal.ReversedType(org.apache.cassandra.db.marshal.TimeUUIDType)'");
-            fail("Expected error for ALTER statement");
-        }
-        catch (RuntimeException e) { }
-    }
-
-    @Test
-    public void testReversedAndNonReversed() throws Throwable
-    {
-        createTable("CREATE TABLE IF NOT EXISTS %s (a int, b 'org.apache.cassandra.db.marshal.DateType', PRIMARY KEY (a, b))");
-        try
-        {
-            alterTable("ALTER TABLE %s ALTER b TYPE 'org.apache.cassandra.db.marshal.ReversedType(org.apache.cassandra.db.marshal.DateType)'");
-            fail("Expected error for ALTER statement");
-        }
-        catch (RuntimeException e) { }
-    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/5f66d48b/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 850d8ee..e776f0e 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/entities/UserTypesTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/UserTypesTest.java
@@ -550,7 +550,6 @@ 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 alter user type " + typeWithKs(t), "ALTER TYPE " + typeWithKs(t) + " ALTER foo TYPE text;");
         assertInvalidMessage("Cannot drop user type " + typeWithKs(t), "DROP TYPE " + typeWithKs(t) + ';');
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/5f66d48b/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 bcd6587..245be30 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/AlterTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/AlterTest.java
@@ -343,37 +343,6 @@ public class AlterTest extends CQLTester
                                            "ALTER TABLE %s WITH compression = { 'class' : 'SnappyCompressor', 'chunk_length_kb' : 32 , 'chunk_length_in_kb' : 32 };");
     }
 
-    @Test
-    public void testAlterType() throws Throwable
-    {
-        createTable("CREATE TABLE %s (id text PRIMARY KEY, content text);");
-        alterTable("ALTER TABLE %s ALTER content TYPE blob");
-
-        createTable("CREATE TABLE %s (pk int, ck text, value blob, PRIMARY KEY (pk, ck)) WITH CLUSTERING ORDER BY (ck DESC)");
-        alterTable("ALTER TABLE %s ALTER ck TYPE blob");
-
-        createTable("CREATE TABLE %s (pk int, ck int, value blob, PRIMARY KEY (pk, ck))");
-        assertThrowsConfigurationException("Cannot change value from type blob to type text: types are incompatible.",
-                                           "ALTER TABLE %s ALTER value TYPE TEXT;");
-    }
-
-    /**
-     * tests CASSANDRA-10027
-     */
-    @Test
-    public void testAlterColumnTypeToDate() throws Throwable
-    {
-        createTable("CREATE TABLE %s (key int PRIMARY KEY, c1 int);");
-        execute("INSERT INTO %s (key, c1) VALUES (1,1);");
-        execute("ALTER TABLE %s ALTER c1 TYPE date;");
-        assertRows(execute("SELECT * FROM %s"), row(1, 1));
-
-        createTable("CREATE TABLE %s (key int PRIMARY KEY, c1 varint);");
-        execute("INSERT INTO %s (key, c1) VALUES (1,1);");
-        assertInvalidMessage("Cannot change c1 from type varint to type date: types are incompatible.",
-                             "ALTER TABLE %s ALTER c1 TYPE date;");
-    }
-
     private void assertThrowsConfigurationException(String errorMsg, String alterStmt) throws Throwable
     {
         try
@@ -386,27 +355,4 @@ public class AlterTest extends CQLTester
             assertEquals(errorMsg, e.getMessage());
         }
     }
-
-    @Test // tests CASSANDRA-8879
-    public void testAlterClusteringColumnTypeInCompactTable() throws Throwable
-    {
-        createTable("CREATE TABLE %s (key blob, column1 blob, value blob, PRIMARY KEY ((key), column1)) WITH COMPACT STORAGE");
-        assertInvalidThrow(InvalidRequestException.class, "ALTER TABLE %s ALTER column1 TYPE ascii");
-    }
-
-    @Test
-    public void testAlterToBlob() throws Throwable
-    {
-        // This tests for the bug from #11820 in particular
-
-        createTable("CREATE TABLE %s (a int PRIMARY KEY, b int)");
-
-        execute("INSERT INTO %s (a, b) VALUES (1, 1)");
-
-        executeNet(Server.CURRENT_VERSION, "ALTER TABLE %s ALTER b TYPE BLOB");
-
-        assertRowsNet(Server.CURRENT_VERSION, executeNet(Server.CURRENT_VERSION, "SELECT * FROM %s WHERE a = 1"),
-            row(1, ByteBufferUtil.bytes(1))
-        );
-    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/5f66d48b/test/unit/org/apache/cassandra/cql3/validation/operations/UpdateTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/UpdateTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/UpdateTest.java
index 690d4f9..0d81fa3 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/UpdateTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/UpdateTest.java
@@ -28,38 +28,6 @@ import org.apache.cassandra.utils.ByteBufferUtil;
 
 public class UpdateTest extends CQLTester
 {
-    /**
-     * Test altering the type of a column, including the one in the primary key (#4041)
-     * migrated from cql_tests.py:TestCQL.update_type_test()
-     */
-    @Test
-    public void testUpdateColumnType() throws Throwable
-    {
-        createTable("CREATE TABLE %s (k text, c text, s set <text>, v text, PRIMARY KEY(k, c))");
-
-        // using utf8 character so that we can see the transition to BytesType
-        execute("INSERT INTO %s (k, c, v, s) VALUES ('\u0278', '\u0278', '\u0278', {'\u0278'})");
-
-        assertRows(execute("SELECT * FROM %s"),
-                   row("\u0278", "\u0278", set("\u0278"), "\u0278"));
-
-        execute("ALTER TABLE %s ALTER v TYPE blob");
-        assertRows(execute("SELECT * FROM %s"),
-                   row("\u0278", "\u0278", set("\u0278"), ByteBufferUtil.bytes("\u0278")));
-
-        execute("ALTER TABLE %s ALTER k TYPE blob");
-        assertRows(execute("SELECT * FROM %s"),
-                   row(ByteBufferUtil.bytes("\u0278"), "\u0278", set("\u0278"), ByteBufferUtil.bytes("\u0278")));
-
-        execute("ALTER TABLE %s ALTER c TYPE blob");
-        assertRows(execute("SELECT * FROM %s"),
-                   row(ByteBufferUtil.bytes("\u0278"), ByteBufferUtil.bytes("\u0278"), set("\u0278"), ByteBufferUtil.bytes("\u0278")));
-
-        execute("ALTER TABLE %s ALTER s TYPE set<blob>");
-        assertRows(execute("SELECT * FROM %s"),
-                   row(ByteBufferUtil.bytes("\u0278"), ByteBufferUtil.bytes("\u0278"), set(ByteBufferUtil.bytes("\u0278")), ByteBufferUtil.bytes("\u0278")));
-    }
-
     @Test
     public void testTypeCasts() throws Throwable
     {