aboutsummaryrefslogtreecommitdiff
path: root/sql
diff options
context:
space:
mode:
Diffstat (limited to 'sql')
-rw-r--r--sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g410
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala29
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala71
-rw-r--r--sql/core/src/test/resources/sql-tests/inputs/change-column.sql55
-rw-r--r--sql/core/src/test/resources/sql-tests/results/change-column.sql.out306
-rw-r--r--sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala43
-rw-r--r--sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala33
7 files changed, 529 insertions, 18 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 63055b62d2..a34087cb6c 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 tableIdentifier partitionSpec?
+ CHANGE COLUMN? identifier colType colPosition? #changeColumn
| ALTER TABLE tableIdentifier (partitionSpec)?
SET SERDE STRING (WITH SERDEPROPERTIES tablePropertyList)? #setTableSerDe
| ALTER TABLE tableIdentifier (partitionSpec)?
@@ -194,7 +196,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,6 +581,10 @@ intervalValue
| STRING
;
+colPosition
+ : FIRST | AFTER identifier
+ ;
+
dataType
: complex=ARRAY '<' dataType '>' #complexDataType
| complex=MAP '<' dataType ',' dataType '>' #complexDataType
@@ -671,7 +676,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
@@ -761,6 +766,7 @@ PRECEDING: 'PRECEDING';
FOLLOWING: 'FOLLOWING';
CURRENT: 'CURRENT';
FIRST: 'FIRST';
+AFTER: 'AFTER';
LAST: 'LAST';
ROW: 'ROW';
WITH: 'WITH';
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 cab1b22c99..14a983e43b 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,6 +887,33 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder {
}
/**
+ * Create a [[AlterTableChangeColumnCommand]] command.
+ *
+ * For example:
+ * {{{
+ * ALTER TABLE table [PARTITION partition_spec]
+ * CHANGE [COLUMN] column_old_name column_new_name column_dataType [COMMENT column_comment]
+ * [FIRST | AFTER column_name];
+ * }}}
+ */
+ override def visitChangeColumn(ctx: ChangeColumnContext): LogicalPlan = withOrigin(ctx) {
+ if (ctx.partitionSpec != null) {
+ operationNotAllowed("ALTER TABLE table PARTITION partition_spec CHANGE COLUMN", ctx)
+ }
+
+ if (ctx.colPosition != null) {
+ operationNotAllowed(
+ "ALTER TABLE table [PARTITION partition_spec] CHANGE COLUMN ... FIRST | AFTER otherCol",
+ ctx)
+ }
+
+ AlterTableChangeColumnCommand(
+ tableName = visitTableIdentifier(ctx.tableIdentifier),
+ columnName = ctx.identifier.getText,
+ newColumn = visitColType(ctx.colType))
+ }
+
+ /**
* Create location string.
*/
override def visitLocationSpec(ctx: LocationSpecContext): String = withOrigin(ctx) {
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 c62c14200c..522158b641 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,77 @@ case class AlterTableUnsetPropertiesCommand(
}
+
+/**
+ * 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:
+ * {{{
+ * ALTER TABLE table_identifier
+ * CHANGE [COLUMN] column_old_name column_new_name column_dataType [COMMENT column_comment]
+ * [FIRST | AFTER column_name];
+ * }}}
+ */
+case class AlterTableChangeColumnCommand(
+ tableName: TableIdentifier,
+ columnName: String,
+ newColumn: StructField) extends RunnableCommand {
+
+ // TODO: support change column name/dataType/metadata/position.
+ 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)
+
+ // 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 =>
+ 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)
+
+ Seq.empty[Row]
+ }
+
+ // Find the origin column from schema by column name, throw an AnalysisException if the column
+ // reference is invalid.
+ 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(
+ 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
+ }
+}
+
/**
* A command that sets the serde class and/or serde properties of a table/view.
*
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 0000000000..818b19c50f
--- /dev/null
+++ b/sql/core/src/test/resources/sql-tests/inputs/change-column.sql
@@ -0,0 +1,55 @@
+-- Create the origin table
+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;
+DESC test_change;
+
+-- Change column dataType (not supported yet)
+ALTER TABLE test_change CHANGE a a STRING;
+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;
+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?`';
+ALTER TABLE test_change CHANGE 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 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;
+
+-- 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
+CREATE TEMPORARY VIEW temp_view(a, b) AS SELECT 1, "one";
+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
new file mode 100644
index 0000000000..156ddb86ad
--- /dev/null
+++ b/sql/core/src/test/resources/sql-tests/results/change-column.sql.out
@@ -0,0 +1,306 @@
+-- Automatically generated by SQLQueryTestSuite
+-- Number of queries: 32
+
+
+-- !query 0
+CREATE TABLE test_change(a INT, b STRING, c INT)
+-- !query 0 schema
+struct<>
+-- !query 0 output
+
+
+
+-- !query 1
+DESC test_change
+-- !query 1 schema
+struct<col_name:string,data_type:string,comment:string>
+-- !query 1 output
+a int
+b string
+c int
+
+
+-- !query 2
+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 'a' with type 'IntegerType' to 'a1' with type 'IntegerType';
+
+
+-- !query 3
+DESC test_change
+-- !query 3 schema
+struct<col_name:string,data_type:string,comment:string>
+-- !query 3 output
+a int
+b string
+c int
+
+
+-- !query 4
+ALTER TABLE test_change CHANGE a a STRING
+-- !query 4 schema
+struct<>
+-- !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 5
+DESC test_change
+-- !query 5 schema
+struct<col_name:string,data_type:string,comment:string>
+-- !query 5 output
+a int
+b string
+c int
+
+
+-- !query 6
+ALTER TABLE test_change CHANGE a a INT AFTER b
+-- !query 6 schema
+struct<>
+-- !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)
+
+== SQL ==
+ALTER TABLE test_change CHANGE a a INT AFTER b
+^^^
+
+
+-- !query 7
+ALTER TABLE test_change CHANGE b b STRING FIRST
+-- !query 7 schema
+struct<>
+-- !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
+^^^
+
+
+-- !query 8
+DESC test_change
+-- !query 8 schema
+struct<col_name:string,data_type:string,comment:string>
+-- !query 8 output
+a int
+b string
+c int
+
+
+-- !query 9
+ALTER TABLE test_change CHANGE a a INT COMMENT 'this is column a'
+-- !query 9 schema
+struct<>
+-- !query 9 output
+
+
+
+-- !query 10
+ALTER TABLE test_change CHANGE b b STRING COMMENT '#*02?`'
+-- !query 10 schema
+struct<>
+-- !query 10 output
+
+
+
+-- !query 11
+ALTER TABLE test_change CHANGE c c INT COMMENT ''
+-- !query 11 schema
+struct<>
+-- !query 11 output
+
+
+
+-- !query 12
+DESC test_change
+-- !query 12 schema
+struct<col_name:string,data_type:string,comment:string>
+-- !query 12 output
+a int this is column a
+b string #*02?`
+c int
+
+
+-- !query 13
+ALTER TABLE test_change CHANGE a a INT COMMENT 'this is column a'
+-- !query 13 schema
+struct<>
+-- !query 13 output
+
+
+
+-- !query 14
+DESC test_change
+-- !query 14 schema
+struct<col_name:string,data_type:string,comment:string>
+-- !query 14 output
+a int this is column a
+b string #*02?`
+c int
+
+
+-- !query 15
+ALTER TABLE test_change CHANGE invalid_col invalid_col INT
+-- !query 15 schema
+struct<>
+-- !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 16
+DESC test_change
+-- !query 16 schema
+struct<col_name:string,data_type:string,comment:string>
+-- !query 16 output
+a int this is column a
+b string #*02?`
+c int
+
+
+-- !query 17
+ALTER TABLE test_change CHANGE a a1 STRING COMMENT 'this is column a1' AFTER b
+-- !query 17 schema
+struct<>
+-- !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)
+
+== SQL ==
+ALTER TABLE test_change CHANGE a a1 STRING COMMENT 'this is column a1' AFTER b
+^^^
+
+
+-- !query 18
+DESC test_change
+-- !query 18 schema
+struct<col_name:string,data_type:string,comment:string>
+-- !query 18 output
+a int this is column a
+b string #*02?`
+c int
+
+
+-- !query 19
+SET spark.sql.caseSensitive=false
+-- !query 19 schema
+struct<key:string,value:string>
+-- !query 19 output
+spark.sql.caseSensitive
+
+
+-- !query 20
+ALTER TABLE test_change CHANGE a A INT COMMENT 'this is column A'
+-- !query 20 schema
+struct<>
+-- !query 20 output
+
+
+
+-- !query 21
+SET spark.sql.caseSensitive=true
+-- !query 21 schema
+struct<key:string,value:string>
+-- !query 21 output
+spark.sql.caseSensitive
+
+
+-- !query 22
+ALTER TABLE test_change CHANGE a A INT COMMENT 'this is column A1'
+-- !query 22 schema
+struct<>
+-- !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 23
+DESC test_change
+-- !query 23 schema
+struct<col_name:string,data_type:string,comment:string>
+-- !query 23 output
+a int this is column A
+b string #*02?`
+c int
+
+
+-- !query 24
+CREATE TEMPORARY VIEW temp_view(a, b) AS SELECT 1, "one"
+-- !query 24 schema
+struct<>
+-- !query 24 output
+
+
+
+-- !query 25
+ALTER TABLE temp_view CHANGE a a INT COMMENT 'this is column a'
+-- !query 25 schema
+struct<>
+-- !query 25 output
+org.apache.spark.sql.catalyst.analysis.NoSuchTableException
+Table or view 'temp_view' not found in database 'default';
+
+
+-- !query 26
+CREATE GLOBAL TEMPORARY VIEW global_temp_view(a, b) AS SELECT 1, "one"
+-- !query 26 schema
+struct<>
+-- !query 26 output
+
+
+
+-- !query 27
+ALTER TABLE global_temp.global_temp_view CHANGE a a INT COMMENT 'this is column a'
+-- !query 27 schema
+struct<>
+-- !query 27 output
+org.apache.spark.sql.catalyst.analysis.NoSuchDatabaseException
+Database 'global_temp' not found;
+
+
+-- !query 28
+CREATE TABLE partition_table(a INT, b STRING) PARTITIONED BY (c INT, d STRING)
+-- !query 28 schema
+struct<>
+-- !query 28 output
+
+
+
+-- !query 29
+ALTER TABLE partition_table PARTITION (c = 1) CHANGE COLUMN a new_a INT
+-- !query 29 schema
+struct<>
+-- !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)
+
+== SQL ==
+ALTER TABLE partition_table PARTITION (c = 1) CHANGE COLUMN a new_a INT
+^^^
+
+
+-- !query 30
+DROP TABLE test_change
+-- !query 30 schema
+struct<>
+-- !query 30 output
+
+
+
+-- !query 31
+DROP TABLE partition_table
+-- !query 31 schema
+struct<>
+-- !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 5ef5f8ee77..1a5e5226c2 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,34 @@ 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 = AlterTableChangeColumnCommand(
+ tableIdent,
+ "col_old_name",
+ StructField("col_new_name", IntegerType))
+ val expected2 = AlterTableChangeColumnCommand(
+ tableIdent,
+ "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')")
@@ -695,19 +723,6 @@ 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")
- 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
- """.stripMargin)
- }
-
test("alter table: add/replace columns (not allowed)") {
assertUnsupported(
"""
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 4c0e0fa990..4bfbebddfc 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) {