From e764be0c4e48ba680f9819690520bca0fba47123 Mon Sep 17 00:00:00 2001 From: jiangxingbo Date: Tue, 1 Nov 2016 18:13:16 +0800 Subject: [PATCH 01/16] implement alter table change column statement. --- .../spark/sql/catalyst/parser/SqlBase.g4 | 10 +++ .../catalyst/catalog/ExternalCatalog.scala | 24 +++++++ .../catalyst/catalog/InMemoryCatalog.scala | 24 +++++++ .../sql/catalyst/catalog/SessionCatalog.scala | 35 ++++++++++ .../apache/spark/sql/types/StructField.scala | 7 ++ .../catalog/ExternalCatalogSuite.scala | 70 +++++++++++++++---- .../catalog/SessionCatalogSuite.scala | 48 +++++++++++++ .../spark/sql/execution/SparkSqlParser.scala | 15 ++++ .../spark/sql/execution/command/tables.scala | 60 ++++++++++++++++ .../spark/sql/hive/HiveExternalCatalog.scala | 22 +++++- .../sql/hive/HiveExternalCatalogSuite.scala | 8 ++- 11 files changed, 308 insertions(+), 15 deletions(-) diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index 075c73d7a320d..87ad60eaa1304 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -89,6 +89,8 @@ statement SET TBLPROPERTIES tablePropertyList #setTableProperties | ALTER (TABLE | VIEW) tableIdentifier UNSET TBLPROPERTIES (IF EXISTS)? tablePropertyList #unsetTableProperties + | ALTER (TABLE | VIEW) tableIdentifier + CHANGE (COLUMN)? changeColTypeList #changeColumns | ALTER TABLE tableIdentifier (partitionSpec)? SET SERDE STRING (WITH SERDEPROPERTIES tablePropertyList)? #setTableSerDe | ALTER TABLE tableIdentifier (partitionSpec)? @@ -593,6 +595,14 @@ colType : identifier dataType (COMMENT STRING)? ; +changeColTypeList + : changeColType (',' changeColType)* + ; + +changeColType + : identifier colType (FIRST | AFTER identifier)? + ; + complexColTypeList : complexColType (',' complexColType)* ; diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalog.scala index 4b8cac8f32b06..d4e70e6decef9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalog.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.catalyst.catalog import org.apache.spark.sql.catalyst.analysis.{FunctionAlreadyExistsException, NoSuchDatabaseException, NoSuchFunctionException, NoSuchTableException} import org.apache.spark.sql.catalyst.expressions.Expression +import org.apache.spark.sql.types.StructField /** @@ -239,6 +240,29 @@ abstract class ExternalCatalog { table: String, predicates: Seq[Expression]): Seq[CatalogTablePartition] + // -------------------------------------------------------------------------- + // Columns + // -------------------------------------------------------------------------- + + /** + * Alter one or more column(s)'s comment. + * + * Note: If the underlying implementation does not support altering a certain field, + * this becomes a no-op. + * @param db database name + * @param table table name + * @param columnComments map from column names to comments + */ + def alterColumnComments(db: String, table: String, columnComments: Map[String, String]): Unit + + /** + * List the metadata of all columns that are defined in the specified table schema, assuming it + * exists. + * @param db database name + * @param table table name + */ + def listColumns(db: String, table: String): Seq[StructField] + // -------------------------------------------------------------------------- // Functions // -------------------------------------------------------------------------- diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala index a6bebe1a3938c..52a729c1614c6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala @@ -19,6 +19,8 @@ package org.apache.spark.sql.catalyst.catalog import java.io.IOException +import org.apache.spark.sql.types.{StructField, StructType} + import scala.collection.mutable import org.apache.hadoop.conf.Configuration @@ -538,6 +540,28 @@ class InMemoryCatalog( "listPartitionsByFilter is not implemented for InMemoryCatalog") } + // -------------------------------------------------------------------------- + // Columns + // -------------------------------------------------------------------------- + + override def alterColumnComments( + db: String, + table: String, + columnComments: Map[String, String]): Unit = synchronized { + val oldTable = getTable(db, table) + val newSchema = oldTable.schema.fields.map { field => + // If `columns` contains field name, update the field with new comment, + // else respect the field. + columnComments.get(field.name).map(field.withComment(_)).getOrElse(field) + } + catalog(db).tables(table).table = oldTable.copy(schema = StructType(newSchema)) + } + + override def listColumns(db: String, table: String): Seq[StructField] = synchronized { + getTable(db, table).schema.fields + } + + // -------------------------------------------------------------------------- // Functions // -------------------------------------------------------------------------- diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala index 7a3d2097a85c5..7c45772d031bf 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala @@ -33,6 +33,7 @@ import org.apache.spark.sql.catalyst.analysis.FunctionRegistry.FunctionBuilder import org.apache.spark.sql.catalyst.expressions.{Expression, ExpressionInfo} import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SubqueryAlias} import org.apache.spark.sql.catalyst.util.StringUtils +import org.apache.spark.sql.types.StructField object SessionCatalog { val DEFAULT_DATABASE = "default" @@ -838,6 +839,40 @@ class SessionCatalog( } } + // ---------------------------------------------------------------------------- + // Columns + // ---------------------------------------------------------------------------- + // All methods in this category interact directly with the underlying catalog. + // These methods are concerned with only metastore tables. + // ---------------------------------------------------------------------------- + + /** + * Alter one or more columns(s)'s comment, assuming they exist. + * + * Note: If the underlying implementation does not support altering a certain field, + * this becomes a no-op. + */ + def alterColumnComments( + tableName: TableIdentifier, + columnComments: Map[String, String]): Unit = { + val db = formatDatabaseName(tableName.database.getOrElse(getCurrentDatabase)) + val table = formatTableName(tableName.table) + requireDbExists(db) + requireTableExists(TableIdentifier(table, Option(db))) + externalCatalog.alterColumnComments(db, table, columnComments) + } + + /** + * List the metadata of all columns that belong to the specified table, assuming it exists. + */ + def listColumns(tableName: TableIdentifier): Seq[StructField] = { + val db = formatDatabaseName(tableName.database.getOrElse(getCurrentDatabase)) + val table = formatTableName(tableName.table) + requireDbExists(db) + requireTableExists(TableIdentifier(table, Option(db))) + externalCatalog.listColumns(db, table) + } + // ---------------------------------------------------------------------------- // Functions // ---------------------------------------------------------------------------- diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructField.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructField.scala index 2c18fdcc497fe..3a3281642c786 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructField.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructField.scala @@ -74,4 +74,11 @@ case class StructField( def getComment(): Option[String] = { if (metadata.contains("comment")) Option(metadata.getString("comment")) else None } + + /** + * Return the full description of the StructField. + */ + def getDesc(): String = { + s"StructField($name,$dataType,$nullable$metadata)" + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogSuite.scala index 00e663c324cb4..d3bb31d24e237 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogSuite.scala @@ -26,7 +26,7 @@ import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} import org.apache.spark.sql.catalyst.analysis.{FunctionAlreadyExistsException, NoSuchDatabaseException, NoSuchFunctionException} import org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException -import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.types._ import org.apache.spark.util.Utils @@ -559,6 +559,45 @@ abstract class ExternalCatalogSuite extends SparkFunSuite with BeforeAndAfterEac } } + // -------------------------------------------------------------------------- + // Columns + // -------------------------------------------------------------------------- + + test("alter column comments") { + val catalog = newBasicCatalog() + val columnComments = Map(field1.name -> "insert comment", field2.name -> s"^*`%?", + field3.name -> "change comment") + catalog.alterColumnComments("db2", "tbl1", columnComments) + val newColumns = field1.withComment("insert comment") :: field2.withComment(s"^*`%?") :: + field3.withComment("change comment") :: field4 :: Nil + assert(columnsEqual(catalog.listColumns("db2", "tbl1"), newColumns)) + } + + test("alter column comments when database/table does not exist") { + val catalog = newBasicCatalog() + val columnComments = Map(field1.name -> "insert comment") + intercept[AnalysisException] { + catalog.alterColumnComments("unknown_db", "unknown_table", columnComments) + } + intercept[AnalysisException] { + catalog.alterColumnComments("db2", "unknown_table", columnComments) + } + } + + test("alter column comments when column does not exist") { + val catalog = newBasicCatalog() + val columnComments = Map("unknown_col" -> "insert comment") + catalog.alterColumnComments("db2", "tbl1", columnComments) + assert(columnsEqual(catalog.listColumns("db2", "tbl1"), schema.fields.toList)) + } + + test("list columns") { + val catalog = newBasicCatalog() + assert(columnsEqual(catalog.listColumns("db2", "tbl1"), schema.fields.toList)) + intercept[AnalysisException] { catalog.listColumns("unknown_db", "unknown_table") } + intercept[AnalysisException] { catalog.listColumns("db2", "unknown_table") } + } + // -------------------------------------------------------------------------- // Functions // -------------------------------------------------------------------------- @@ -685,7 +724,7 @@ abstract class ExternalCatalogSuite extends SparkFunSuite with BeforeAndAfterEac identifier = TableIdentifier("my_table", Some("db1")), tableType = CatalogTableType.MANAGED, storage = CatalogStorageFormat(None, None, None, None, false, Map.empty), - schema = new StructType().add("a", "int").add("b", "string"), + schema = schema, provider = Some("hive") ) @@ -705,7 +744,7 @@ abstract class ExternalCatalogSuite extends SparkFunSuite with BeforeAndAfterEac storage = CatalogStorageFormat( Some(Utils.createTempDir().getAbsolutePath), None, None, None, false, Map.empty), - schema = new StructType().add("a", "int").add("b", "string"), + schema = schema, provider = Some("hive") ) catalog.createTable(externalTable, ignoreIfExists = false) @@ -718,11 +757,7 @@ abstract class ExternalCatalogSuite extends SparkFunSuite with BeforeAndAfterEac identifier = TableIdentifier("tbl", Some("db1")), tableType = CatalogTableType.MANAGED, storage = CatalogStorageFormat(None, None, None, None, false, Map.empty), - schema = new StructType() - .add("col1", "int") - .add("col2", "string") - .add("partCol1", "int") - .add("partCol2", "string"), + schema = schema, provider = Some("hive"), partitionColumnNames = Seq("partCol1", "partCol2")) catalog.createTable(table, ignoreIfExists = false) @@ -808,6 +843,11 @@ abstract class CatalogTestUtils { CatalogTablePartition(Map("a" -> "5", "b" -> "6", "c" -> "7"), storageFormat) lazy val partWithUnknownColumns = CatalogTablePartition(Map("a" -> "5", "unknown" -> "6"), storageFormat) + lazy val field1 = StructField("col1", IntegerType) + lazy val field2 = StructField("col2", StringType, nullable = false).withComment("some blabla...") + lazy val field3 = StructField("a", IntegerType, nullable = false).withComment(s"^*`%?") + lazy val field4 = StructField("b", StringType) + lazy val schema = StructType(field1 :: field2 :: field3 :: field4 :: Nil) lazy val funcClass = "org.apache.spark.myFunc" /** @@ -850,11 +890,7 @@ abstract class CatalogTestUtils { identifier = TableIdentifier(name, database), tableType = CatalogTableType.EXTERNAL, storage = storageFormat.copy(locationUri = Some(Utils.createTempDir().getAbsolutePath)), - schema = new StructType() - .add("col1", "int") - .add("col2", "string") - .add("a", "int") - .add("b", "string"), + schema = schema, provider = Some("hive"), partitionColumnNames = Seq("a", "b"), bucketSpec = Some(BucketSpec(4, Seq("col1"), Nil))) @@ -876,4 +912,12 @@ abstract class CatalogTestUtils { catalog.listPartitions(db, table).map(_.spec).toSet == parts.map(_.spec).toSet } + /** + * Whether the columns equal the ones given. + * Node: HIVE column don't have `nullable` field, so we should rewrite this function for + * HiveExternalCatalogSuite. + */ + def columnsEqual(columns: Seq[StructField], others: Seq[StructField]): Boolean = { + columns == others + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala index 5cc772d8e9a1e..b51f107a10ad7 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala @@ -945,6 +945,54 @@ class SessionCatalogSuite extends SparkFunSuite { expectedParts.map(p => p.copy(storage = p.storage.copy(locationUri = None))).toSet } + // -------------------------------------------------------------------------- + // Columns + // -------------------------------------------------------------------------- + + test("alter column comments") { + val externalCatalog = newBasicCatalog() + val sessionCatalog = new SessionCatalog(externalCatalog) + assert(columnsEqual(externalCatalog.listColumns("db2", "tbl1"), schema.fields.toList)) + val columnComments = Map(field1.name -> "insert comment", field2.name -> s"^*`%?", + field3.name -> "change comment") + sessionCatalog.alterColumnComments(TableIdentifier("tbl1", Some("db2")), columnComments) + val newColumns = field1.withComment("insert comment") :: field2.withComment(s"^*`%?") :: + field3.withComment("change comment") :: field4 :: Nil + assert(columnsEqual(externalCatalog.listColumns("db2", "tbl1"), newColumns)) + } + + test("alter column comments when database/table does not exist") { + val catalog = new SessionCatalog(newBasicCatalog()) + val columnComments = Map(field1.name -> "insert comment") + intercept[NoSuchDatabaseException] { + catalog.alterColumnComments(TableIdentifier("tbl1", Some("unknown_db")), columnComments) + } + intercept[NoSuchTableException] { + catalog.alterColumnComments(TableIdentifier("unknown_table", Some("db2")), columnComments) + } + } + + test("alter column comments when column does not exist") { + val externalCatalog = newBasicCatalog() + val sessionCatalog = new SessionCatalog(externalCatalog) + val columnComments = Map("unknown_col" -> "insert comment") + sessionCatalog.alterColumnComments(TableIdentifier("tbl1", Some("db2")), columnComments) + // Columns are not changed. + assert(columnsEqual(externalCatalog.listColumns("db2", "tbl1"), schema.fields.toList)) + } + + test("list columns") { + val catalog = new SessionCatalog(newBasicCatalog()) + assert(columnsEqual( + catalog.listColumns(TableIdentifier("tbl1", Some("db2"))), schema.fields.toList)) + intercept[NoSuchDatabaseException] { + catalog.listColumns(TableIdentifier("tbl1", Some("unknown_db"))) + } + intercept[NoSuchTableException] { + catalog.listColumns(TableIdentifier("unknown_table", Some("db2"))) + } + } + // -------------------------------------------------------------------------- // Functions // -------------------------------------------------------------------------- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index 4400174e92727..ab22bb6a36fde 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -876,6 +876,21 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { visitLocationSpec(ctx.locationSpec)) } + /** + * Create a [[AlterTableChangeColumnsCommand]] command. + * + * For example: + * {{{ + * ALTER TABLE table CHANGE COLUMN `col` `col` dataType "comment" (FIRST | AFTER `otherCol`) + * [, `col2` `col2` dataType "comment" (FIRST | AFTER `otherCol`), ...] + * ALTER VIEW view CHANGE COLUMN `col` `col` dataType "comment" (FIRST | AFTER `otherCol`) + * [, `col2` `col2` dataType "comment" (FIRST | AFTER `otherCol`), ...] + * }}} + */ +// override def visitChangeColumns(ctx: ChangeColumnsContext): LogicalPlan = withOrigin(ctx) { +// AlterTableChangeColumnsCommand() +// } + /** * Create location string. */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala index 32e2f75737394..d5ed9092cedc7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala @@ -166,6 +166,66 @@ case class AlterTableRenameCommand( } +/** + * A command to change the columns for a table, only support change column comment for now. + * This function creates a [[AlterTableChangeColumnsCommand]] logical plan. + * + * The syntax of using this command in SQL is: + * {{{ + * ALTER (TABLE | VIEW) table_identifier + * CHANGE (COLUMN) column_name column_name column_dataType column_comment + * (FIRST | AFTER column_name); + * }}} + */ +case class AlterTableChangeColumnsCommand( + tableName: TableIdentifier, + columns: Map[String, StructField], + isView: Boolean) extends RunnableCommand { + + // TODO: support change column name/dataType/metadata/index. + override def run(sparkSession: SparkSession): Seq[Row] = { + val catalog = sparkSession.sessionState.catalog + // If this is a temp view, throw an [[AnalysisException]] because we don't change the schema + // of a temp view. + if (catalog.isTemporaryTable(tableName)) { + throw new AnalysisException( + s"ALTER TABLE CHANGE COLUMN can't apply to a temporary view ${tableName}.") + } else { + val table = catalog.getTableMetadata(tableName) + DDLUtils.verifyAlterTableType(catalog, table, isView) + // Assert we only alter the column's comment, other fields e.g. name/dataType remain + // unchanged. + columns.foreach { pair => + val originColumn = table.schema.collectFirst { + case field if field.name == pair._1 => field + } + val newColumn = pair._2 + val unchanged = originColumn.map(equalIgnoreComment(_, newColumn)).getOrElse(true) + if (!unchanged) { + throw new AnalysisException( + s"ALTER TABLE CHANGE COLUMN don't support change column " + + s"'${originColumn.get.getDesc}' to '${newColumn.getDesc}'") + } + } + + // Create map from columns to new comments. + val columnComments = columns.map { + case (col, field: StructField) if field.getComment().isDefined => + (col, field.getComment.get) + } + catalog.alterColumnComments(tableName, columnComments) + } + + Seq.empty[Row] + } + + // Compare a [[StructField]] to another, return true if they have equal values except for + // comment. + private def equalIgnoreComment(field: StructField, other: StructField): Boolean = { + field.withComment("") == other.withComment("") + } +} + /** * A command that loads data into a Hive table. * diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala index f67ddc9be1a5a..3c324da5f693b 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala @@ -1007,6 +1007,27 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat } } + // -------------------------------------------------------------------------- + // Columns + // -------------------------------------------------------------------------- + + override def alterColumnComments( + db: String, + table: String, + columnComments: Map[String, String]): Unit = withClient { + val oldTable = client.getTable(db, table) + val newSchema = oldTable.schema.fields.map { field => + // If `columns` contains field name, update the field with new comment, + // else respect the field. + columnComments.get(field.name).map(field.withComment(_)).getOrElse(field) + } + client.alterTable(oldTable.copy(schema = StructType(newSchema))) + } + + override def listColumns(db: String, table: String): Seq[StructField] = withClient { + client.getTable(db, table).schema.fields + } + // -------------------------------------------------------------------------- // Functions // -------------------------------------------------------------------------- @@ -1049,7 +1070,6 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat requireDbExists(db) client.listFunctions(db, pattern) } - } object HiveExternalCatalog { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogSuite.scala index 6fee45824ea3f..12ad74f167735 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogSuite.scala @@ -24,7 +24,7 @@ import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.execution.command.DDLUtils -import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.types.{StructField, StructType} /** * Test suite for the [[HiveExternalCatalog]]. @@ -41,6 +41,12 @@ class HiveExternalCatalogSuite extends ExternalCatalogSuite { override val tableInputFormat: String = "org.apache.hadoop.mapred.SequenceFileInputFormat" override val tableOutputFormat: String = "org.apache.hadoop.mapred.SequenceFileOutputFormat" override def newEmptyCatalog(): ExternalCatalog = externalCatalog + // Compare columns ignore nullability. + override def columnsEqual(columns: Seq[StructField], others: Seq[StructField]): Boolean = { + val normalColumns = columns.map(_.copy(nullable = true)) + val normalOthers = others.map(_.copy(nullable = true)) + normalColumns == normalOthers + } } protected override def resetState(): Unit = { From d134b5f2347522e2b4671d930de876bbbf1ee0d2 Mon Sep 17 00:00:00 2001 From: jiangxingbo Date: Tue, 1 Nov 2016 23:50:45 +0800 Subject: [PATCH 02/16] implement parser for ALTER TABLE CHANGE COLUMN --- .../spark/sql/catalyst/parser/SqlBase.g4 | 14 +++++++++----- .../catalyst/catalog/InMemoryCatalog.scala | 3 +-- .../spark/sql/execution/SparkSqlParser.scala | 19 +++++++++++++++---- 3 files changed, 25 insertions(+), 11 deletions(-) diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index 87ad60eaa1304..a9e534a08d381 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -90,7 +90,7 @@ statement | ALTER (TABLE | VIEW) tableIdentifier UNSET TBLPROPERTIES (IF EXISTS)? tablePropertyList #unsetTableProperties | ALTER (TABLE | VIEW) tableIdentifier - CHANGE (COLUMN)? changeColTypeList #changeColumns + CHANGE COLUMN? expandColTypeList #changeColumns | ALTER TABLE tableIdentifier (partitionSpec)? SET SERDE STRING (WITH SERDEPROPERTIES tablePropertyList)? #setTableSerDe | ALTER TABLE tableIdentifier (partitionSpec)? @@ -580,6 +580,10 @@ intervalValue | STRING ; +colIndex + : FIRST | AFTER identifier + ; + dataType : complex=ARRAY '<' dataType '>' #complexDataType | complex=MAP '<' dataType ',' dataType '>' #complexDataType @@ -595,12 +599,12 @@ colType : identifier dataType (COMMENT STRING)? ; -changeColTypeList - : changeColType (',' changeColType)* +expandColTypeList + : expandColType (',' expandColType)* ; -changeColType - : identifier colType (FIRST | AFTER identifier)? +expandColType + : identifier colType colIndex? ; complexColTypeList diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala index 52a729c1614c6..d8b45d1b6976f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala @@ -19,8 +19,6 @@ package org.apache.spark.sql.catalyst.catalog import java.io.IOException -import org.apache.spark.sql.types.{StructField, StructType} - import scala.collection.mutable import org.apache.hadoop.conf.Configuration @@ -33,6 +31,7 @@ import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils.escapePathName import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.catalyst.util.StringUtils +import org.apache.spark.sql.types.{StructField, StructType} /** * An in-memory (ephemeral) implementation of the system catalog. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index ab22bb6a36fde..a5d942a750d4d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -31,7 +31,7 @@ import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, OneRowRelation, import org.apache.spark.sql.execution.command._ import org.apache.spark.sql.execution.datasources.{CreateTable, _} import org.apache.spark.sql.internal.{HiveSerDe, SQLConf, VariableSubstitution} -import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.types.{StructField, StructType} /** * Concrete parser for Spark SQL statements. @@ -887,9 +887,20 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { * [, `col2` `col2` dataType "comment" (FIRST | AFTER `otherCol`), ...] * }}} */ -// override def visitChangeColumns(ctx: ChangeColumnsContext): LogicalPlan = withOrigin(ctx) { -// AlterTableChangeColumnsCommand() -// } + override def visitChangeColumns(ctx: ChangeColumnsContext): LogicalPlan = withOrigin(ctx) { + val tableName = visitTableIdentifier(ctx.tableIdentifier) + val isView = ctx.VIEW != null + + val columns = ctx.expandColTypeList.expandColType.asScala.map { col => + if (col.colIndex != null) { + throw new AnalysisException( + s"Specifying column index in ALTER TABLE CHANGE COLUMN is not supported yet.") + } + col.identifier.getText -> visitColType(col.colType) + }.toMap + + AlterTableChangeColumnsCommand(tableName, columns, isView) + } /** * Create location string. From a4aa6f136d9b593c320520684f51a2f4738219da Mon Sep 17 00:00:00 2001 From: jiangxingbo Date: Mon, 7 Nov 2016 23:53:03 +0800 Subject: [PATCH 03/16] add test cases --- .../spark/sql/catalyst/parser/SqlBase.g4 | 6 +- .../apache/spark/sql/types/StructField.scala | 2 +- .../spark/sql/execution/SparkSqlParser.scala | 4 +- .../spark/sql/execution/command/ddl.scala | 62 ++++ .../spark/sql/execution/command/tables.scala | 60 ---- .../sql-tests/inputs/change-column.sql | 50 +++ .../sql-tests/results/change-column.sql.out | 327 ++++++++++++++++++ .../execution/command/DDLCommandSuite.scala | 9 +- 8 files changed, 447 insertions(+), 73 deletions(-) create mode 100644 sql/core/src/test/resources/sql-tests/inputs/change-column.sql create mode 100644 sql/core/src/test/resources/sql-tests/results/change-column.sql.out diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index a9e534a08d381..8a0ee76acdb88 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -194,7 +194,6 @@ unsupportedHiveNativeCommands | kw1=ALTER kw2=TABLE tableIdentifier partitionSpec? kw3=CONCATENATE | kw1=ALTER kw2=TABLE tableIdentifier partitionSpec? kw3=SET kw4=FILEFORMAT | kw1=ALTER kw2=TABLE tableIdentifier partitionSpec? kw3=ADD kw4=COLUMNS - | kw1=ALTER kw2=TABLE tableIdentifier partitionSpec? kw3=CHANGE kw4=COLUMN? | kw1=ALTER kw2=TABLE tableIdentifier partitionSpec? kw3=REPLACE kw4=COLUMNS | kw1=START kw2=TRANSACTION | kw1=COMMIT @@ -580,7 +579,7 @@ intervalValue | STRING ; -colIndex +colPosition : FIRST | AFTER identifier ; @@ -604,7 +603,7 @@ expandColTypeList ; expandColType - : identifier colType colIndex? + : identifier colType colPosition? ; complexColTypeList @@ -773,6 +772,7 @@ PRECEDING: 'PRECEDING'; FOLLOWING: 'FOLLOWING'; CURRENT: 'CURRENT'; FIRST: 'FIRST'; +AFTER: 'AFTER'; LAST: 'LAST'; ROW: 'ROW'; WITH: 'WITH'; diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructField.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructField.scala index 3a3281642c786..07d77d2dad54b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructField.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructField.scala @@ -79,6 +79,6 @@ case class StructField( * Return the full description of the StructField. */ def getDesc(): String = { - s"StructField($name,$dataType,$nullable$metadata)" + s"StructField($name,$dataType,$nullable,$metadata)" } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index a5d942a750d4d..ba2b95d54aaed 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -892,9 +892,9 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { val isView = ctx.VIEW != null val columns = ctx.expandColTypeList.expandColType.asScala.map { col => - if (col.colIndex != null) { + if (col.colPosition != null) { throw new AnalysisException( - s"Specifying column index in ALTER TABLE CHANGE COLUMN is not supported yet.") + s"Specifying column position in ALTER TABLE CHANGE COLUMN is not supported yet.") } col.identifier.getText -> visitColType(col.colType) }.toMap diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala index c62c14200c24a..9b8cbcb211aa2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala @@ -274,6 +274,68 @@ case class AlterTableUnsetPropertiesCommand( } + +/** + * A command to change the columns for a table, only support change column comment for now. + * This function creates a [[AlterTableChangeColumnsCommand]] logical plan. + * + * The syntax of using this command in SQL is: + * {{{ + * ALTER (TABLE | VIEW) table_identifier + * CHANGE (COLUMN) column_name column_name column_dataType column_comment + * (FIRST | AFTER column_name); + * }}} + */ +case class AlterTableChangeColumnsCommand( + tableName: TableIdentifier, + columns: Map[String, StructField], + isView: Boolean) extends RunnableCommand { + + // TODO: support change column name/dataType/metadata/position. + override def run(sparkSession: SparkSession): Seq[Row] = { + val catalog = sparkSession.sessionState.catalog + // If this is a temp view, throw an [[AnalysisException]] because we don't change the schema + // of a temp view. + if (catalog.isTemporaryTable(tableName)) { + throw new AnalysisException( + s"ALTER TABLE CHANGE COLUMN can't apply to a temporary view ${tableName}.") + } else { + val table = catalog.getTableMetadata(tableName) + DDLUtils.verifyAlterTableType(catalog, table, isView) + // Assert we only alter the column's comment, other fields e.g. name/dataType remain + // unchanged. + columns.foreach { pair => + val originColumn = table.schema.collectFirst { + case field if field.name == pair._1 => field + } + val newColumn = pair._2 + val unchanged = originColumn.map(equalIgnoreComment(_, newColumn)).getOrElse(true) + if (!unchanged) { + throw new AnalysisException( + s"ALTER TABLE CHANGE COLUMN don't support change column " + + s"'${originColumn.get.getDesc}' to '${newColumn.getDesc}'") + } + } + + // Create map from columns to new comments. + val columnComments = columns.map { + case (col, field: StructField) if field.getComment().isDefined => + (col, field.getComment.get) + } + catalog.alterColumnComments(tableName, columnComments) + } + + Seq.empty[Row] + } + + // Compare a [[StructField]] to another, return true if they have equal values except for + // comment. + private def equalIgnoreComment(field: StructField, other: StructField): Boolean = { + field.withComment("") == other.withComment("") + } + +} + /** * A command that sets the serde class and/or serde properties of a table/view. * diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala index d5ed9092cedc7..32e2f75737394 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala @@ -166,66 +166,6 @@ case class AlterTableRenameCommand( } -/** - * A command to change the columns for a table, only support change column comment for now. - * This function creates a [[AlterTableChangeColumnsCommand]] logical plan. - * - * The syntax of using this command in SQL is: - * {{{ - * ALTER (TABLE | VIEW) table_identifier - * CHANGE (COLUMN) column_name column_name column_dataType column_comment - * (FIRST | AFTER column_name); - * }}} - */ -case class AlterTableChangeColumnsCommand( - tableName: TableIdentifier, - columns: Map[String, StructField], - isView: Boolean) extends RunnableCommand { - - // TODO: support change column name/dataType/metadata/index. - override def run(sparkSession: SparkSession): Seq[Row] = { - val catalog = sparkSession.sessionState.catalog - // If this is a temp view, throw an [[AnalysisException]] because we don't change the schema - // of a temp view. - if (catalog.isTemporaryTable(tableName)) { - throw new AnalysisException( - s"ALTER TABLE CHANGE COLUMN can't apply to a temporary view ${tableName}.") - } else { - val table = catalog.getTableMetadata(tableName) - DDLUtils.verifyAlterTableType(catalog, table, isView) - // Assert we only alter the column's comment, other fields e.g. name/dataType remain - // unchanged. - columns.foreach { pair => - val originColumn = table.schema.collectFirst { - case field if field.name == pair._1 => field - } - val newColumn = pair._2 - val unchanged = originColumn.map(equalIgnoreComment(_, newColumn)).getOrElse(true) - if (!unchanged) { - throw new AnalysisException( - s"ALTER TABLE CHANGE COLUMN don't support change column " + - s"'${originColumn.get.getDesc}' to '${newColumn.getDesc}'") - } - } - - // Create map from columns to new comments. - val columnComments = columns.map { - case (col, field: StructField) if field.getComment().isDefined => - (col, field.getComment.get) - } - catalog.alterColumnComments(tableName, columnComments) - } - - Seq.empty[Row] - } - - // Compare a [[StructField]] to another, return true if they have equal values except for - // comment. - private def equalIgnoreComment(field: StructField, other: StructField): Boolean = { - field.withComment("") == other.withComment("") - } -} - /** * A command that loads data into a Hive table. * diff --git a/sql/core/src/test/resources/sql-tests/inputs/change-column.sql b/sql/core/src/test/resources/sql-tests/inputs/change-column.sql new file mode 100644 index 0000000000000..4c158b21582e1 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/change-column.sql @@ -0,0 +1,50 @@ +-- Create the origin table +CREATE TABLE test_change(a Int, b String, c Int); +CREATE VIEW test_view(a, b, c) AS +SELECT * FROM VALUES (1, "one", 11), (null, "two", 22) AS testData(a, b, c); +DESC test_change; +DESC test_view; + +-- Change column name (not supported yet) +ALTER TABLE test_change CHANGE a a1 Int; +ALTER TABLE test_change CHANGE b b1 String, c c1 Int; +DESC test_change; +ALTER VIEW test_view CHANGE a a1 Int; +DESC test_view; + +-- Change column dataType (not supported yet) +ALTER TABLE test_change CHANGE a a String; +ALTER TABLE test_change CHANGE b b Int, c c Double; +DESC test_change; +ALTER VIEW test_view CHANGE a a String; +DESC test_view; + +-- Change column position (not supported yet) +ALTER TABLE test_change CHANGE a a Int AFTER b; +ALTER TABLE test_change CHANGE b b String FIRST, c c Int AFTER b; +DESC test_change; +ALTER VIEW test_view CHANGE c c Int FIRST; +DESC test_view; + +-- Change column comment +ALTER TABLE test_change CHANGE a a Int COMMENT 'this is column a'; +ALTER TABLE test_change CHANGE b b String COMMENT '#*02?`', c c Int COMMENT ''; +DESC test_change; +ALTER VIEW test_view CHANGE a a Int COMMENT 'this is column a'; +DESC test_view; + +-- Change column name/dataType/position/comment together (not supported yet) +ALTER TABLE test_change CHANGE a a1 String COMMENT 'this is column a1' AFTER b; +DESC test_change; +ALTER VIEW test_view CHANGE a a1 String COMMENT 'this is column a1' AFTER b; +DESC test_view; + +-- Change column can't apply to a temporary/global_temporary view +CREATE TEMPORARY VIEW temp_view(a, b) AS SELECT 1, "one"; +ALTER VIEW temp_view CHANGE a a Int COMMENT 'this is column a'; +CREATE GLOBAL TEMPORARY VIEW global_temp_view(a, b) AS SELECT 1, "one"; +ALTER VIEW global_temp.global_temp_view CHANGE a a Int COMMENT 'this is column a'; + +-- DROP TEST TABLE +DROP TABLE test_change; +DROP VIEW test_view; diff --git a/sql/core/src/test/resources/sql-tests/results/change-column.sql.out b/sql/core/src/test/resources/sql-tests/results/change-column.sql.out new file mode 100644 index 0000000000000..9eeb1f70c3176 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/change-column.sql.out @@ -0,0 +1,327 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 34 + + +-- !query 0 +CREATE TABLE test_change(a Int, b String, c Int) +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +CREATE VIEW test_view(a, b, c) AS +SELECT * FROM VALUES (1, "one", 11), (null, "two", 22) AS testData(a, b, c) +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +DESC test_change +-- !query 2 schema +struct +-- !query 2 output +a int +b string +c int + + +-- !query 3 +DESC test_view +-- !query 3 schema +struct +-- !query 3 output +a int +b string +c int + + +-- !query 4 +ALTER TABLE test_change CHANGE a a1 Int +-- !query 4 schema +struct<> +-- !query 4 output +org.apache.spark.sql.AnalysisException +ALTER TABLE CHANGE COLUMN don't support change column 'StructField(a,IntegerType,true,{})' to 'StructField(a1,IntegerType,true,{})'; + + +-- !query 5 +ALTER TABLE test_change CHANGE b b1 String, c c1 Int +-- !query 5 schema +struct<> +-- !query 5 output +org.apache.spark.sql.AnalysisException +ALTER TABLE CHANGE COLUMN don't support change column 'StructField(b,StringType,true,{})' to 'StructField(b1,StringType,true,{})'; + + +-- !query 6 +DESC test_change +-- !query 6 schema +struct +-- !query 6 output +a int +b string +c int + + +-- !query 7 +ALTER VIEW test_view CHANGE a a1 Int +-- !query 7 schema +struct<> +-- !query 7 output +org.apache.spark.sql.AnalysisException +ALTER TABLE CHANGE COLUMN don't support change column 'StructField(a,IntegerType,true,{})' to 'StructField(a1,IntegerType,true,{})'; + + +-- !query 8 +DESC test_view +-- !query 8 schema +struct +-- !query 8 output +a int +b string +c int + + +-- !query 9 +ALTER TABLE test_change CHANGE a a String +-- !query 9 schema +struct<> +-- !query 9 output +org.apache.spark.sql.AnalysisException +ALTER TABLE CHANGE COLUMN don't support change column 'StructField(a,IntegerType,true,{})' to 'StructField(a,StringType,true,{})'; + + +-- !query 10 +ALTER TABLE test_change CHANGE b b Int, c c Double +-- !query 10 schema +struct<> +-- !query 10 output +org.apache.spark.sql.AnalysisException +ALTER TABLE CHANGE COLUMN don't support change column 'StructField(b,StringType,true,{})' to 'StructField(b,IntegerType,true,{})'; + + +-- !query 11 +DESC test_change +-- !query 11 schema +struct +-- !query 11 output +a int +b string +c int + + +-- !query 12 +ALTER VIEW test_view CHANGE a a String +-- !query 12 schema +struct<> +-- !query 12 output +org.apache.spark.sql.AnalysisException +ALTER TABLE CHANGE COLUMN don't support change column 'StructField(a,IntegerType,true,{})' to 'StructField(a,StringType,true,{})'; + + +-- !query 13 +DESC test_view +-- !query 13 schema +struct +-- !query 13 output +a int +b string +c int + + +-- !query 14 +ALTER TABLE test_change CHANGE a a Int AFTER b +-- !query 14 schema +struct<> +-- !query 14 output +org.apache.spark.sql.catalyst.parser.ParseException + +Specifying column position in ALTER TABLE CHANGE COLUMN is not supported yet. +== SQL == +ALTER TABLE test_change CHANGE a a Int AFTER b + + +-- !query 15 +ALTER TABLE test_change CHANGE b b String FIRST, c c Int AFTER b +-- !query 15 schema +struct<> +-- !query 15 output +org.apache.spark.sql.catalyst.parser.ParseException + +Specifying column position in ALTER TABLE CHANGE COLUMN is not supported yet. +== SQL == +ALTER TABLE test_change CHANGE b b String FIRST, c c Int AFTER b + + +-- !query 16 +DESC test_change +-- !query 16 schema +struct +-- !query 16 output +a int +b string +c int + + +-- !query 17 +ALTER VIEW test_view CHANGE c c Int FIRST +-- !query 17 schema +struct<> +-- !query 17 output +org.apache.spark.sql.catalyst.parser.ParseException + +Specifying column position in ALTER TABLE CHANGE COLUMN is not supported yet. +== SQL == +ALTER VIEW test_view CHANGE c c Int FIRST + + +-- !query 18 +DESC test_view +-- !query 18 schema +struct +-- !query 18 output +a int +b string +c int + + +-- !query 19 +ALTER TABLE test_change CHANGE a a Int COMMENT 'this is column a' +-- !query 19 schema +struct<> +-- !query 19 output + + + +-- !query 20 +ALTER TABLE test_change CHANGE b b String COMMENT '#*02?`', c c Int COMMENT '' +-- !query 20 schema +struct<> +-- !query 20 output + + + +-- !query 21 +DESC test_change +-- !query 21 schema +struct +-- !query 21 output +a int this is column a +b string #*02?` +c int + + +-- !query 22 +ALTER VIEW test_view CHANGE a a Int COMMENT 'this is column a' +-- !query 22 schema +struct<> +-- !query 22 output + + + +-- !query 23 +DESC test_view +-- !query 23 schema +struct +-- !query 23 output +a int this is column a +b string +c int + + +-- !query 24 +ALTER TABLE test_change CHANGE a a1 String COMMENT 'this is column a1' AFTER b +-- !query 24 schema +struct<> +-- !query 24 output +org.apache.spark.sql.catalyst.parser.ParseException + +Specifying column position in ALTER TABLE CHANGE COLUMN is not supported yet. +== SQL == +ALTER TABLE test_change CHANGE a a1 String COMMENT 'this is column a1' AFTER b + + +-- !query 25 +DESC test_change +-- !query 25 schema +struct +-- !query 25 output +a int this is column a +b string #*02?` +c int + + +-- !query 26 +ALTER VIEW test_view CHANGE a a1 String COMMENT 'this is column a1' AFTER b +-- !query 26 schema +struct<> +-- !query 26 output +org.apache.spark.sql.catalyst.parser.ParseException + +Specifying column position in ALTER TABLE CHANGE COLUMN is not supported yet. +== SQL == +ALTER VIEW test_view CHANGE a a1 String COMMENT 'this is column a1' AFTER b + + +-- !query 27 +DESC test_view +-- !query 27 schema +struct +-- !query 27 output +a int this is column a +b string +c int + + +-- !query 28 +CREATE TEMPORARY VIEW temp_view(a, b) AS SELECT 1, "one" +-- !query 28 schema +struct<> +-- !query 28 output + + + +-- !query 29 +ALTER VIEW temp_view CHANGE a a Int COMMENT 'this is column a' +-- !query 29 schema +struct<> +-- !query 29 output +org.apache.spark.sql.AnalysisException +ALTER TABLE CHANGE COLUMN can't apply to a temporary view `temp_view`.; + + +-- !query 30 +CREATE GLOBAL TEMPORARY VIEW global_temp_view(a, b) AS SELECT 1, "one" +-- !query 30 schema +struct<> +-- !query 30 output + + + +-- !query 31 +ALTER VIEW global_temp.global_temp_view CHANGE a a Int COMMENT 'this is column a' +-- !query 31 schema +struct<> +-- !query 31 output +org.apache.spark.sql.AnalysisException +ALTER TABLE CHANGE COLUMN can't apply to a temporary view `global_temp`.`global_temp_view`.; + + +-- !query 32 +DROP TABLE test_change +-- !query 32 schema +struct<> +-- !query 32 output + + + +-- !query 33 +DROP VIEW test_view +-- !query 33 schema +struct<> +-- !query 33 output + diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala index 5ef5f8ee77418..aa4d48982595e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala @@ -695,16 +695,11 @@ class DDLCommandSuite extends PlanTest { assertUnsupported("ALTER TABLE table_name SKEWED BY (key) ON (1,5,6) STORED AS DIRECTORIES") } - test("alter table: change column name/type/position/comment (not allowed)") { - assertUnsupported("ALTER TABLE table_name CHANGE col_old_name col_new_name INT") + test("alter table: change partition name/type/position/comment (not allowed)") { assertUnsupported( """ |ALTER TABLE table_name CHANGE COLUMN col_old_name col_new_name INT - |COMMENT 'col_comment' FIRST CASCADE - """.stripMargin) - assertUnsupported(""" - |ALTER TABLE table_name CHANGE COLUMN col_old_name col_new_name INT - |COMMENT 'col_comment' AFTER column_name RESTRICT + |COMMENT 'col_comment' AFTER column_name CASCADE """.stripMargin) } From dea5c272662eb122011bca238889ef44a3c216a7 Mon Sep 17 00:00:00 2001 From: jiangxingbo Date: Tue, 8 Nov 2016 14:35:50 +0800 Subject: [PATCH 04/16] test --- .../scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala index 3c324da5f693b..c7e1dd3849922 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala @@ -44,7 +44,7 @@ import org.apache.spark.sql.execution.datasources.PartitioningUtils import org.apache.spark.sql.hive.client.HiveClient import org.apache.spark.sql.internal.HiveSerDe import org.apache.spark.sql.internal.StaticSQLConf._ -import org.apache.spark.sql.types.{DataType, StructType} +import org.apache.spark.sql.types._ /** From 49d74fb30fc69938f9b5e19c54b5fc7275c2c930 Mon Sep 17 00:00:00 2001 From: jiangxingbo Date: Tue, 8 Nov 2016 17:05:05 +0800 Subject: [PATCH 05/16] Don't test RESTRICT/CASCADE for now. --- .../spark/sql/execution/command/DDLCommandSuite.scala | 8 -------- 1 file changed, 8 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala index aa4d48982595e..cf00dfec163eb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala @@ -695,14 +695,6 @@ class DDLCommandSuite extends PlanTest { assertUnsupported("ALTER TABLE table_name SKEWED BY (key) ON (1,5,6) STORED AS DIRECTORIES") } - test("alter table: change partition name/type/position/comment (not allowed)") { - assertUnsupported( - """ - |ALTER TABLE table_name CHANGE COLUMN col_old_name col_new_name INT - |COMMENT 'col_comment' AFTER column_name CASCADE - """.stripMargin) - } - test("alter table: add/replace columns (not allowed)") { assertUnsupported( """ From f6679429e57bf21d5fabdcf316791265aa9e7bfc Mon Sep 17 00:00:00 2001 From: jiangxingbo Date: Sun, 13 Nov 2016 00:18:19 +0800 Subject: [PATCH 06/16] bugfix --- .../main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index 8a0ee76acdb88..0dc3b76f098f8 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -682,7 +682,7 @@ number nonReserved : SHOW | TABLES | COLUMNS | COLUMN | PARTITIONS | FUNCTIONS | DATABASES | ADD - | OVER | PARTITION | RANGE | ROWS | PRECEDING | FOLLOWING | CURRENT | ROW | LAST | FIRST + | OVER | PARTITION | RANGE | ROWS | PRECEDING | FOLLOWING | CURRENT | ROW | LAST | FIRST | AFTER | MAP | ARRAY | STRUCT | LATERAL | WINDOW | REDUCE | TRANSFORM | USING | SERDE | SERDEPROPERTIES | RECORDREADER | DELIMITED | FIELDS | TERMINATED | COLLECTION | ITEMS | KEYS | ESCAPED | LINES | SEPARATED From ce6cfa3d6bec4acd97345910283489abbbd9a1ce Mon Sep 17 00:00:00 2001 From: jiangxingbo Date: Mon, 14 Nov 2016 01:04:28 +0800 Subject: [PATCH 07/16] remove ALTER VIEW CHANGE COLUMN statement. --- .../spark/sql/catalyst/parser/SqlBase.g4 | 2 +- .../catalog/ExternalCatalogSuite.scala | 6 +- .../spark/sql/execution/SparkSqlParser.scala | 5 +- .../spark/sql/execution/command/ddl.scala | 59 ++-- .../sql-tests/inputs/change-column.sql | 18 +- .../sql-tests/results/change-column.sql.out | 259 +++++------------- 6 files changed, 100 insertions(+), 249 deletions(-) diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index 0dc3b76f098f8..7e842682c77a0 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -89,7 +89,7 @@ statement SET TBLPROPERTIES tablePropertyList #setTableProperties | ALTER (TABLE | VIEW) tableIdentifier UNSET TBLPROPERTIES (IF EXISTS)? tablePropertyList #unsetTableProperties - | ALTER (TABLE | VIEW) tableIdentifier + | ALTER TABLE tableIdentifier CHANGE COLUMN? expandColTypeList #changeColumns | ALTER TABLE tableIdentifier (partitionSpec)? SET SERDE STRING (WITH SERDEPROPERTIES tablePropertyList)? #setTableSerDe diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogSuite.scala index d3bb31d24e237..7b4be28ae13cb 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogSuite.scala @@ -757,7 +757,11 @@ abstract class ExternalCatalogSuite extends SparkFunSuite with BeforeAndAfterEac identifier = TableIdentifier("tbl", Some("db1")), tableType = CatalogTableType.MANAGED, storage = CatalogStorageFormat(None, None, None, None, false, Map.empty), - schema = schema, + schema = new StructType() + .add("col1", "int") + .add("col2", "string") + .add("partCol1", "int") + .add("partCol2", "string"), provider = Some("hive"), partitionColumnNames = Seq("partCol1", "partCol2")) catalog.createTable(table, ignoreIfExists = false) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index ba2b95d54aaed..9c54765e67fc4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -883,13 +883,10 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { * {{{ * ALTER TABLE table CHANGE COLUMN `col` `col` dataType "comment" (FIRST | AFTER `otherCol`) * [, `col2` `col2` dataType "comment" (FIRST | AFTER `otherCol`), ...] - * ALTER VIEW view CHANGE COLUMN `col` `col` dataType "comment" (FIRST | AFTER `otherCol`) - * [, `col2` `col2` dataType "comment" (FIRST | AFTER `otherCol`), ...] * }}} */ override def visitChangeColumns(ctx: ChangeColumnsContext): LogicalPlan = withOrigin(ctx) { val tableName = visitTableIdentifier(ctx.tableIdentifier) - val isView = ctx.VIEW != null val columns = ctx.expandColTypeList.expandColType.asScala.map { col => if (col.colPosition != null) { @@ -899,7 +896,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { col.identifier.getText -> visitColType(col.colType) }.toMap - AlterTableChangeColumnsCommand(tableName, columns, isView) + AlterTableChangeColumnsCommand(tableName, columns) } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala index 9b8cbcb211aa2..5392a97741bec 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala @@ -276,55 +276,46 @@ case class AlterTableUnsetPropertiesCommand( /** - * A command to change the columns for a table, only support change column comment for now. - * This function creates a [[AlterTableChangeColumnsCommand]] logical plan. + * A command to change the columns for a table, only support changing column comment for now. * * The syntax of using this command in SQL is: * {{{ - * ALTER (TABLE | VIEW) table_identifier - * CHANGE (COLUMN) column_name column_name column_dataType column_comment - * (FIRST | AFTER column_name); + * ALTER TABLE table_identifier + * CHANGE [COLUMN] column_old_name column_new_name column_dataType [COMMENT column_comment] + * [FIRST | AFTER column_name]; * }}} */ case class AlterTableChangeColumnsCommand( tableName: TableIdentifier, - columns: Map[String, StructField], - isView: Boolean) extends RunnableCommand { + columns: Map[String, StructField]) extends RunnableCommand { // TODO: support change column name/dataType/metadata/position. override def run(sparkSession: SparkSession): Seq[Row] = { val catalog = sparkSession.sessionState.catalog - // If this is a temp view, throw an [[AnalysisException]] because we don't change the schema - // of a temp view. - if (catalog.isTemporaryTable(tableName)) { - throw new AnalysisException( - s"ALTER TABLE CHANGE COLUMN can't apply to a temporary view ${tableName}.") - } else { - val table = catalog.getTableMetadata(tableName) - DDLUtils.verifyAlterTableType(catalog, table, isView) - // Assert we only alter the column's comment, other fields e.g. name/dataType remain - // unchanged. - columns.foreach { pair => - val originColumn = table.schema.collectFirst { - case field if field.name == pair._1 => field - } - val newColumn = pair._2 - val unchanged = originColumn.map(equalIgnoreComment(_, newColumn)).getOrElse(true) - if (!unchanged) { - throw new AnalysisException( - s"ALTER TABLE CHANGE COLUMN don't support change column " + - s"'${originColumn.get.getDesc}' to '${newColumn.getDesc}'") - } + val table = catalog.getTableMetadata(tableName) + DDLUtils.verifyAlterTableType(catalog, table, false) + // Assert we only alter the column's comment, other fields e.g. name/dataType remain + // unchanged. + columns.foreach { pair => + val originColumn = table.schema.collectFirst { + case field if field.name == pair._1 => field } - - // Create map from columns to new comments. - val columnComments = columns.map { - case (col, field: StructField) if field.getComment().isDefined => - (col, field.getComment.get) + val newColumn = pair._2 + val unchanged = originColumn.map(equalIgnoreComment(_, newColumn)).getOrElse(true) + if (!unchanged) { + throw new AnalysisException( + s"ALTER TABLE CHANGE COLUMN don't support change column " + + s"'${originColumn.get.getDesc}' to '${newColumn.getDesc}'") } - catalog.alterColumnComments(tableName, columnComments) } + // Create map from columns to new comments. + val columnComments = columns.map { + case (col, field: StructField) if field.getComment().isDefined => + (col, field.getComment.get) + } + catalog.alterColumnComments(tableName, columnComments) + Seq.empty[Row] } diff --git a/sql/core/src/test/resources/sql-tests/inputs/change-column.sql b/sql/core/src/test/resources/sql-tests/inputs/change-column.sql index 4c158b21582e1..6e152ee309c15 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/change-column.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/change-column.sql @@ -1,50 +1,36 @@ -- Create the origin table CREATE TABLE test_change(a Int, b String, c Int); -CREATE VIEW test_view(a, b, c) AS -SELECT * FROM VALUES (1, "one", 11), (null, "two", 22) AS testData(a, b, c); DESC test_change; -DESC test_view; -- Change column name (not supported yet) ALTER TABLE test_change CHANGE a a1 Int; ALTER TABLE test_change CHANGE b b1 String, c c1 Int; DESC test_change; -ALTER VIEW test_view CHANGE a a1 Int; -DESC test_view; -- Change column dataType (not supported yet) ALTER TABLE test_change CHANGE a a String; ALTER TABLE test_change CHANGE b b Int, c c Double; DESC test_change; -ALTER VIEW test_view CHANGE a a String; -DESC test_view; -- Change column position (not supported yet) ALTER TABLE test_change CHANGE a a Int AFTER b; ALTER TABLE test_change CHANGE b b String FIRST, c c Int AFTER b; DESC test_change; -ALTER VIEW test_view CHANGE c c Int FIRST; -DESC test_view; -- Change column comment ALTER TABLE test_change CHANGE a a Int COMMENT 'this is column a'; ALTER TABLE test_change CHANGE b b String COMMENT '#*02?`', c c Int COMMENT ''; DESC test_change; -ALTER VIEW test_view CHANGE a a Int COMMENT 'this is column a'; -DESC test_view; -- Change column name/dataType/position/comment together (not supported yet) ALTER TABLE test_change CHANGE a a1 String COMMENT 'this is column a1' AFTER b; DESC test_change; -ALTER VIEW test_view CHANGE a a1 String COMMENT 'this is column a1' AFTER b; -DESC test_view; -- Change column can't apply to a temporary/global_temporary view CREATE TEMPORARY VIEW temp_view(a, b) AS SELECT 1, "one"; -ALTER VIEW temp_view CHANGE a a Int COMMENT 'this is column a'; +ALTER TABLE temp_view CHANGE a a Int COMMENT 'this is column a'; CREATE GLOBAL TEMPORARY VIEW global_temp_view(a, b) AS SELECT 1, "one"; -ALTER VIEW global_temp.global_temp_view CHANGE a a Int COMMENT 'this is column a'; +ALTER TABLE global_temp.global_temp_view CHANGE a a Int COMMENT 'this is column a'; -- DROP TEST TABLE DROP TABLE test_change; -DROP VIEW test_view; diff --git a/sql/core/src/test/resources/sql-tests/results/change-column.sql.out b/sql/core/src/test/resources/sql-tests/results/change-column.sql.out index 9eeb1f70c3176..15fe05775594e 100644 --- a/sql/core/src/test/resources/sql-tests/results/change-column.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/change-column.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 34 +-- Number of queries: 21 -- !query 0 @@ -11,133 +11,76 @@ struct<> -- !query 1 -CREATE VIEW test_view(a, b, c) AS -SELECT * FROM VALUES (1, "one", 11), (null, "two", 22) AS testData(a, b, c) --- !query 1 schema -struct<> --- !query 1 output - - - --- !query 2 DESC test_change --- !query 2 schema -struct --- !query 2 output -a int -b string -c int - - --- !query 3 -DESC test_view --- !query 3 schema +-- !query 1 schema struct --- !query 3 output +-- !query 1 output a int b string c int --- !query 4 +-- !query 2 ALTER TABLE test_change CHANGE a a1 Int --- !query 4 schema +-- !query 2 schema struct<> --- !query 4 output +-- !query 2 output org.apache.spark.sql.AnalysisException ALTER TABLE CHANGE COLUMN don't support change column 'StructField(a,IntegerType,true,{})' to 'StructField(a1,IntegerType,true,{})'; --- !query 5 +-- !query 3 ALTER TABLE test_change CHANGE b b1 String, c c1 Int --- !query 5 schema +-- !query 3 schema struct<> --- !query 5 output +-- !query 3 output org.apache.spark.sql.AnalysisException ALTER TABLE CHANGE COLUMN don't support change column 'StructField(b,StringType,true,{})' to 'StructField(b1,StringType,true,{})'; --- !query 6 +-- !query 4 DESC test_change --- !query 6 schema -struct --- !query 6 output -a int -b string -c int - - --- !query 7 -ALTER VIEW test_view CHANGE a a1 Int --- !query 7 schema -struct<> --- !query 7 output -org.apache.spark.sql.AnalysisException -ALTER TABLE CHANGE COLUMN don't support change column 'StructField(a,IntegerType,true,{})' to 'StructField(a1,IntegerType,true,{})'; - - --- !query 8 -DESC test_view --- !query 8 schema +-- !query 4 schema struct --- !query 8 output +-- !query 4 output a int b string c int --- !query 9 +-- !query 5 ALTER TABLE test_change CHANGE a a String --- !query 9 schema +-- !query 5 schema struct<> --- !query 9 output +-- !query 5 output org.apache.spark.sql.AnalysisException ALTER TABLE CHANGE COLUMN don't support change column 'StructField(a,IntegerType,true,{})' to 'StructField(a,StringType,true,{})'; --- !query 10 +-- !query 6 ALTER TABLE test_change CHANGE b b Int, c c Double --- !query 10 schema +-- !query 6 schema struct<> --- !query 10 output +-- !query 6 output org.apache.spark.sql.AnalysisException ALTER TABLE CHANGE COLUMN don't support change column 'StructField(b,StringType,true,{})' to 'StructField(b,IntegerType,true,{})'; --- !query 11 +-- !query 7 DESC test_change --- !query 11 schema -struct --- !query 11 output -a int -b string -c int - - --- !query 12 -ALTER VIEW test_view CHANGE a a String --- !query 12 schema -struct<> --- !query 12 output -org.apache.spark.sql.AnalysisException -ALTER TABLE CHANGE COLUMN don't support change column 'StructField(a,IntegerType,true,{})' to 'StructField(a,StringType,true,{})'; - - --- !query 13 -DESC test_view --- !query 13 schema +-- !query 7 schema struct --- !query 13 output +-- !query 7 output a int b string c int --- !query 14 +-- !query 8 ALTER TABLE test_change CHANGE a a Int AFTER b --- !query 14 schema +-- !query 8 schema struct<> --- !query 14 output +-- !query 8 output org.apache.spark.sql.catalyst.parser.ParseException Specifying column position in ALTER TABLE CHANGE COLUMN is not supported yet. @@ -145,11 +88,11 @@ Specifying column position in ALTER TABLE CHANGE COLUMN is not supported yet. ALTER TABLE test_change CHANGE a a Int AFTER b --- !query 15 +-- !query 9 ALTER TABLE test_change CHANGE b b String FIRST, c c Int AFTER b --- !query 15 schema +-- !query 9 schema struct<> --- !query 15 output +-- !query 9 output org.apache.spark.sql.catalyst.parser.ParseException Specifying column position in ALTER TABLE CHANGE COLUMN is not supported yet. @@ -157,87 +100,47 @@ Specifying column position in ALTER TABLE CHANGE COLUMN is not supported yet. ALTER TABLE test_change CHANGE b b String FIRST, c c Int AFTER b --- !query 16 +-- !query 10 DESC test_change --- !query 16 schema -struct --- !query 16 output -a int -b string -c int - - --- !query 17 -ALTER VIEW test_view CHANGE c c Int FIRST --- !query 17 schema -struct<> --- !query 17 output -org.apache.spark.sql.catalyst.parser.ParseException - -Specifying column position in ALTER TABLE CHANGE COLUMN is not supported yet. -== SQL == -ALTER VIEW test_view CHANGE c c Int FIRST - - --- !query 18 -DESC test_view --- !query 18 schema +-- !query 10 schema struct --- !query 18 output +-- !query 10 output a int b string c int --- !query 19 +-- !query 11 ALTER TABLE test_change CHANGE a a Int COMMENT 'this is column a' --- !query 19 schema +-- !query 11 schema struct<> --- !query 19 output +-- !query 11 output --- !query 20 +-- !query 12 ALTER TABLE test_change CHANGE b b String COMMENT '#*02?`', c c Int COMMENT '' --- !query 20 schema +-- !query 12 schema struct<> --- !query 20 output +-- !query 12 output --- !query 21 +-- !query 13 DESC test_change --- !query 21 schema +-- !query 13 schema struct --- !query 21 output +-- !query 13 output a int this is column a b string #*02?` c int --- !query 22 -ALTER VIEW test_view CHANGE a a Int COMMENT 'this is column a' --- !query 22 schema -struct<> --- !query 22 output - - - --- !query 23 -DESC test_view --- !query 23 schema -struct --- !query 23 output -a int this is column a -b string -c int - - --- !query 24 +-- !query 14 ALTER TABLE test_change CHANGE a a1 String COMMENT 'this is column a1' AFTER b --- !query 24 schema +-- !query 14 schema struct<> --- !query 24 output +-- !query 14 output org.apache.spark.sql.catalyst.parser.ParseException Specifying column position in ALTER TABLE CHANGE COLUMN is not supported yet. @@ -245,83 +148,53 @@ Specifying column position in ALTER TABLE CHANGE COLUMN is not supported yet. ALTER TABLE test_change CHANGE a a1 String COMMENT 'this is column a1' AFTER b --- !query 25 +-- !query 15 DESC test_change --- !query 25 schema +-- !query 15 schema struct --- !query 25 output +-- !query 15 output a int this is column a b string #*02?` c int --- !query 26 -ALTER VIEW test_view CHANGE a a1 String COMMENT 'this is column a1' AFTER b --- !query 26 schema -struct<> --- !query 26 output -org.apache.spark.sql.catalyst.parser.ParseException - -Specifying column position in ALTER TABLE CHANGE COLUMN is not supported yet. -== SQL == -ALTER VIEW test_view CHANGE a a1 String COMMENT 'this is column a1' AFTER b - - --- !query 27 -DESC test_view --- !query 27 schema -struct --- !query 27 output -a int this is column a -b string -c int - - --- !query 28 +-- !query 16 CREATE TEMPORARY VIEW temp_view(a, b) AS SELECT 1, "one" --- !query 28 schema +-- !query 16 schema struct<> --- !query 28 output +-- !query 16 output --- !query 29 -ALTER VIEW temp_view CHANGE a a Int COMMENT 'this is column a' --- !query 29 schema +-- !query 17 +ALTER TABLE temp_view CHANGE a a Int COMMENT 'this is column a' +-- !query 17 schema struct<> --- !query 29 output -org.apache.spark.sql.AnalysisException -ALTER TABLE CHANGE COLUMN can't apply to a temporary view `temp_view`.; +-- !query 17 output +org.apache.spark.sql.catalyst.analysis.NoSuchTableException +Table or view 'temp_view' not found in database 'default'; --- !query 30 +-- !query 18 CREATE GLOBAL TEMPORARY VIEW global_temp_view(a, b) AS SELECT 1, "one" --- !query 30 schema +-- !query 18 schema struct<> --- !query 30 output +-- !query 18 output --- !query 31 -ALTER VIEW global_temp.global_temp_view CHANGE a a Int COMMENT 'this is column a' --- !query 31 schema +-- !query 19 +ALTER TABLE global_temp.global_temp_view CHANGE a a Int COMMENT 'this is column a' +-- !query 19 schema struct<> --- !query 31 output -org.apache.spark.sql.AnalysisException -ALTER TABLE CHANGE COLUMN can't apply to a temporary view `global_temp`.`global_temp_view`.; +-- !query 19 output +org.apache.spark.sql.catalyst.analysis.NoSuchDatabaseException +Database 'global_temp' not found; --- !query 32 +-- !query 20 DROP TABLE test_change --- !query 32 schema -struct<> --- !query 32 output - - - --- !query 33 -DROP VIEW test_view --- !query 33 schema +-- !query 20 schema struct<> --- !query 33 output +-- !query 20 output From 9515a0944935563595696343d8a43b19cb8203a4 Mon Sep 17 00:00:00 2001 From: jiangxingbo Date: Wed, 16 Nov 2016 23:41:38 +0800 Subject: [PATCH 08/16] add test cases for data source tables --- .../sql/execution/command/DDLSuite.scala | 52 +++++++++++++++++-- 1 file changed, 47 insertions(+), 5 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala index e61beb49e47f4..cc70acbcc0ead 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala @@ -38,6 +38,12 @@ import org.apache.spark.util.Utils class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { private val escapedIdentifier = "`(.+)`".r + lazy val field1 = StructField("col1", IntegerType) + lazy val field2 = StructField("col2", StringType) + lazy val field3 = StructField("a", IntegerType) + lazy val field4 = StructField("b", StringType) + lazy val schema = StructType(field1 :: field2 :: field3 :: field4 :: Nil) + override def afterEach(): Unit = { try { // drop all databases, tables and functions after each test @@ -88,11 +94,7 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { identifier = name, tableType = CatalogTableType.EXTERNAL, storage = storage, - schema = new StructType() - .add("col1", "int") - .add("col2", "string") - .add("a", "int") - .add("b", "int"), + schema = schema, provider = Some("hive"), partitionColumnNames = Seq("a", "b"), createTime = 0L, @@ -815,6 +817,14 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { } } + test("alter table: change column comments") { + testChangeColumnComments(isDatasourceTable = false) + } + + test("alter table: change column comments (datasource table)") { + testChangeColumnComments(isDatasourceTable = true) + } + private def testRecoverPartitions() { val catalog = spark.sessionState.catalog // table to alter does not exist @@ -1368,6 +1378,38 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { Set(Map("a" -> "1", "b" -> "p"), Map("a" -> "20", "b" -> "c"), Map("a" -> "3", "b" -> "p"))) } + private def testChangeColumnComments(isDatasourceTable: Boolean): Unit = { + val catalog = spark.sessionState.catalog + val tableIdent = TableIdentifier("tab1", Some("dbx")) + createDatabase(catalog, "dbx") + createTable(catalog, tableIdent) + if (isDatasourceTable) { + convertToDatasourceTable(catalog, tableIdent) + } + + // alter column comments + val columnComments = Map(field1.name -> "insert comment", field2.name -> s"^*`%?", + field3.name -> "change comment") + catalog.alterColumnComments(tableIdent, columnComments) + val newColumns = field1.withComment("insert comment") :: field2.withComment(s"^*`%?") :: + field3.withComment("change comment") :: field4 :: Nil + assert(catalog.listColumns(tableIdent) == newColumns) + + // database/table does not exist + intercept[AnalysisException] { + catalog.alterColumnComments(TableIdentifier("unknown_table", Some("unknown_db")), + columnComments) + } + intercept[AnalysisException] { + catalog.alterColumnComments(TableIdentifier("unknown_table", Some("dbx")), columnComments) + } + + // column does not exist + val columnComments2 = Map("unknown_col" -> "insert comment") + catalog.alterColumnComments(tableIdent, columnComments2) + assert(catalog.listColumns(tableIdent) == newColumns) + } + test("drop build-in function") { Seq("true", "false").foreach { caseSensitive => withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive) { From 3e4be49fa5d363baf5eb11a76b42938c15c6eb71 Mon Sep 17 00:00:00 2001 From: jiangxingbo Date: Tue, 22 Nov 2016 16:45:54 +0800 Subject: [PATCH 09/16] resolve the parser/syntax issues. --- .../spark/sql/catalyst/parser/SqlBase.g4 | 2 +- .../spark/sql/execution/SparkSqlParser.scala | 14 ++++-- .../sql-tests/inputs/change-column.sql | 5 ++ .../sql-tests/results/change-column.sql.out | 46 +++++++++++++++++-- .../execution/command/DDLCommandSuite.scala | 28 ++++++++++- 5 files changed, 84 insertions(+), 11 deletions(-) diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index 7e842682c77a0..a44f47adb51fd 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -89,7 +89,7 @@ statement SET TBLPROPERTIES tablePropertyList #setTableProperties | ALTER (TABLE | VIEW) tableIdentifier UNSET TBLPROPERTIES (IF EXISTS)? tablePropertyList #unsetTableProperties - | ALTER TABLE tableIdentifier + | ALTER TABLE tableIdentifier partitionSpec? CHANGE COLUMN? expandColTypeList #changeColumns | ALTER TABLE tableIdentifier (partitionSpec)? SET SERDE STRING (WITH SERDEPROPERTIES tablePropertyList)? #setTableSerDe diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index 9c54765e67fc4..e76e54e79d40d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -881,17 +881,23 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { * * For example: * {{{ - * ALTER TABLE table CHANGE COLUMN `col` `col` dataType "comment" (FIRST | AFTER `otherCol`) - * [, `col2` `col2` dataType "comment" (FIRST | AFTER `otherCol`), ...] + * ALTER TABLE table [PARTITION partition_spec] + * CHANGE [COLUMN] `col` `col` dataType [COMMENT "comment"] [FIRST | AFTER `otherCol`] + * [, `col2` `col2` dataType [COMMENT "comment"] [FIRST | AFTER `otherCol`], ...] * }}} */ override def visitChangeColumns(ctx: ChangeColumnsContext): LogicalPlan = withOrigin(ctx) { + if (ctx.partitionSpec != null) { + operationNotAllowed(s"ALTER TABLE PARTITION partition_spec CHANGE COLUMN, can not change " + + "the columns in partition spec", ctx) + } + val tableName = visitTableIdentifier(ctx.tableIdentifier) val columns = ctx.expandColTypeList.expandColType.asScala.map { col => if (col.colPosition != null) { - throw new AnalysisException( - s"Specifying column position in ALTER TABLE CHANGE COLUMN is not supported yet.") + operationNotAllowed( + s"ALTER TABLE [PARTITION] CHANGE COLUMN ... FIRST | AFTER otherCol", ctx) } col.identifier.getText -> visitColType(col.colType) }.toMap diff --git a/sql/core/src/test/resources/sql-tests/inputs/change-column.sql b/sql/core/src/test/resources/sql-tests/inputs/change-column.sql index 6e152ee309c15..7e29c94d08906 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/change-column.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/change-column.sql @@ -32,5 +32,10 @@ ALTER TABLE temp_view CHANGE a a Int COMMENT 'this is column a'; CREATE GLOBAL TEMPORARY VIEW global_temp_view(a, b) AS SELECT 1, "one"; ALTER TABLE global_temp.global_temp_view CHANGE a a Int COMMENT 'this is column a'; +-- Change column in partition spec (not supported yet) +CREATE TABLE partition_table(a Int, b String) PARTITIONED BY (c Int, d String); +ALTER TABLE partition_table PARTITION (c = 1) CHANGE COLUMN a new_a Int; + -- DROP TEST TABLE DROP TABLE test_change; +DROP TABLE partition_table; diff --git a/sql/core/src/test/resources/sql-tests/results/change-column.sql.out b/sql/core/src/test/resources/sql-tests/results/change-column.sql.out index 15fe05775594e..82c1beee26b3c 100644 --- a/sql/core/src/test/resources/sql-tests/results/change-column.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/change-column.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 21 +-- Number of queries: 24 -- !query 0 @@ -83,9 +83,11 @@ struct<> -- !query 8 output org.apache.spark.sql.catalyst.parser.ParseException -Specifying column position in ALTER TABLE CHANGE COLUMN is not supported yet. +Operation not allowed: ALTER TABLE [PARTITION] CHANGE COLUMN ... FIRST | AFTER otherCol(line 1, pos 0) + == SQL == ALTER TABLE test_change CHANGE a a Int AFTER b +^^^ -- !query 9 @@ -95,9 +97,11 @@ struct<> -- !query 9 output org.apache.spark.sql.catalyst.parser.ParseException -Specifying column position in ALTER TABLE CHANGE COLUMN is not supported yet. +Operation not allowed: ALTER TABLE [PARTITION] CHANGE COLUMN ... FIRST | AFTER otherCol(line 1, pos 0) + == SQL == ALTER TABLE test_change CHANGE b b String FIRST, c c Int AFTER b +^^^ -- !query 10 @@ -143,9 +147,11 @@ struct<> -- !query 14 output org.apache.spark.sql.catalyst.parser.ParseException -Specifying column position in ALTER TABLE CHANGE COLUMN is not supported yet. +Operation not allowed: ALTER TABLE [PARTITION] CHANGE COLUMN ... FIRST | AFTER otherCol(line 1, pos 0) + == SQL == ALTER TABLE test_change CHANGE a a1 String COMMENT 'this is column a1' AFTER b +^^^ -- !query 15 @@ -193,8 +199,38 @@ Database 'global_temp' not found; -- !query 20 -DROP TABLE test_change +CREATE TABLE partition_table(a Int, b String) PARTITIONED BY (c Int, d String) -- !query 20 schema struct<> -- !query 20 output + + +-- !query 21 +ALTER TABLE partition_table PARTITION (c = 1) CHANGE COLUMN a new_a Int +-- !query 21 schema +struct<> +-- !query 21 output +org.apache.spark.sql.catalyst.parser.ParseException + +Operation not allowed: ALTER TABLE PARTITION partition_spec CHANGE COLUMN, can not change the columns in partition spec(line 1, pos 0) + +== SQL == +ALTER TABLE partition_table PARTITION (c = 1) CHANGE COLUMN a new_a Int +^^^ + + +-- !query 22 +DROP TABLE test_change +-- !query 22 schema +struct<> +-- !query 22 output + + + +-- !query 23 +DROP TABLE partition_table +-- !query 23 schema +struct<> +-- !query 23 output + diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala index cf00dfec163eb..98c47c354d6e0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala @@ -27,7 +27,7 @@ import org.apache.spark.sql.catalyst.plans.logical.Project import org.apache.spark.sql.execution.SparkSqlParser import org.apache.spark.sql.execution.datasources.CreateTable import org.apache.spark.sql.internal.{HiveSerDe, SQLConf} -import org.apache.spark.sql.types.{IntegerType, StringType, StructType} +import org.apache.spark.sql.types.{IntegerType, StringType, StructField, StructType} // TODO: merge this with DDLSuite (SPARK-14441) @@ -660,6 +660,32 @@ class DDLCommandSuite extends PlanTest { comparePlans(parsed2, expected2) } + test("alter table: change column name/type/comment") { + val sql1 = "ALTER TABLE table_name CHANGE COLUMN col_old_name col_new_name INT" + val sql2 = "ALTER TABLE table_name CHANGE COLUMN col_name col_name INT COMMENT 'new_comment'" + val parsed1 = parser.parsePlan(sql1) + val parsed2 = parser.parsePlan(sql2) + val tableIdent = TableIdentifier("table_name", None) + val expected1 = AlterTableChangeColumnsCommand( + tableIdent, + Map("col_old_name" -> StructField("col_new_name", IntegerType))) + val expected2 = AlterTableChangeColumnsCommand( + tableIdent, + Map("col_name" -> StructField("col_name", IntegerType).withComment("new_comment"))) + comparePlans(parsed1, expected1) + comparePlans(parsed2, expected2) + } + + test("alter table: change column position (not supported)") { + assertUnsupported("ALTER TABLE table_name CHANGE COLUMN col_old_name col_new_name INT FIRST") + assertUnsupported( + "ALTER TABLE table_name CHANGE COLUMN col_old_name col_new_name INT AFTER other_col") + } + + test("alter table: change column in partition spec") { + assertUnsupported("ALTER TABLE table_name PARTITION (a='1', a='2') CHANGE COLUMN a new_a INT") + } + test("alter table: touch (not supported)") { assertUnsupported("ALTER TABLE table_name TOUCH") assertUnsupported("ALTER TABLE table_name TOUCH PARTITION (dt='2008-08-08', country='us')") From 8c46704032acde5db7dae435c2a185d415e74a51 Mon Sep 17 00:00:00 2001 From: jiangxingbo Date: Fri, 25 Nov 2016 19:21:27 +0800 Subject: [PATCH 10/16] improve code quality. --- .../apache/spark/sql/types/StructField.scala | 7 -- .../spark/sql/execution/command/ddl.scala | 21 ++--- .../sql-tests/inputs/change-column.sql | 4 + .../sql-tests/results/change-column.sql.out | 76 ++++++++++++------- 4 files changed, 63 insertions(+), 45 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructField.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructField.scala index 07d77d2dad54b..2c18fdcc497fe 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructField.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructField.scala @@ -74,11 +74,4 @@ case class StructField( def getComment(): Option[String] = { if (metadata.contains("comment")) Option(metadata.getString("comment")) else None } - - /** - * Return the full description of the StructField. - */ - def getDesc(): String = { - s"StructField($name,$dataType,$nullable,$metadata)" - } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala index 5392a97741bec..9947abb19ff8e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala @@ -293,19 +293,18 @@ case class AlterTableChangeColumnsCommand( override def run(sparkSession: SparkSession): Seq[Row] = { val catalog = sparkSession.sessionState.catalog val table = catalog.getTableMetadata(tableName) - DDLUtils.verifyAlterTableType(catalog, table, false) - // Assert we only alter the column's comment, other fields e.g. name/dataType remain - // unchanged. - columns.foreach { pair => + DDLUtils.verifyAlterTableType(catalog, table, isView = false) + // Currently only support changing column comment, throw a Exception if other fields e.g. + // name/dataType are changed. + columns.foreach { case (oldColumnName, newColumn) => val originColumn = table.schema.collectFirst { - case field if field.name == pair._1 => field + case field if field.name == oldColumnName => field } - val newColumn = pair._2 - val unchanged = originColumn.map(equalIgnoreComment(_, newColumn)).getOrElse(true) + val unchanged = originColumn.forall(equalIgnoreComment(_, newColumn)) if (!unchanged) { throw new AnalysisException( - s"ALTER TABLE CHANGE COLUMN don't support change column " + - s"'${originColumn.get.getDesc}' to '${newColumn.getDesc}'") + s"ALTER TABLE CHANGE COLUMN is not supported for changing column " + + s"'${getDesc(originColumn.get)}' to '${getDesc(newColumn)}'") } } @@ -325,6 +324,10 @@ case class AlterTableChangeColumnsCommand( field.withComment("") == other.withComment("") } + // Genereate the full description of a StructField. + private def getDesc(field: StructField): String = { + s"StructField(${field.name},${field.dataType},${field.nullable},${field.metadata})" + } } /** diff --git a/sql/core/src/test/resources/sql-tests/inputs/change-column.sql b/sql/core/src/test/resources/sql-tests/inputs/change-column.sql index 7e29c94d08906..3b718edbaf1af 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/change-column.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/change-column.sql @@ -22,6 +22,10 @@ ALTER TABLE test_change CHANGE a a Int COMMENT 'this is column a'; ALTER TABLE test_change CHANGE b b String COMMENT '#*02?`', c c Int COMMENT ''; DESC test_change; +-- Don't change anything. +ALTER TABLE test_change CHANGE a a Int COMMENT 'this is column a'; +DESC test_change; + -- Change column name/dataType/position/comment together (not supported yet) ALTER TABLE test_change CHANGE a a1 String COMMENT 'this is column a1' AFTER b; DESC test_change; diff --git a/sql/core/src/test/resources/sql-tests/results/change-column.sql.out b/sql/core/src/test/resources/sql-tests/results/change-column.sql.out index 82c1beee26b3c..5cfa9a5eeb989 100644 --- a/sql/core/src/test/resources/sql-tests/results/change-column.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/change-column.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 24 +-- Number of queries: 26 -- !query 0 @@ -26,7 +26,7 @@ ALTER TABLE test_change CHANGE a a1 Int struct<> -- !query 2 output org.apache.spark.sql.AnalysisException -ALTER TABLE CHANGE COLUMN don't support change column 'StructField(a,IntegerType,true,{})' to 'StructField(a1,IntegerType,true,{})'; +ALTER TABLE CHANGE COLUMN is not supported for changing column 'StructField(a,IntegerType,true,{})' to 'StructField(a1,IntegerType,true,{})'; -- !query 3 @@ -35,7 +35,7 @@ ALTER TABLE test_change CHANGE b b1 String, c c1 Int struct<> -- !query 3 output org.apache.spark.sql.AnalysisException -ALTER TABLE CHANGE COLUMN don't support change column 'StructField(b,StringType,true,{})' to 'StructField(b1,StringType,true,{})'; +ALTER TABLE CHANGE COLUMN is not supported for changing column 'StructField(b,StringType,true,{})' to 'StructField(b1,StringType,true,{})'; -- !query 4 @@ -54,7 +54,7 @@ ALTER TABLE test_change CHANGE a a String struct<> -- !query 5 output org.apache.spark.sql.AnalysisException -ALTER TABLE CHANGE COLUMN don't support change column 'StructField(a,IntegerType,true,{})' to 'StructField(a,StringType,true,{})'; +ALTER TABLE CHANGE COLUMN is not supported for changing column 'StructField(a,IntegerType,true,{})' to 'StructField(a,StringType,true,{})'; -- !query 6 @@ -63,7 +63,7 @@ ALTER TABLE test_change CHANGE b b Int, c c Double struct<> -- !query 6 output org.apache.spark.sql.AnalysisException -ALTER TABLE CHANGE COLUMN don't support change column 'StructField(b,StringType,true,{})' to 'StructField(b,IntegerType,true,{})'; +ALTER TABLE CHANGE COLUMN is not supported for changing column 'StructField(b,StringType,true,{})' to 'StructField(b,IntegerType,true,{})'; -- !query 7 @@ -141,17 +141,11 @@ c int -- !query 14 -ALTER TABLE test_change CHANGE a a1 String COMMENT 'this is column a1' AFTER b +ALTER TABLE test_change CHANGE a a Int COMMENT 'this is column a' -- !query 14 schema struct<> -- !query 14 output -org.apache.spark.sql.catalyst.parser.ParseException - -Operation not allowed: ALTER TABLE [PARTITION] CHANGE COLUMN ... FIRST | AFTER otherCol(line 1, pos 0) -== SQL == -ALTER TABLE test_change CHANGE a a1 String COMMENT 'this is column a1' AFTER b -^^^ -- !query 15 @@ -165,24 +159,31 @@ c int -- !query 16 -CREATE TEMPORARY VIEW temp_view(a, b) AS SELECT 1, "one" +ALTER TABLE test_change CHANGE a a1 String COMMENT 'this is column a1' AFTER b -- !query 16 schema struct<> -- !query 16 output +org.apache.spark.sql.catalyst.parser.ParseException +Operation not allowed: ALTER TABLE [PARTITION] CHANGE COLUMN ... FIRST | AFTER otherCol(line 1, pos 0) + +== SQL == +ALTER TABLE test_change CHANGE a a1 String COMMENT 'this is column a1' AFTER b +^^^ -- !query 17 -ALTER TABLE temp_view CHANGE a a Int COMMENT 'this is column a' +DESC test_change -- !query 17 schema -struct<> +struct -- !query 17 output -org.apache.spark.sql.catalyst.analysis.NoSuchTableException -Table or view 'temp_view' not found in database 'default'; +a int this is column a +b string #*02?` +c int -- !query 18 -CREATE GLOBAL TEMPORARY VIEW global_temp_view(a, b) AS SELECT 1, "one" +CREATE TEMPORARY VIEW temp_view(a, b) AS SELECT 1, "one" -- !query 18 schema struct<> -- !query 18 output @@ -190,16 +191,16 @@ struct<> -- !query 19 -ALTER TABLE global_temp.global_temp_view CHANGE a a Int COMMENT 'this is column a' +ALTER TABLE temp_view CHANGE a a Int COMMENT 'this is column a' -- !query 19 schema struct<> -- !query 19 output -org.apache.spark.sql.catalyst.analysis.NoSuchDatabaseException -Database 'global_temp' not found; +org.apache.spark.sql.catalyst.analysis.NoSuchTableException +Table or view 'temp_view' not found in database 'default'; -- !query 20 -CREATE TABLE partition_table(a Int, b String) PARTITIONED BY (c Int, d String) +CREATE GLOBAL TEMPORARY VIEW global_temp_view(a, b) AS SELECT 1, "one" -- !query 20 schema struct<> -- !query 20 output @@ -207,10 +208,27 @@ struct<> -- !query 21 -ALTER TABLE partition_table PARTITION (c = 1) CHANGE COLUMN a new_a Int +ALTER TABLE global_temp.global_temp_view CHANGE a a Int COMMENT 'this is column a' -- !query 21 schema struct<> -- !query 21 output +org.apache.spark.sql.catalyst.analysis.NoSuchDatabaseException +Database 'global_temp' not found; + + +-- !query 22 +CREATE TABLE partition_table(a Int, b String) PARTITIONED BY (c Int, d String) +-- !query 22 schema +struct<> +-- !query 22 output + + + +-- !query 23 +ALTER TABLE partition_table PARTITION (c = 1) CHANGE COLUMN a new_a Int +-- !query 23 schema +struct<> +-- !query 23 output org.apache.spark.sql.catalyst.parser.ParseException Operation not allowed: ALTER TABLE PARTITION partition_spec CHANGE COLUMN, can not change the columns in partition spec(line 1, pos 0) @@ -220,17 +238,17 @@ ALTER TABLE partition_table PARTITION (c = 1) CHANGE COLUMN a new_a Int ^^^ --- !query 22 +-- !query 24 DROP TABLE test_change --- !query 22 schema +-- !query 24 schema struct<> --- !query 22 output +-- !query 24 output --- !query 23 +-- !query 25 DROP TABLE partition_table --- !query 23 schema +-- !query 25 schema struct<> --- !query 23 output +-- !query 25 output From 945aa2861bcdec1fbe1caa3074febd4633bac96d Mon Sep 17 00:00:00 2001 From: jiangxingbo Date: Mon, 12 Dec 2016 02:16:13 +0800 Subject: [PATCH 11/16] remove new APIs --- .../catalyst/catalog/ExternalCatalog.scala | 24 ------- .../catalyst/catalog/InMemoryCatalog.scala | 23 ------- .../sql/catalyst/catalog/SessionCatalog.scala | 35 ---------- .../catalog/ExternalCatalogSuite.scala | 64 +++---------------- .../catalog/SessionCatalogSuite.scala | 48 -------------- .../spark/sql/execution/command/ddl.scala | 11 ++-- .../sql/execution/command/DDLSuite.scala | 52 ++------------- .../spark/sql/hive/HiveExternalCatalog.scala | 24 +------ .../sql/hive/HiveExternalCatalogSuite.scala | 8 +-- 9 files changed, 22 insertions(+), 267 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalog.scala index d4e70e6decef9..4b8cac8f32b06 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalog.scala @@ -19,7 +19,6 @@ package org.apache.spark.sql.catalyst.catalog import org.apache.spark.sql.catalyst.analysis.{FunctionAlreadyExistsException, NoSuchDatabaseException, NoSuchFunctionException, NoSuchTableException} import org.apache.spark.sql.catalyst.expressions.Expression -import org.apache.spark.sql.types.StructField /** @@ -240,29 +239,6 @@ abstract class ExternalCatalog { table: String, predicates: Seq[Expression]): Seq[CatalogTablePartition] - // -------------------------------------------------------------------------- - // Columns - // -------------------------------------------------------------------------- - - /** - * Alter one or more column(s)'s comment. - * - * Note: If the underlying implementation does not support altering a certain field, - * this becomes a no-op. - * @param db database name - * @param table table name - * @param columnComments map from column names to comments - */ - def alterColumnComments(db: String, table: String, columnComments: Map[String, String]): Unit - - /** - * List the metadata of all columns that are defined in the specified table schema, assuming it - * exists. - * @param db database name - * @param table table name - */ - def listColumns(db: String, table: String): Seq[StructField] - // -------------------------------------------------------------------------- // Functions // -------------------------------------------------------------------------- diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala index d8b45d1b6976f..a6bebe1a3938c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala @@ -31,7 +31,6 @@ import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils.escapePathName import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.catalyst.util.StringUtils -import org.apache.spark.sql.types.{StructField, StructType} /** * An in-memory (ephemeral) implementation of the system catalog. @@ -539,28 +538,6 @@ class InMemoryCatalog( "listPartitionsByFilter is not implemented for InMemoryCatalog") } - // -------------------------------------------------------------------------- - // Columns - // -------------------------------------------------------------------------- - - override def alterColumnComments( - db: String, - table: String, - columnComments: Map[String, String]): Unit = synchronized { - val oldTable = getTable(db, table) - val newSchema = oldTable.schema.fields.map { field => - // If `columns` contains field name, update the field with new comment, - // else respect the field. - columnComments.get(field.name).map(field.withComment(_)).getOrElse(field) - } - catalog(db).tables(table).table = oldTable.copy(schema = StructType(newSchema)) - } - - override def listColumns(db: String, table: String): Seq[StructField] = synchronized { - getTable(db, table).schema.fields - } - - // -------------------------------------------------------------------------- // Functions // -------------------------------------------------------------------------- diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala index 7c45772d031bf..7a3d2097a85c5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala @@ -33,7 +33,6 @@ import org.apache.spark.sql.catalyst.analysis.FunctionRegistry.FunctionBuilder import org.apache.spark.sql.catalyst.expressions.{Expression, ExpressionInfo} import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SubqueryAlias} import org.apache.spark.sql.catalyst.util.StringUtils -import org.apache.spark.sql.types.StructField object SessionCatalog { val DEFAULT_DATABASE = "default" @@ -839,40 +838,6 @@ class SessionCatalog( } } - // ---------------------------------------------------------------------------- - // Columns - // ---------------------------------------------------------------------------- - // All methods in this category interact directly with the underlying catalog. - // These methods are concerned with only metastore tables. - // ---------------------------------------------------------------------------- - - /** - * Alter one or more columns(s)'s comment, assuming they exist. - * - * Note: If the underlying implementation does not support altering a certain field, - * this becomes a no-op. - */ - def alterColumnComments( - tableName: TableIdentifier, - columnComments: Map[String, String]): Unit = { - val db = formatDatabaseName(tableName.database.getOrElse(getCurrentDatabase)) - val table = formatTableName(tableName.table) - requireDbExists(db) - requireTableExists(TableIdentifier(table, Option(db))) - externalCatalog.alterColumnComments(db, table, columnComments) - } - - /** - * List the metadata of all columns that belong to the specified table, assuming it exists. - */ - def listColumns(tableName: TableIdentifier): Seq[StructField] = { - val db = formatDatabaseName(tableName.database.getOrElse(getCurrentDatabase)) - val table = formatTableName(tableName.table) - requireDbExists(db) - requireTableExists(TableIdentifier(table, Option(db))) - externalCatalog.listColumns(db, table) - } - // ---------------------------------------------------------------------------- // Functions // ---------------------------------------------------------------------------- diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogSuite.scala index 7b4be28ae13cb..00e663c324cb4 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogSuite.scala @@ -26,7 +26,7 @@ import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} import org.apache.spark.sql.catalyst.analysis.{FunctionAlreadyExistsException, NoSuchDatabaseException, NoSuchFunctionException} import org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException -import org.apache.spark.sql.types._ +import org.apache.spark.sql.types.StructType import org.apache.spark.util.Utils @@ -559,45 +559,6 @@ abstract class ExternalCatalogSuite extends SparkFunSuite with BeforeAndAfterEac } } - // -------------------------------------------------------------------------- - // Columns - // -------------------------------------------------------------------------- - - test("alter column comments") { - val catalog = newBasicCatalog() - val columnComments = Map(field1.name -> "insert comment", field2.name -> s"^*`%?", - field3.name -> "change comment") - catalog.alterColumnComments("db2", "tbl1", columnComments) - val newColumns = field1.withComment("insert comment") :: field2.withComment(s"^*`%?") :: - field3.withComment("change comment") :: field4 :: Nil - assert(columnsEqual(catalog.listColumns("db2", "tbl1"), newColumns)) - } - - test("alter column comments when database/table does not exist") { - val catalog = newBasicCatalog() - val columnComments = Map(field1.name -> "insert comment") - intercept[AnalysisException] { - catalog.alterColumnComments("unknown_db", "unknown_table", columnComments) - } - intercept[AnalysisException] { - catalog.alterColumnComments("db2", "unknown_table", columnComments) - } - } - - test("alter column comments when column does not exist") { - val catalog = newBasicCatalog() - val columnComments = Map("unknown_col" -> "insert comment") - catalog.alterColumnComments("db2", "tbl1", columnComments) - assert(columnsEqual(catalog.listColumns("db2", "tbl1"), schema.fields.toList)) - } - - test("list columns") { - val catalog = newBasicCatalog() - assert(columnsEqual(catalog.listColumns("db2", "tbl1"), schema.fields.toList)) - intercept[AnalysisException] { catalog.listColumns("unknown_db", "unknown_table") } - intercept[AnalysisException] { catalog.listColumns("db2", "unknown_table") } - } - // -------------------------------------------------------------------------- // Functions // -------------------------------------------------------------------------- @@ -724,7 +685,7 @@ abstract class ExternalCatalogSuite extends SparkFunSuite with BeforeAndAfterEac identifier = TableIdentifier("my_table", Some("db1")), tableType = CatalogTableType.MANAGED, storage = CatalogStorageFormat(None, None, None, None, false, Map.empty), - schema = schema, + schema = new StructType().add("a", "int").add("b", "string"), provider = Some("hive") ) @@ -744,7 +705,7 @@ abstract class ExternalCatalogSuite extends SparkFunSuite with BeforeAndAfterEac storage = CatalogStorageFormat( Some(Utils.createTempDir().getAbsolutePath), None, None, None, false, Map.empty), - schema = schema, + schema = new StructType().add("a", "int").add("b", "string"), provider = Some("hive") ) catalog.createTable(externalTable, ignoreIfExists = false) @@ -847,11 +808,6 @@ abstract class CatalogTestUtils { CatalogTablePartition(Map("a" -> "5", "b" -> "6", "c" -> "7"), storageFormat) lazy val partWithUnknownColumns = CatalogTablePartition(Map("a" -> "5", "unknown" -> "6"), storageFormat) - lazy val field1 = StructField("col1", IntegerType) - lazy val field2 = StructField("col2", StringType, nullable = false).withComment("some blabla...") - lazy val field3 = StructField("a", IntegerType, nullable = false).withComment(s"^*`%?") - lazy val field4 = StructField("b", StringType) - lazy val schema = StructType(field1 :: field2 :: field3 :: field4 :: Nil) lazy val funcClass = "org.apache.spark.myFunc" /** @@ -894,7 +850,11 @@ abstract class CatalogTestUtils { identifier = TableIdentifier(name, database), tableType = CatalogTableType.EXTERNAL, storage = storageFormat.copy(locationUri = Some(Utils.createTempDir().getAbsolutePath)), - schema = schema, + schema = new StructType() + .add("col1", "int") + .add("col2", "string") + .add("a", "int") + .add("b", "string"), provider = Some("hive"), partitionColumnNames = Seq("a", "b"), bucketSpec = Some(BucketSpec(4, Seq("col1"), Nil))) @@ -916,12 +876,4 @@ abstract class CatalogTestUtils { catalog.listPartitions(db, table).map(_.spec).toSet == parts.map(_.spec).toSet } - /** - * Whether the columns equal the ones given. - * Node: HIVE column don't have `nullable` field, so we should rewrite this function for - * HiveExternalCatalogSuite. - */ - def columnsEqual(columns: Seq[StructField], others: Seq[StructField]): Boolean = { - columns == others - } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala index b51f107a10ad7..5cc772d8e9a1e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala @@ -945,54 +945,6 @@ class SessionCatalogSuite extends SparkFunSuite { expectedParts.map(p => p.copy(storage = p.storage.copy(locationUri = None))).toSet } - // -------------------------------------------------------------------------- - // Columns - // -------------------------------------------------------------------------- - - test("alter column comments") { - val externalCatalog = newBasicCatalog() - val sessionCatalog = new SessionCatalog(externalCatalog) - assert(columnsEqual(externalCatalog.listColumns("db2", "tbl1"), schema.fields.toList)) - val columnComments = Map(field1.name -> "insert comment", field2.name -> s"^*`%?", - field3.name -> "change comment") - sessionCatalog.alterColumnComments(TableIdentifier("tbl1", Some("db2")), columnComments) - val newColumns = field1.withComment("insert comment") :: field2.withComment(s"^*`%?") :: - field3.withComment("change comment") :: field4 :: Nil - assert(columnsEqual(externalCatalog.listColumns("db2", "tbl1"), newColumns)) - } - - test("alter column comments when database/table does not exist") { - val catalog = new SessionCatalog(newBasicCatalog()) - val columnComments = Map(field1.name -> "insert comment") - intercept[NoSuchDatabaseException] { - catalog.alterColumnComments(TableIdentifier("tbl1", Some("unknown_db")), columnComments) - } - intercept[NoSuchTableException] { - catalog.alterColumnComments(TableIdentifier("unknown_table", Some("db2")), columnComments) - } - } - - test("alter column comments when column does not exist") { - val externalCatalog = newBasicCatalog() - val sessionCatalog = new SessionCatalog(externalCatalog) - val columnComments = Map("unknown_col" -> "insert comment") - sessionCatalog.alterColumnComments(TableIdentifier("tbl1", Some("db2")), columnComments) - // Columns are not changed. - assert(columnsEqual(externalCatalog.listColumns("db2", "tbl1"), schema.fields.toList)) - } - - test("list columns") { - val catalog = new SessionCatalog(newBasicCatalog()) - assert(columnsEqual( - catalog.listColumns(TableIdentifier("tbl1", Some("db2"))), schema.fields.toList)) - intercept[NoSuchDatabaseException] { - catalog.listColumns(TableIdentifier("tbl1", Some("unknown_db"))) - } - intercept[NoSuchTableException] { - catalog.listColumns(TableIdentifier("unknown_table", Some("db2"))) - } - } - // -------------------------------------------------------------------------- // Functions // -------------------------------------------------------------------------- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala index 9947abb19ff8e..427abb46fb952 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala @@ -308,12 +308,13 @@ case class AlterTableChangeColumnsCommand( } } - // Create map from columns to new comments. - val columnComments = columns.map { - case (col, field: StructField) if field.getComment().isDefined => - (col, field.getComment.get) + val newSchema = table.schema.fields.map { field => + // If `columns` contains field name, update the field to the new column, else respect the + // field. + columns.get(field.name).getOrElse(field) } - catalog.alterColumnComments(tableName, columnComments) + val newTable = table.copy(schema = StructType(newSchema)) + catalog.alterTable(newTable) Seq.empty[Row] } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala index cc70acbcc0ead..e61beb49e47f4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala @@ -38,12 +38,6 @@ import org.apache.spark.util.Utils class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { private val escapedIdentifier = "`(.+)`".r - lazy val field1 = StructField("col1", IntegerType) - lazy val field2 = StructField("col2", StringType) - lazy val field3 = StructField("a", IntegerType) - lazy val field4 = StructField("b", StringType) - lazy val schema = StructType(field1 :: field2 :: field3 :: field4 :: Nil) - override def afterEach(): Unit = { try { // drop all databases, tables and functions after each test @@ -94,7 +88,11 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { identifier = name, tableType = CatalogTableType.EXTERNAL, storage = storage, - schema = schema, + schema = new StructType() + .add("col1", "int") + .add("col2", "string") + .add("a", "int") + .add("b", "int"), provider = Some("hive"), partitionColumnNames = Seq("a", "b"), createTime = 0L, @@ -817,14 +815,6 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { } } - test("alter table: change column comments") { - testChangeColumnComments(isDatasourceTable = false) - } - - test("alter table: change column comments (datasource table)") { - testChangeColumnComments(isDatasourceTable = true) - } - private def testRecoverPartitions() { val catalog = spark.sessionState.catalog // table to alter does not exist @@ -1378,38 +1368,6 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { Set(Map("a" -> "1", "b" -> "p"), Map("a" -> "20", "b" -> "c"), Map("a" -> "3", "b" -> "p"))) } - private def testChangeColumnComments(isDatasourceTable: Boolean): Unit = { - val catalog = spark.sessionState.catalog - val tableIdent = TableIdentifier("tab1", Some("dbx")) - createDatabase(catalog, "dbx") - createTable(catalog, tableIdent) - if (isDatasourceTable) { - convertToDatasourceTable(catalog, tableIdent) - } - - // alter column comments - val columnComments = Map(field1.name -> "insert comment", field2.name -> s"^*`%?", - field3.name -> "change comment") - catalog.alterColumnComments(tableIdent, columnComments) - val newColumns = field1.withComment("insert comment") :: field2.withComment(s"^*`%?") :: - field3.withComment("change comment") :: field4 :: Nil - assert(catalog.listColumns(tableIdent) == newColumns) - - // database/table does not exist - intercept[AnalysisException] { - catalog.alterColumnComments(TableIdentifier("unknown_table", Some("unknown_db")), - columnComments) - } - intercept[AnalysisException] { - catalog.alterColumnComments(TableIdentifier("unknown_table", Some("dbx")), columnComments) - } - - // column does not exist - val columnComments2 = Map("unknown_col" -> "insert comment") - catalog.alterColumnComments(tableIdent, columnComments2) - assert(catalog.listColumns(tableIdent) == newColumns) - } - test("drop build-in function") { Seq("true", "false").foreach { caseSensitive => withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive) { diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala index c7e1dd3849922..f67ddc9be1a5a 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala @@ -44,7 +44,7 @@ import org.apache.spark.sql.execution.datasources.PartitioningUtils import org.apache.spark.sql.hive.client.HiveClient import org.apache.spark.sql.internal.HiveSerDe import org.apache.spark.sql.internal.StaticSQLConf._ -import org.apache.spark.sql.types._ +import org.apache.spark.sql.types.{DataType, StructType} /** @@ -1007,27 +1007,6 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat } } - // -------------------------------------------------------------------------- - // Columns - // -------------------------------------------------------------------------- - - override def alterColumnComments( - db: String, - table: String, - columnComments: Map[String, String]): Unit = withClient { - val oldTable = client.getTable(db, table) - val newSchema = oldTable.schema.fields.map { field => - // If `columns` contains field name, update the field with new comment, - // else respect the field. - columnComments.get(field.name).map(field.withComment(_)).getOrElse(field) - } - client.alterTable(oldTable.copy(schema = StructType(newSchema))) - } - - override def listColumns(db: String, table: String): Seq[StructField] = withClient { - client.getTable(db, table).schema.fields - } - // -------------------------------------------------------------------------- // Functions // -------------------------------------------------------------------------- @@ -1070,6 +1049,7 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat requireDbExists(db) client.listFunctions(db, pattern) } + } object HiveExternalCatalog { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogSuite.scala index 12ad74f167735..6fee45824ea3f 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogSuite.scala @@ -24,7 +24,7 @@ import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.execution.command.DDLUtils -import org.apache.spark.sql.types.{StructField, StructType} +import org.apache.spark.sql.types.StructType /** * Test suite for the [[HiveExternalCatalog]]. @@ -41,12 +41,6 @@ class HiveExternalCatalogSuite extends ExternalCatalogSuite { override val tableInputFormat: String = "org.apache.hadoop.mapred.SequenceFileInputFormat" override val tableOutputFormat: String = "org.apache.hadoop.mapred.SequenceFileOutputFormat" override def newEmptyCatalog(): ExternalCatalog = externalCatalog - // Compare columns ignore nullability. - override def columnsEqual(columns: Seq[StructField], others: Seq[StructField]): Boolean = { - val normalColumns = columns.map(_.copy(nullable = true)) - val normalOthers = others.map(_.copy(nullable = true)) - normalColumns == normalOthers - } } protected override def resetState(): Unit = { From f49b0a0b3cf3ed422597a27123dac9a8ca943651 Mon Sep 17 00:00:00 2001 From: jiangxingbo Date: Mon, 12 Dec 2016 19:04:23 +0800 Subject: [PATCH 12/16] bugfix --- .../spark/sql/execution/SparkSqlParser.scala | 4 +- .../spark/sql/execution/command/ddl.scala | 57 ++++++++---- .../sql-tests/inputs/change-column.sql | 8 ++ .../sql-tests/results/change-column.sql.out | 91 +++++++++++++------ 4 files changed, 111 insertions(+), 49 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index e76e54e79d40d..7e5c5c7e6ee84 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -888,7 +888,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { */ override def visitChangeColumns(ctx: ChangeColumnsContext): LogicalPlan = withOrigin(ctx) { if (ctx.partitionSpec != null) { - operationNotAllowed(s"ALTER TABLE PARTITION partition_spec CHANGE COLUMN, can not change " + + operationNotAllowed("ALTER TABLE PARTITION partition_spec CHANGE COLUMN, can not change " + "the columns in partition spec", ctx) } @@ -897,7 +897,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { val columns = ctx.expandColTypeList.expandColType.asScala.map { col => if (col.colPosition != null) { operationNotAllowed( - s"ALTER TABLE [PARTITION] CHANGE COLUMN ... FIRST | AFTER otherCol", ctx) + "ALTER TABLE [PARTITION] CHANGE COLUMN ... FIRST | AFTER otherCol", ctx) } col.identifier.getText -> visitColType(col.colType) }.toMap diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala index 427abb46fb952..0f2693c0ce5af 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala @@ -276,7 +276,8 @@ case class AlterTableUnsetPropertiesCommand( /** - * A command to change the columns for a table, only support changing column comment for now. + * A command to change the columns for a table, only support changing the comments of non-partition + * columns for now. * * The syntax of using this command in SQL is: * {{{ @@ -293,36 +294,52 @@ case class AlterTableChangeColumnsCommand( override def run(sparkSession: SparkSession): Seq[Row] = { val catalog = sparkSession.sessionState.catalog val table = catalog.getTableMetadata(tableName) + val resolver = sparkSession.sessionState.conf.resolver DDLUtils.verifyAlterTableType(catalog, table, isView = false) - // Currently only support changing column comment, throw a Exception if other fields e.g. - // name/dataType are changed. - columns.foreach { case (oldColumnName, newColumn) => - val originColumn = table.schema.collectFirst { - case field if field.name == oldColumnName => field - } - val unchanged = originColumn.forall(equalIgnoreComment(_, newColumn)) - if (!unchanged) { + + // Create a map that converts the origin column to the new column with changed comment, throw + // a Exception if the column reference is invalid or the column name/dataType is changed. + val columnsMap = columns.map { case (oldName: String, newField: StructField) => + // Find the origin column from schema by column name. + val originColumn = findColumn(table.schema, oldName, resolver) + // Throw a Exception if the column name/dataType is changed. + if (!columnEqual(originColumn, newField, resolver)) { throw new AnalysisException( - s"ALTER TABLE CHANGE COLUMN is not supported for changing column " + - s"'${getDesc(originColumn.get)}' to '${getDesc(newColumn)}'") + "ALTER TABLE CHANGE COLUMN is not supported for changing column " + + s"'${getDesc(originColumn)}' to '${getDesc(newField)}'") } + // Create a new column from the origin column with new comment. + val newColumn = addComment(originColumn, newField.getComment) + // Create the map from origin column to changed column + originColumn -> newColumn } - val newSchema = table.schema.fields.map { field => - // If `columns` contains field name, update the field to the new column, else respect the - // field. - columns.get(field.name).getOrElse(field) - } + val newSchema = table.schema.fields.map(field => columnsMap.getOrElse(field, field)) val newTable = table.copy(schema = StructType(newSchema)) catalog.alterTable(newTable) Seq.empty[Row] } - // Compare a [[StructField]] to another, return true if they have equal values except for - // comment. - private def equalIgnoreComment(field: StructField, other: StructField): Boolean = { - field.withComment("") == other.withComment("") + // Find the origin column from schema by column name, throw a Exception if the column + // reference is invalid. + private def findColumn(schema: StructType, name: String, resolver: Resolver): StructField = { + schema.fields.collectFirst { + case field if resolver(field.name, name) => field + }.getOrElse(throw new AnalysisException( + s"Invalid column reference '$name', table schema is '${schema}'")) + } + + // Add the comment to a column, if comment is empty, return the original column. + private def addComment(column: StructField, comment: Option[String]): StructField = { + comment.map(column.withComment(_)).getOrElse(column) + } + + // Compare a [[StructField]] to another, return true if they have the same column + // name(by resolver) and dataType. + private def columnEqual( + field: StructField, other: StructField, resolver: Resolver): Boolean = { + resolver(field.name, other.name) && field.dataType == other.dataType } // Genereate the full description of a StructField. diff --git a/sql/core/src/test/resources/sql-tests/inputs/change-column.sql b/sql/core/src/test/resources/sql-tests/inputs/change-column.sql index 3b718edbaf1af..488dc43c4c482 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/change-column.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/change-column.sql @@ -26,10 +26,18 @@ DESC test_change; ALTER TABLE test_change CHANGE a a Int COMMENT 'this is column a'; DESC test_change; +-- Change a invalid column +ALTER TABLE test_change CHANGE invalid_col invalid_col Int; +DESC test_change; + -- Change column name/dataType/position/comment together (not supported yet) ALTER TABLE test_change CHANGE a a1 String COMMENT 'this is column a1' AFTER b; DESC test_change; +-- Case sensitive +ALTER TABLE test_change CHANGE a A Int COMMENT 'this is column A'; +DESC test_change; + -- Change column can't apply to a temporary/global_temporary view CREATE TEMPORARY VIEW temp_view(a, b) AS SELECT 1, "one"; ALTER TABLE temp_view CHANGE a a Int COMMENT 'this is column a'; diff --git a/sql/core/src/test/resources/sql-tests/results/change-column.sql.out b/sql/core/src/test/resources/sql-tests/results/change-column.sql.out index 5cfa9a5eeb989..8ab695c579674 100644 --- a/sql/core/src/test/resources/sql-tests/results/change-column.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/change-column.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 26 +-- Number of queries: 30 -- !query 0 @@ -159,17 +159,12 @@ c int -- !query 16 -ALTER TABLE test_change CHANGE a a1 String COMMENT 'this is column a1' AFTER b +ALTER TABLE test_change CHANGE invalid_col invalid_col Int -- !query 16 schema struct<> -- !query 16 output -org.apache.spark.sql.catalyst.parser.ParseException - -Operation not allowed: ALTER TABLE [PARTITION] CHANGE COLUMN ... FIRST | AFTER otherCol(line 1, pos 0) - -== SQL == -ALTER TABLE test_change CHANGE a a1 String COMMENT 'this is column a1' AFTER b -^^^ +org.apache.spark.sql.AnalysisException +Invalid column reference 'invalid_col', table schema is 'StructType(StructField(a,IntegerType,true), StructField(b,StringType,true), StructField(c,IntegerType,true))'; -- !query 17 @@ -183,24 +178,31 @@ c int -- !query 18 -CREATE TEMPORARY VIEW temp_view(a, b) AS SELECT 1, "one" +ALTER TABLE test_change CHANGE a a1 String COMMENT 'this is column a1' AFTER b -- !query 18 schema struct<> -- !query 18 output +org.apache.spark.sql.catalyst.parser.ParseException +Operation not allowed: ALTER TABLE [PARTITION] CHANGE COLUMN ... FIRST | AFTER otherCol(line 1, pos 0) + +== SQL == +ALTER TABLE test_change CHANGE a a1 String COMMENT 'this is column a1' AFTER b +^^^ -- !query 19 -ALTER TABLE temp_view CHANGE a a Int COMMENT 'this is column a' +DESC test_change -- !query 19 schema -struct<> +struct -- !query 19 output -org.apache.spark.sql.catalyst.analysis.NoSuchTableException -Table or view 'temp_view' not found in database 'default'; +a int this is column a +b string #*02?` +c int -- !query 20 -CREATE GLOBAL TEMPORARY VIEW global_temp_view(a, b) AS SELECT 1, "one" +ALTER TABLE test_change CHANGE a A Int COMMENT 'this is column A' -- !query 20 schema struct<> -- !query 20 output @@ -208,16 +210,17 @@ struct<> -- !query 21 -ALTER TABLE global_temp.global_temp_view CHANGE a a Int COMMENT 'this is column a' +DESC test_change -- !query 21 schema -struct<> +struct -- !query 21 output -org.apache.spark.sql.catalyst.analysis.NoSuchDatabaseException -Database 'global_temp' not found; +a int this is column A +b string #*02?` +c int -- !query 22 -CREATE TABLE partition_table(a Int, b String) PARTITIONED BY (c Int, d String) +CREATE TEMPORARY VIEW temp_view(a, b) AS SELECT 1, "one" -- !query 22 schema struct<> -- !query 22 output @@ -225,10 +228,44 @@ struct<> -- !query 23 -ALTER TABLE partition_table PARTITION (c = 1) CHANGE COLUMN a new_a Int +ALTER TABLE temp_view CHANGE a a Int COMMENT 'this is column a' -- !query 23 schema struct<> -- !query 23 output +org.apache.spark.sql.catalyst.analysis.NoSuchTableException +Table or view 'temp_view' not found in database 'default'; + + +-- !query 24 +CREATE GLOBAL TEMPORARY VIEW global_temp_view(a, b) AS SELECT 1, "one" +-- !query 24 schema +struct<> +-- !query 24 output + + + +-- !query 25 +ALTER TABLE global_temp.global_temp_view CHANGE a a Int COMMENT 'this is column a' +-- !query 25 schema +struct<> +-- !query 25 output +org.apache.spark.sql.catalyst.analysis.NoSuchDatabaseException +Database 'global_temp' not found; + + +-- !query 26 +CREATE TABLE partition_table(a Int, b String) PARTITIONED BY (c Int, d String) +-- !query 26 schema +struct<> +-- !query 26 output + + + +-- !query 27 +ALTER TABLE partition_table PARTITION (c = 1) CHANGE COLUMN a new_a Int +-- !query 27 schema +struct<> +-- !query 27 output org.apache.spark.sql.catalyst.parser.ParseException Operation not allowed: ALTER TABLE PARTITION partition_spec CHANGE COLUMN, can not change the columns in partition spec(line 1, pos 0) @@ -238,17 +275,17 @@ ALTER TABLE partition_table PARTITION (c = 1) CHANGE COLUMN a new_a Int ^^^ --- !query 24 +-- !query 28 DROP TABLE test_change --- !query 24 schema +-- !query 28 schema struct<> --- !query 24 output +-- !query 28 output --- !query 25 +-- !query 29 DROP TABLE partition_table --- !query 25 schema +-- !query 29 schema struct<> --- !query 25 output +-- !query 29 output From b9e88945c06f874c30fc000875fe4051e7845c68 Mon Sep 17 00:00:00 2001 From: jiangxingbo Date: Tue, 13 Dec 2016 13:34:46 +0800 Subject: [PATCH 13/16] change the Exception messages. --- .../org/apache/spark/sql/execution/SparkSqlParser.scala | 6 +++--- .../org/apache/spark/sql/execution/command/ddl.scala | 8 ++------ 2 files changed, 5 insertions(+), 9 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index 7e5c5c7e6ee84..2f65bed1db2fc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -888,8 +888,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { */ override def visitChangeColumns(ctx: ChangeColumnsContext): LogicalPlan = withOrigin(ctx) { if (ctx.partitionSpec != null) { - operationNotAllowed("ALTER TABLE PARTITION partition_spec CHANGE COLUMN, can not change " + - "the columns in partition spec", ctx) + operationNotAllowed("ALTER TABLE table PARTITION partition_spec CHANGE COLUMN", ctx) } val tableName = visitTableIdentifier(ctx.tableIdentifier) @@ -897,7 +896,8 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { val columns = ctx.expandColTypeList.expandColType.asScala.map { col => if (col.colPosition != null) { operationNotAllowed( - "ALTER TABLE [PARTITION] CHANGE COLUMN ... FIRST | AFTER otherCol", ctx) + "ALTER TABLE table [PARTITION partition_spec] CHANGE COLUMN ... FIRST | AFTER otherCol", + ctx) } col.identifier.getText -> visitColType(col.colType) }.toMap diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala index 0f2693c0ce5af..25f178c2fe097 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala @@ -306,7 +306,8 @@ case class AlterTableChangeColumnsCommand( if (!columnEqual(originColumn, newField, resolver)) { throw new AnalysisException( "ALTER TABLE CHANGE COLUMN is not supported for changing column " + - s"'${getDesc(originColumn)}' to '${getDesc(newField)}'") + s"'${originColumn.name}' with type '${originColumn.dataType}' to " + + s"'${newField.name}' with type '${newField.dataType}'") } // Create a new column from the origin column with new comment. val newColumn = addComment(originColumn, newField.getComment) @@ -341,11 +342,6 @@ case class AlterTableChangeColumnsCommand( field: StructField, other: StructField, resolver: Resolver): Boolean = { resolver(field.name, other.name) && field.dataType == other.dataType } - - // Genereate the full description of a StructField. - private def getDesc(field: StructField): String = { - s"StructField(${field.name},${field.dataType},${field.nullable},${field.metadata})" - } } /** From a71683c773a1117f58c7e2b5cb5a4b1b101326f4 Mon Sep 17 00:00:00 2001 From: jiangxingbo Date: Tue, 13 Dec 2016 14:25:31 +0800 Subject: [PATCH 14/16] refactor. --- .../spark/sql/execution/command/ddl.scala | 4 +- .../sql-tests/inputs/change-column.sql | 34 +++++------ .../sql-tests/results/change-column.sql.out | 58 +++++++++---------- 3 files changed, 48 insertions(+), 48 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala index 25f178c2fe097..8164b9a96883c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala @@ -312,10 +312,10 @@ case class AlterTableChangeColumnsCommand( // Create a new column from the origin column with new comment. val newColumn = addComment(originColumn, newField.getComment) // Create the map from origin column to changed column - originColumn -> newColumn + originColumn.name -> newColumn } - val newSchema = table.schema.fields.map(field => columnsMap.getOrElse(field, field)) + val newSchema = table.schema.fields.map(field => columnsMap.getOrElse(field.name, field)) val newTable = table.copy(schema = StructType(newSchema)) catalog.alterTable(newTable) diff --git a/sql/core/src/test/resources/sql-tests/inputs/change-column.sql b/sql/core/src/test/resources/sql-tests/inputs/change-column.sql index 488dc43c4c482..6a5404ee8ef6e 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/change-column.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/change-column.sql @@ -1,52 +1,52 @@ -- Create the origin table -CREATE TABLE test_change(a Int, b String, c Int); +CREATE TABLE test_change(a INT, b STRING, c INT); DESC test_change; -- Change column name (not supported yet) -ALTER TABLE test_change CHANGE a a1 Int; -ALTER TABLE test_change CHANGE b b1 String, c c1 Int; +ALTER TABLE test_change CHANGE a a1 INT; +ALTER TABLE test_change CHANGE b b1 STRING, c c1 INT; DESC test_change; -- Change column dataType (not supported yet) -ALTER TABLE test_change CHANGE a a String; -ALTER TABLE test_change CHANGE b b Int, c c Double; +ALTER TABLE test_change CHANGE a a STRING; +ALTER TABLE test_change CHANGE b b INT, c c DOUBLE; DESC test_change; -- Change column position (not supported yet) -ALTER TABLE test_change CHANGE a a Int AFTER b; -ALTER TABLE test_change CHANGE b b String FIRST, c c Int AFTER b; +ALTER TABLE test_change CHANGE a a INT AFTER b; +ALTER TABLE test_change CHANGE b b STRING FIRST, c c INT AFTER b; DESC test_change; -- Change column comment -ALTER TABLE test_change CHANGE a a Int COMMENT 'this is column a'; -ALTER TABLE test_change CHANGE b b String COMMENT '#*02?`', c c Int COMMENT ''; +ALTER TABLE test_change CHANGE a a INT COMMENT 'this is column a'; +ALTER TABLE test_change CHANGE b b STRING COMMENT '#*02?`', c c INT COMMENT ''; DESC test_change; -- Don't change anything. -ALTER TABLE test_change CHANGE a a Int COMMENT 'this is column a'; +ALTER TABLE test_change CHANGE a a INT COMMENT 'this is column a'; DESC test_change; -- Change a invalid column -ALTER TABLE test_change CHANGE invalid_col invalid_col Int; +ALTER TABLE test_change CHANGE invalid_col invalid_col INT; DESC test_change; -- Change column name/dataType/position/comment together (not supported yet) -ALTER TABLE test_change CHANGE a a1 String COMMENT 'this is column a1' AFTER b; +ALTER TABLE test_change CHANGE a a1 STRING COMMENT 'this is column a1' AFTER b; DESC test_change; -- Case sensitive -ALTER TABLE test_change CHANGE a A Int COMMENT 'this is column A'; +ALTER TABLE test_change CHANGE a A INT COMMENT 'this is column A'; DESC test_change; -- Change column can't apply to a temporary/global_temporary view CREATE TEMPORARY VIEW temp_view(a, b) AS SELECT 1, "one"; -ALTER TABLE temp_view CHANGE a a Int COMMENT 'this is column a'; +ALTER TABLE temp_view CHANGE a a INT COMMENT 'this is column a'; CREATE GLOBAL TEMPORARY VIEW global_temp_view(a, b) AS SELECT 1, "one"; -ALTER TABLE global_temp.global_temp_view CHANGE a a Int COMMENT 'this is column a'; +ALTER TABLE global_temp.global_temp_view CHANGE a a INT COMMENT 'this is column a'; -- Change column in partition spec (not supported yet) -CREATE TABLE partition_table(a Int, b String) PARTITIONED BY (c Int, d String); -ALTER TABLE partition_table PARTITION (c = 1) CHANGE COLUMN a new_a Int; +CREATE TABLE partition_table(a INT, b STRING) PARTITIONED BY (c INT, d STRING); +ALTER TABLE partition_table PARTITION (c = 1) CHANGE COLUMN a new_a INT; -- DROP TEST TABLE DROP TABLE test_change; diff --git a/sql/core/src/test/resources/sql-tests/results/change-column.sql.out b/sql/core/src/test/resources/sql-tests/results/change-column.sql.out index 8ab695c579674..d4c04ce83b5c5 100644 --- a/sql/core/src/test/resources/sql-tests/results/change-column.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/change-column.sql.out @@ -3,7 +3,7 @@ -- !query 0 -CREATE TABLE test_change(a Int, b String, c Int) +CREATE TABLE test_change(a INT, b STRING, c INT) -- !query 0 schema struct<> -- !query 0 output @@ -21,21 +21,21 @@ c int -- !query 2 -ALTER TABLE test_change CHANGE a a1 Int +ALTER TABLE test_change CHANGE a a1 INT -- !query 2 schema struct<> -- !query 2 output org.apache.spark.sql.AnalysisException -ALTER TABLE CHANGE COLUMN is not supported for changing column 'StructField(a,IntegerType,true,{})' to 'StructField(a1,IntegerType,true,{})'; +ALTER TABLE CHANGE COLUMN is not supported for changing column 'a' with type 'IntegerType' to 'a1' with type 'IntegerType'; -- !query 3 -ALTER TABLE test_change CHANGE b b1 String, c c1 Int +ALTER TABLE test_change CHANGE b b1 STRING, c c1 INT -- !query 3 schema struct<> -- !query 3 output org.apache.spark.sql.AnalysisException -ALTER TABLE CHANGE COLUMN is not supported for changing column 'StructField(b,StringType,true,{})' to 'StructField(b1,StringType,true,{})'; +ALTER TABLE CHANGE COLUMN is not supported for changing column 'b' with type 'StringType' to 'b1' with type 'StringType'; -- !query 4 @@ -49,21 +49,21 @@ c int -- !query 5 -ALTER TABLE test_change CHANGE a a String +ALTER TABLE test_change CHANGE a a STRING -- !query 5 schema struct<> -- !query 5 output org.apache.spark.sql.AnalysisException -ALTER TABLE CHANGE COLUMN is not supported for changing column 'StructField(a,IntegerType,true,{})' to 'StructField(a,StringType,true,{})'; +ALTER TABLE CHANGE COLUMN is not supported for changing column 'a' with type 'IntegerType' to 'a' with type 'StringType'; -- !query 6 -ALTER TABLE test_change CHANGE b b Int, c c Double +ALTER TABLE test_change CHANGE b b INT, c c DOUBLE -- !query 6 schema struct<> -- !query 6 output org.apache.spark.sql.AnalysisException -ALTER TABLE CHANGE COLUMN is not supported for changing column 'StructField(b,StringType,true,{})' to 'StructField(b,IntegerType,true,{})'; +ALTER TABLE CHANGE COLUMN is not supported for changing column 'b' with type 'StringType' to 'b' with type 'IntegerType'; -- !query 7 @@ -77,30 +77,30 @@ c int -- !query 8 -ALTER TABLE test_change CHANGE a a Int AFTER b +ALTER TABLE test_change CHANGE a a INT AFTER b -- !query 8 schema struct<> -- !query 8 output org.apache.spark.sql.catalyst.parser.ParseException -Operation not allowed: ALTER TABLE [PARTITION] CHANGE COLUMN ... FIRST | AFTER otherCol(line 1, pos 0) +Operation not allowed: ALTER TABLE table [PARTITION partition_spec] CHANGE COLUMN ... FIRST | AFTER otherCol(line 1, pos 0) == SQL == -ALTER TABLE test_change CHANGE a a Int AFTER b +ALTER TABLE test_change CHANGE a a INT AFTER b ^^^ -- !query 9 -ALTER TABLE test_change CHANGE b b String FIRST, c c Int AFTER b +ALTER TABLE test_change CHANGE b b STRING FIRST, c c INT AFTER b -- !query 9 schema struct<> -- !query 9 output org.apache.spark.sql.catalyst.parser.ParseException -Operation not allowed: ALTER TABLE [PARTITION] CHANGE COLUMN ... FIRST | AFTER otherCol(line 1, pos 0) +Operation not allowed: ALTER TABLE table [PARTITION partition_spec] CHANGE COLUMN ... FIRST | AFTER otherCol(line 1, pos 0) == SQL == -ALTER TABLE test_change CHANGE b b String FIRST, c c Int AFTER b +ALTER TABLE test_change CHANGE b b STRING FIRST, c c INT AFTER b ^^^ @@ -115,7 +115,7 @@ c int -- !query 11 -ALTER TABLE test_change CHANGE a a Int COMMENT 'this is column a' +ALTER TABLE test_change CHANGE a a INT COMMENT 'this is column a' -- !query 11 schema struct<> -- !query 11 output @@ -123,7 +123,7 @@ struct<> -- !query 12 -ALTER TABLE test_change CHANGE b b String COMMENT '#*02?`', c c Int COMMENT '' +ALTER TABLE test_change CHANGE b b STRING COMMENT '#*02?`', c c INT COMMENT '' -- !query 12 schema struct<> -- !query 12 output @@ -141,7 +141,7 @@ c int -- !query 14 -ALTER TABLE test_change CHANGE a a Int COMMENT 'this is column a' +ALTER TABLE test_change CHANGE a a INT COMMENT 'this is column a' -- !query 14 schema struct<> -- !query 14 output @@ -159,7 +159,7 @@ c int -- !query 16 -ALTER TABLE test_change CHANGE invalid_col invalid_col Int +ALTER TABLE test_change CHANGE invalid_col invalid_col INT -- !query 16 schema struct<> -- !query 16 output @@ -178,16 +178,16 @@ c int -- !query 18 -ALTER TABLE test_change CHANGE a a1 String COMMENT 'this is column a1' AFTER b +ALTER TABLE test_change CHANGE a a1 STRING COMMENT 'this is column a1' AFTER b -- !query 18 schema struct<> -- !query 18 output org.apache.spark.sql.catalyst.parser.ParseException -Operation not allowed: ALTER TABLE [PARTITION] CHANGE COLUMN ... FIRST | AFTER otherCol(line 1, pos 0) +Operation not allowed: ALTER TABLE table [PARTITION partition_spec] CHANGE COLUMN ... FIRST | AFTER otherCol(line 1, pos 0) == SQL == -ALTER TABLE test_change CHANGE a a1 String COMMENT 'this is column a1' AFTER b +ALTER TABLE test_change CHANGE a a1 STRING COMMENT 'this is column a1' AFTER b ^^^ @@ -202,7 +202,7 @@ c int -- !query 20 -ALTER TABLE test_change CHANGE a A Int COMMENT 'this is column A' +ALTER TABLE test_change CHANGE a A INT COMMENT 'this is column A' -- !query 20 schema struct<> -- !query 20 output @@ -228,7 +228,7 @@ struct<> -- !query 23 -ALTER TABLE temp_view CHANGE a a Int COMMENT 'this is column a' +ALTER TABLE temp_view CHANGE a a INT COMMENT 'this is column a' -- !query 23 schema struct<> -- !query 23 output @@ -245,7 +245,7 @@ struct<> -- !query 25 -ALTER TABLE global_temp.global_temp_view CHANGE a a Int COMMENT 'this is column a' +ALTER TABLE global_temp.global_temp_view CHANGE a a INT COMMENT 'this is column a' -- !query 25 schema struct<> -- !query 25 output @@ -254,7 +254,7 @@ Database 'global_temp' not found; -- !query 26 -CREATE TABLE partition_table(a Int, b String) PARTITIONED BY (c Int, d String) +CREATE TABLE partition_table(a INT, b STRING) PARTITIONED BY (c INT, d STRING) -- !query 26 schema struct<> -- !query 26 output @@ -262,16 +262,16 @@ struct<> -- !query 27 -ALTER TABLE partition_table PARTITION (c = 1) CHANGE COLUMN a new_a Int +ALTER TABLE partition_table PARTITION (c = 1) CHANGE COLUMN a new_a INT -- !query 27 schema struct<> -- !query 27 output org.apache.spark.sql.catalyst.parser.ParseException -Operation not allowed: ALTER TABLE PARTITION partition_spec CHANGE COLUMN, can not change the columns in partition spec(line 1, pos 0) +Operation not allowed: ALTER TABLE table PARTITION partition_spec CHANGE COLUMN(line 1, pos 0) == SQL == -ALTER TABLE partition_table PARTITION (c = 1) CHANGE COLUMN a new_a Int +ALTER TABLE partition_table PARTITION (c = 1) CHANGE COLUMN a new_a INT ^^^ From 57934d4bba2887a2cacff45e6303e96fa9759e1d Mon Sep 17 00:00:00 2001 From: jiangxingbo Date: Wed, 14 Dec 2016 01:13:19 +0800 Subject: [PATCH 15/16] add more test cases. --- .../spark/sql/execution/command/ddl.scala | 14 +-- .../sql-tests/inputs/change-column.sql | 5 +- .../sql-tests/results/change-column.sql.out | 85 ++++++++++++------- .../sql/execution/command/DDLSuite.scala | 33 ++++++- 4 files changed, 99 insertions(+), 38 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala index 8164b9a96883c..78574b450586f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala @@ -298,18 +298,19 @@ case class AlterTableChangeColumnsCommand( DDLUtils.verifyAlterTableType(catalog, table, isView = false) // Create a map that converts the origin column to the new column with changed comment, throw - // a Exception if the column reference is invalid or the column name/dataType is changed. + // an AnalysisException if the column reference is invalid or the column name/dataType is + // changed. val columnsMap = columns.map { case (oldName: String, newField: StructField) => // Find the origin column from schema by column name. - val originColumn = findColumn(table.schema, oldName, resolver) - // Throw a Exception if the column name/dataType is changed. + val originColumn = findColumnByName(table.schema, oldName, resolver) + // Throw an AnalysisException if the column name/dataType is changed. if (!columnEqual(originColumn, newField, resolver)) { throw new AnalysisException( "ALTER TABLE CHANGE COLUMN is not supported for changing column " + s"'${originColumn.name}' with type '${originColumn.dataType}' to " + s"'${newField.name}' with type '${newField.dataType}'") } - // Create a new column from the origin column with new comment. + // Create a new column from the origin column with the new comment. val newColumn = addComment(originColumn, newField.getComment) // Create the map from origin column to changed column originColumn.name -> newColumn @@ -322,9 +323,10 @@ case class AlterTableChangeColumnsCommand( Seq.empty[Row] } - // Find the origin column from schema by column name, throw a Exception if the column + // Find the origin column from schema by column name, throw an AnalysisException if the column // reference is invalid. - private def findColumn(schema: StructType, name: String, resolver: Resolver): StructField = { + private def findColumnByName( + schema: StructType, name: String, resolver: Resolver): StructField = { schema.fields.collectFirst { case field if resolver(field.name, name) => field }.getOrElse(throw new AnalysisException( diff --git a/sql/core/src/test/resources/sql-tests/inputs/change-column.sql b/sql/core/src/test/resources/sql-tests/inputs/change-column.sql index 6a5404ee8ef6e..620cd7b7490d9 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/change-column.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/change-column.sql @@ -34,8 +34,11 @@ DESC test_change; ALTER TABLE test_change CHANGE a a1 STRING COMMENT 'this is column a1' AFTER b; DESC test_change; --- Case sensitive +-- Check the behavior with different values of CASE_SENSITIVE +SET spark.sql.caseSensitive=false; ALTER TABLE test_change CHANGE a A INT COMMENT 'this is column A'; +SET spark.sql.caseSensitive=true; +ALTER TABLE test_change CHANGE a A INT COMMENT 'this is column A1'; DESC test_change; -- Change column can't apply to a temporary/global_temporary view diff --git a/sql/core/src/test/resources/sql-tests/results/change-column.sql.out b/sql/core/src/test/resources/sql-tests/results/change-column.sql.out index d4c04ce83b5c5..47bef7cc0fda4 100644 --- a/sql/core/src/test/resources/sql-tests/results/change-column.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/change-column.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 30 +-- Number of queries: 33 -- !query 0 @@ -202,70 +202,95 @@ c int -- !query 20 -ALTER TABLE test_change CHANGE a A INT COMMENT 'this is column A' +SET spark.sql.caseSensitive=false -- !query 20 schema -struct<> +struct -- !query 20 output - +spark.sql.caseSensitive -- !query 21 -DESC test_change +ALTER TABLE test_change CHANGE a A INT COMMENT 'this is column A' -- !query 21 schema -struct +struct<> -- !query 21 output -a int this is column A -b string #*02?` -c int + -- !query 22 -CREATE TEMPORARY VIEW temp_view(a, b) AS SELECT 1, "one" +SET spark.sql.caseSensitive=true -- !query 22 schema -struct<> +struct -- !query 22 output - +spark.sql.caseSensitive -- !query 23 -ALTER TABLE temp_view CHANGE a a INT COMMENT 'this is column a' +ALTER TABLE test_change CHANGE a A INT COMMENT 'this is column A1' -- !query 23 schema struct<> -- !query 23 output -org.apache.spark.sql.catalyst.analysis.NoSuchTableException -Table or view 'temp_view' not found in database 'default'; +org.apache.spark.sql.AnalysisException +ALTER TABLE CHANGE COLUMN is not supported for changing column 'a' with type 'IntegerType' to 'A' with type 'IntegerType'; -- !query 24 -CREATE GLOBAL TEMPORARY VIEW global_temp_view(a, b) AS SELECT 1, "one" +DESC test_change -- !query 24 schema -struct<> +struct -- !query 24 output - +a int this is column A +b string #*02?` +c int -- !query 25 -ALTER TABLE global_temp.global_temp_view CHANGE a a INT COMMENT 'this is column a' +CREATE TEMPORARY VIEW temp_view(a, b) AS SELECT 1, "one" -- !query 25 schema struct<> -- !query 25 output -org.apache.spark.sql.catalyst.analysis.NoSuchDatabaseException -Database 'global_temp' not found; + -- !query 26 -CREATE TABLE partition_table(a INT, b STRING) PARTITIONED BY (c INT, d STRING) +ALTER TABLE temp_view CHANGE a a INT COMMENT 'this is column a' -- !query 26 schema struct<> -- !query 26 output - +org.apache.spark.sql.catalyst.analysis.NoSuchTableException +Table or view 'temp_view' not found in database 'default'; -- !query 27 -ALTER TABLE partition_table PARTITION (c = 1) CHANGE COLUMN a new_a INT +CREATE GLOBAL TEMPORARY VIEW global_temp_view(a, b) AS SELECT 1, "one" -- !query 27 schema struct<> -- !query 27 output + + + +-- !query 28 +ALTER TABLE global_temp.global_temp_view CHANGE a a INT COMMENT 'this is column a' +-- !query 28 schema +struct<> +-- !query 28 output +org.apache.spark.sql.catalyst.analysis.NoSuchDatabaseException +Database 'global_temp' not found; + + +-- !query 29 +CREATE TABLE partition_table(a INT, b STRING) PARTITIONED BY (c INT, d STRING) +-- !query 29 schema +struct<> +-- !query 29 output + + + +-- !query 30 +ALTER TABLE partition_table PARTITION (c = 1) CHANGE COLUMN a new_a INT +-- !query 30 schema +struct<> +-- !query 30 output org.apache.spark.sql.catalyst.parser.ParseException Operation not allowed: ALTER TABLE table PARTITION partition_spec CHANGE COLUMN(line 1, pos 0) @@ -275,17 +300,17 @@ ALTER TABLE partition_table PARTITION (c = 1) CHANGE COLUMN a new_a INT ^^^ --- !query 28 +-- !query 31 DROP TABLE test_change --- !query 28 schema +-- !query 31 schema struct<> --- !query 28 output +-- !query 31 output --- !query 29 +-- !query 32 DROP TABLE partition_table --- !query 29 schema +-- !query 32 schema struct<> --- !query 29 output +-- !query 32 output diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala index e61beb49e47f4..dd9f6fe69504d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala @@ -84,12 +84,15 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { serde = None, compressed = false, properties = Map()) + val metadata = new MetadataBuilder() + .putString("key", "value") + .build() CatalogTable( identifier = name, tableType = CatalogTableType.EXTERNAL, storage = storage, schema = new StructType() - .add("col1", "int") + .add("col1", "int", nullable = true, metadata = metadata) .add("col2", "string") .add("a", "int") .add("b", "int"), @@ -771,6 +774,14 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { testSetSerdePartition(isDatasourceTable = true) } + test("alter table: change column") { + testChangeColumn(isDatasourceTable = false) + } + + test("alter table: change column (datasource table)") { + testChangeColumn(isDatasourceTable = true) + } + test("alter table: bucketing is not supported") { val catalog = spark.sessionState.catalog val tableIdent = TableIdentifier("tab1", Some("dbx")) @@ -1368,6 +1379,26 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { Set(Map("a" -> "1", "b" -> "p"), Map("a" -> "20", "b" -> "c"), Map("a" -> "3", "b" -> "p"))) } + private def testChangeColumn(isDatasourceTable: Boolean): Unit = { + val catalog = spark.sessionState.catalog + val resolver = spark.sessionState.conf.resolver + val tableIdent = TableIdentifier("tab1", Some("dbx")) + createDatabase(catalog, "dbx") + createTable(catalog, tableIdent) + if (isDatasourceTable) { + convertToDatasourceTable(catalog, tableIdent) + } + def getMetadata(colName: String): Metadata = { + val column = catalog.getTableMetadata(tableIdent).schema.fields.find { field => + resolver(field.name, colName) + } + column.map(_.metadata).getOrElse(Metadata.empty) + } + // Ensure that change column will preserve other metadata fields. + sql("ALTER TABLE dbx.tab1 CHANGE COLUMN col1 col1 INT COMMENT 'this is col1'") + assert(getMetadata("col1").getString("key") == "value") + } + test("drop build-in function") { Seq("true", "false").foreach { caseSensitive => withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive) { From 73c82fc1e2192a7bd91273735c36697d949ae76d Mon Sep 17 00:00:00 2001 From: jiangxingbo Date: Wed, 14 Dec 2016 19:11:19 +0800 Subject: [PATCH 16/16] modify syntax --- .../spark/sql/catalyst/parser/SqlBase.g4 | 10 +- .../spark/sql/execution/SparkSqlParser.scala | 28 ++- .../spark/sql/execution/command/ddl.scala | 43 ++-- .../sql-tests/inputs/change-column.sql | 7 +- .../sql-tests/results/change-column.sql.out | 200 +++++++++--------- .../execution/command/DDLCommandSuite.scala | 10 +- 6 files changed, 139 insertions(+), 159 deletions(-) diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index a44f47adb51fd..70a664b965bdb 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -90,7 +90,7 @@ statement | ALTER (TABLE | VIEW) tableIdentifier UNSET TBLPROPERTIES (IF EXISTS)? tablePropertyList #unsetTableProperties | ALTER TABLE tableIdentifier partitionSpec? - CHANGE COLUMN? expandColTypeList #changeColumns + CHANGE COLUMN? identifier colType colPosition? #changeColumn | ALTER TABLE tableIdentifier (partitionSpec)? SET SERDE STRING (WITH SERDEPROPERTIES tablePropertyList)? #setTableSerDe | ALTER TABLE tableIdentifier (partitionSpec)? @@ -598,14 +598,6 @@ colType : identifier dataType (COMMENT STRING)? ; -expandColTypeList - : expandColType (',' expandColType)* - ; - -expandColType - : identifier colType colPosition? - ; - complexColTypeList : complexColType (',' complexColType)* ; diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index 2f65bed1db2fc..6cc655c9ff00d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -877,32 +877,30 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { } /** - * Create a [[AlterTableChangeColumnsCommand]] command. + * Create a [[AlterTableChangeColumnCommand]] command. * * For example: * {{{ * ALTER TABLE table [PARTITION partition_spec] - * CHANGE [COLUMN] `col` `col` dataType [COMMENT "comment"] [FIRST | AFTER `otherCol`] - * [, `col2` `col2` dataType [COMMENT "comment"] [FIRST | AFTER `otherCol`], ...] + * CHANGE [COLUMN] column_old_name column_new_name column_dataType [COMMENT column_comment] + * [FIRST | AFTER column_name]; * }}} */ - override def visitChangeColumns(ctx: ChangeColumnsContext): LogicalPlan = withOrigin(ctx) { + override def visitChangeColumn(ctx: ChangeColumnContext): LogicalPlan = withOrigin(ctx) { if (ctx.partitionSpec != null) { operationNotAllowed("ALTER TABLE table PARTITION partition_spec CHANGE COLUMN", ctx) } - val tableName = visitTableIdentifier(ctx.tableIdentifier) - - val columns = ctx.expandColTypeList.expandColType.asScala.map { col => - if (col.colPosition != null) { - operationNotAllowed( - "ALTER TABLE table [PARTITION partition_spec] CHANGE COLUMN ... FIRST | AFTER otherCol", - ctx) - } - col.identifier.getText -> visitColType(col.colType) - }.toMap + if (ctx.colPosition != null) { + operationNotAllowed( + "ALTER TABLE table [PARTITION partition_spec] CHANGE COLUMN ... FIRST | AFTER otherCol", + ctx) + } - AlterTableChangeColumnsCommand(tableName, columns) + AlterTableChangeColumnCommand( + tableName = visitTableIdentifier(ctx.tableIdentifier), + columnName = ctx.identifier.getText, + newColumn = visitColType(ctx.colType)) } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala index 78574b450586f..522158b641673 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala @@ -276,8 +276,8 @@ case class AlterTableUnsetPropertiesCommand( /** - * A command to change the columns for a table, only support changing the comments of non-partition - * columns for now. + * A command to change the column for a table, only support changing the comment of a non-partition + * column for now. * * The syntax of using this command in SQL is: * {{{ @@ -286,9 +286,10 @@ case class AlterTableUnsetPropertiesCommand( * [FIRST | AFTER column_name]; * }}} */ -case class AlterTableChangeColumnsCommand( +case class AlterTableChangeColumnCommand( tableName: TableIdentifier, - columns: Map[String, StructField]) extends RunnableCommand { + columnName: String, + newColumn: StructField) extends RunnableCommand { // TODO: support change column name/dataType/metadata/position. override def run(sparkSession: SparkSession): Seq[Row] = { @@ -297,26 +298,24 @@ case class AlterTableChangeColumnsCommand( val resolver = sparkSession.sessionState.conf.resolver DDLUtils.verifyAlterTableType(catalog, table, isView = false) - // Create a map that converts the origin column to the new column with changed comment, throw - // an AnalysisException if the column reference is invalid or the column name/dataType is - // changed. - val columnsMap = columns.map { case (oldName: String, newField: StructField) => - // Find the origin column from schema by column name. - val originColumn = findColumnByName(table.schema, oldName, resolver) - // Throw an AnalysisException if the column name/dataType is changed. - if (!columnEqual(originColumn, newField, resolver)) { - throw new AnalysisException( - "ALTER TABLE CHANGE COLUMN is not supported for changing column " + - s"'${originColumn.name}' with type '${originColumn.dataType}' to " + - s"'${newField.name}' with type '${newField.dataType}'") - } - // Create a new column from the origin column with the new comment. - val newColumn = addComment(originColumn, newField.getComment) - // Create the map from origin column to changed column - originColumn.name -> newColumn + // Find the origin column from schema by column name. + val originColumn = findColumnByName(table.schema, columnName, resolver) + // Throw an AnalysisException if the column name/dataType is changed. + if (!columnEqual(originColumn, newColumn, resolver)) { + throw new AnalysisException( + "ALTER TABLE CHANGE COLUMN is not supported for changing column " + + s"'${originColumn.name}' with type '${originColumn.dataType}' to " + + s"'${newColumn.name}' with type '${newColumn.dataType}'") } - val newSchema = table.schema.fields.map(field => columnsMap.getOrElse(field.name, field)) + val newSchema = table.schema.fields.map { field => + if (field.name == originColumn.name) { + // Create a new column from the origin column with the new comment. + addComment(field, newColumn.getComment) + } else { + field + } + } val newTable = table.copy(schema = StructType(newSchema)) catalog.alterTable(newTable) diff --git a/sql/core/src/test/resources/sql-tests/inputs/change-column.sql b/sql/core/src/test/resources/sql-tests/inputs/change-column.sql index 620cd7b7490d9..818b19c50f16f 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/change-column.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/change-column.sql @@ -4,22 +4,21 @@ DESC test_change; -- Change column name (not supported yet) ALTER TABLE test_change CHANGE a a1 INT; -ALTER TABLE test_change CHANGE b b1 STRING, c c1 INT; DESC test_change; -- Change column dataType (not supported yet) ALTER TABLE test_change CHANGE a a STRING; -ALTER TABLE test_change CHANGE b b INT, c c DOUBLE; DESC test_change; -- Change column position (not supported yet) ALTER TABLE test_change CHANGE a a INT AFTER b; -ALTER TABLE test_change CHANGE b b STRING FIRST, c c INT AFTER b; +ALTER TABLE test_change CHANGE b b STRING FIRST; DESC test_change; -- Change column comment ALTER TABLE test_change CHANGE a a INT COMMENT 'this is column a'; -ALTER TABLE test_change CHANGE b b STRING COMMENT '#*02?`', c c INT COMMENT ''; +ALTER TABLE test_change CHANGE b b STRING COMMENT '#*02?`'; +ALTER TABLE test_change CHANGE c c INT COMMENT ''; DESC test_change; -- Don't change anything. diff --git a/sql/core/src/test/resources/sql-tests/results/change-column.sql.out b/sql/core/src/test/resources/sql-tests/results/change-column.sql.out index 47bef7cc0fda4..156ddb86adf9c 100644 --- a/sql/core/src/test/resources/sql-tests/results/change-column.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/change-column.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 33 +-- Number of queries: 32 -- !query 0 @@ -30,57 +30,39 @@ ALTER TABLE CHANGE COLUMN is not supported for changing column 'a' with type 'In -- !query 3 -ALTER TABLE test_change CHANGE b b1 STRING, c c1 INT --- !query 3 schema -struct<> --- !query 3 output -org.apache.spark.sql.AnalysisException -ALTER TABLE CHANGE COLUMN is not supported for changing column 'b' with type 'StringType' to 'b1' with type 'StringType'; - - --- !query 4 DESC test_change --- !query 4 schema +-- !query 3 schema struct --- !query 4 output +-- !query 3 output a int b string c int --- !query 5 +-- !query 4 ALTER TABLE test_change CHANGE a a STRING --- !query 5 schema +-- !query 4 schema struct<> --- !query 5 output +-- !query 4 output org.apache.spark.sql.AnalysisException ALTER TABLE CHANGE COLUMN is not supported for changing column 'a' with type 'IntegerType' to 'a' with type 'StringType'; --- !query 6 -ALTER TABLE test_change CHANGE b b INT, c c DOUBLE --- !query 6 schema -struct<> --- !query 6 output -org.apache.spark.sql.AnalysisException -ALTER TABLE CHANGE COLUMN is not supported for changing column 'b' with type 'StringType' to 'b' with type 'IntegerType'; - - --- !query 7 +-- !query 5 DESC test_change --- !query 7 schema +-- !query 5 schema struct --- !query 7 output +-- !query 5 output a int b string c int --- !query 8 +-- !query 6 ALTER TABLE test_change CHANGE a a INT AFTER b --- !query 8 schema +-- !query 6 schema struct<> --- !query 8 output +-- !query 6 output org.apache.spark.sql.catalyst.parser.ParseException Operation not allowed: ALTER TABLE table [PARTITION partition_spec] CHANGE COLUMN ... FIRST | AFTER otherCol(line 1, pos 0) @@ -90,98 +72,106 @@ ALTER TABLE test_change CHANGE a a INT AFTER b ^^^ --- !query 9 -ALTER TABLE test_change CHANGE b b STRING FIRST, c c INT AFTER b --- !query 9 schema +-- !query 7 +ALTER TABLE test_change CHANGE b b STRING FIRST +-- !query 7 schema struct<> --- !query 9 output +-- !query 7 output org.apache.spark.sql.catalyst.parser.ParseException Operation not allowed: ALTER TABLE table [PARTITION partition_spec] CHANGE COLUMN ... FIRST | AFTER otherCol(line 1, pos 0) == SQL == -ALTER TABLE test_change CHANGE b b STRING FIRST, c c INT AFTER b +ALTER TABLE test_change CHANGE b b STRING FIRST ^^^ --- !query 10 +-- !query 8 DESC test_change --- !query 10 schema +-- !query 8 schema struct --- !query 10 output +-- !query 8 output a int b string c int --- !query 11 +-- !query 9 ALTER TABLE test_change CHANGE a a INT COMMENT 'this is column a' --- !query 11 schema +-- !query 9 schema struct<> --- !query 11 output +-- !query 9 output --- !query 12 -ALTER TABLE test_change CHANGE b b STRING COMMENT '#*02?`', c c INT COMMENT '' --- !query 12 schema +-- !query 10 +ALTER TABLE test_change CHANGE b b STRING COMMENT '#*02?`' +-- !query 10 schema struct<> --- !query 12 output +-- !query 10 output --- !query 13 +-- !query 11 +ALTER TABLE test_change CHANGE c c INT COMMENT '' +-- !query 11 schema +struct<> +-- !query 11 output + + + +-- !query 12 DESC test_change --- !query 13 schema +-- !query 12 schema struct --- !query 13 output +-- !query 12 output a int this is column a b string #*02?` c int --- !query 14 +-- !query 13 ALTER TABLE test_change CHANGE a a INT COMMENT 'this is column a' --- !query 14 schema +-- !query 13 schema struct<> --- !query 14 output +-- !query 13 output --- !query 15 +-- !query 14 DESC test_change --- !query 15 schema +-- !query 14 schema struct --- !query 15 output +-- !query 14 output a int this is column a b string #*02?` c int --- !query 16 +-- !query 15 ALTER TABLE test_change CHANGE invalid_col invalid_col INT --- !query 16 schema +-- !query 15 schema struct<> --- !query 16 output +-- !query 15 output org.apache.spark.sql.AnalysisException Invalid column reference 'invalid_col', table schema is 'StructType(StructField(a,IntegerType,true), StructField(b,StringType,true), StructField(c,IntegerType,true))'; --- !query 17 +-- !query 16 DESC test_change --- !query 17 schema +-- !query 16 schema struct --- !query 17 output +-- !query 16 output a int this is column a b string #*02?` c int --- !query 18 +-- !query 17 ALTER TABLE test_change CHANGE a a1 STRING COMMENT 'this is column a1' AFTER b --- !query 18 schema +-- !query 17 schema struct<> --- !query 18 output +-- !query 17 output org.apache.spark.sql.catalyst.parser.ParseException Operation not allowed: ALTER TABLE table [PARTITION partition_spec] CHANGE COLUMN ... FIRST | AFTER otherCol(line 1, pos 0) @@ -191,106 +181,106 @@ ALTER TABLE test_change CHANGE a a1 STRING COMMENT 'this is column a1' AFTER b ^^^ --- !query 19 +-- !query 18 DESC test_change --- !query 19 schema +-- !query 18 schema struct --- !query 19 output +-- !query 18 output a int this is column a b string #*02?` c int --- !query 20 +-- !query 19 SET spark.sql.caseSensitive=false --- !query 20 schema +-- !query 19 schema struct --- !query 20 output +-- !query 19 output spark.sql.caseSensitive --- !query 21 +-- !query 20 ALTER TABLE test_change CHANGE a A INT COMMENT 'this is column A' --- !query 21 schema +-- !query 20 schema struct<> --- !query 21 output +-- !query 20 output --- !query 22 +-- !query 21 SET spark.sql.caseSensitive=true --- !query 22 schema +-- !query 21 schema struct --- !query 22 output +-- !query 21 output spark.sql.caseSensitive --- !query 23 +-- !query 22 ALTER TABLE test_change CHANGE a A INT COMMENT 'this is column A1' --- !query 23 schema +-- !query 22 schema struct<> --- !query 23 output +-- !query 22 output org.apache.spark.sql.AnalysisException ALTER TABLE CHANGE COLUMN is not supported for changing column 'a' with type 'IntegerType' to 'A' with type 'IntegerType'; --- !query 24 +-- !query 23 DESC test_change --- !query 24 schema +-- !query 23 schema struct --- !query 24 output +-- !query 23 output a int this is column A b string #*02?` c int --- !query 25 +-- !query 24 CREATE TEMPORARY VIEW temp_view(a, b) AS SELECT 1, "one" --- !query 25 schema +-- !query 24 schema struct<> --- !query 25 output +-- !query 24 output --- !query 26 +-- !query 25 ALTER TABLE temp_view CHANGE a a INT COMMENT 'this is column a' --- !query 26 schema +-- !query 25 schema struct<> --- !query 26 output +-- !query 25 output org.apache.spark.sql.catalyst.analysis.NoSuchTableException Table or view 'temp_view' not found in database 'default'; --- !query 27 +-- !query 26 CREATE GLOBAL TEMPORARY VIEW global_temp_view(a, b) AS SELECT 1, "one" --- !query 27 schema +-- !query 26 schema struct<> --- !query 27 output +-- !query 26 output --- !query 28 +-- !query 27 ALTER TABLE global_temp.global_temp_view CHANGE a a INT COMMENT 'this is column a' --- !query 28 schema +-- !query 27 schema struct<> --- !query 28 output +-- !query 27 output org.apache.spark.sql.catalyst.analysis.NoSuchDatabaseException Database 'global_temp' not found; --- !query 29 +-- !query 28 CREATE TABLE partition_table(a INT, b STRING) PARTITIONED BY (c INT, d STRING) --- !query 29 schema +-- !query 28 schema struct<> --- !query 29 output +-- !query 28 output --- !query 30 +-- !query 29 ALTER TABLE partition_table PARTITION (c = 1) CHANGE COLUMN a new_a INT --- !query 30 schema +-- !query 29 schema struct<> --- !query 30 output +-- !query 29 output org.apache.spark.sql.catalyst.parser.ParseException Operation not allowed: ALTER TABLE table PARTITION partition_spec CHANGE COLUMN(line 1, pos 0) @@ -300,17 +290,17 @@ ALTER TABLE partition_table PARTITION (c = 1) CHANGE COLUMN a new_a INT ^^^ --- !query 31 +-- !query 30 DROP TABLE test_change --- !query 31 schema +-- !query 30 schema struct<> --- !query 31 output +-- !query 30 output --- !query 32 +-- !query 31 DROP TABLE partition_table --- !query 32 schema +-- !query 31 schema struct<> --- !query 32 output +-- !query 31 output diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala index 98c47c354d6e0..1a5e5226c2ec9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala @@ -666,12 +666,14 @@ class DDLCommandSuite extends PlanTest { val parsed1 = parser.parsePlan(sql1) val parsed2 = parser.parsePlan(sql2) val tableIdent = TableIdentifier("table_name", None) - val expected1 = AlterTableChangeColumnsCommand( + val expected1 = AlterTableChangeColumnCommand( tableIdent, - Map("col_old_name" -> StructField("col_new_name", IntegerType))) - val expected2 = AlterTableChangeColumnsCommand( + "col_old_name", + StructField("col_new_name", IntegerType)) + val expected2 = AlterTableChangeColumnCommand( tableIdent, - Map("col_name" -> StructField("col_name", IntegerType).withComment("new_comment"))) + "col_name", + StructField("col_name", IntegerType).withComment("new_comment")) comparePlans(parsed1, expected1) comparePlans(parsed2, expected2) }