Skip to content

Commit f9542d0

Browse files
nikolamand-dbcloud-fan
authored andcommitted
[SPARK-48413][SQL] ALTER COLUMN with collation
### What changes were proposed in this pull request? Add support for changing collation of a column with `ALTER COLUMN` command. Use existing support for `ALTER COLUMN` with type to enable changing collations of column. Syntax example: ``` ALTER TABLE t1 ALTER COLUMN col TYPE STRING COLLATE UTF8_BINARY_LCASE ``` ### Why are the changes needed? Enable changing collation on column. ### Does this PR introduce _any_ user-facing change? Yes, it adds support for changing collation of column. ### How was this patch tested? Added tests to `DDLSuite` and `DataTypeSuite`. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #46734 from nikolamand-db/SPARK-48413. Authored-by: Nikola Mandic <nikola.mandic@databricks.com> Signed-off-by: Wenchen Fan <wenchen@databricks.com>
1 parent 5baaa61 commit f9542d0

File tree

6 files changed

+290
-13
lines changed

6 files changed

+290
-13
lines changed

common/utils/src/main/resources/error/error-conditions.json

Lines changed: 6 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -119,6 +119,12 @@
119119
],
120120
"sqlState" : "42KDE"
121121
},
122+
"CANNOT_ALTER_COLLATION_BUCKET_COLUMN" : {
123+
"message" : [
124+
"ALTER TABLE (ALTER|CHANGE) COLUMN cannot change collation of type/subtypes of bucket columns, but found the bucket column <columnName> in the table <tableName>."
125+
],
126+
"sqlState" : "428FR"
127+
},
122128
"CANNOT_ALTER_PARTITION_COLUMN" : {
123129
"message" : [
124130
"ALTER TABLE (ALTER|CHANGE) COLUMN is not supported for partition columns, but found the partition column <columnName> in the table <tableName>."

sql/api/src/main/scala/org/apache/spark/sql/types/DataType.scala

Lines changed: 35 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -408,6 +408,41 @@ object DataType {
408408
}
409409
}
410410

411+
/**
412+
* Check if `from` is equal to `to` type except for collations, which are checked to be
413+
* compatible so that data of type `from` can be interpreted as of type `to`.
414+
*/
415+
private[sql] def equalsIgnoreCompatibleCollation(
416+
from: DataType,
417+
to: DataType): Boolean = {
418+
(from, to) match {
419+
// String types with possibly different collations are compatible.
420+
case (_: StringType, _: StringType) => true
421+
422+
case (ArrayType(fromElement, fromContainsNull), ArrayType(toElement, toContainsNull)) =>
423+
(fromContainsNull == toContainsNull) &&
424+
equalsIgnoreCompatibleCollation(fromElement, toElement)
425+
426+
case (MapType(fromKey, fromValue, fromContainsNull),
427+
MapType(toKey, toValue, toContainsNull)) =>
428+
fromContainsNull == toContainsNull &&
429+
// Map keys cannot change collation.
430+
fromKey == toKey &&
431+
equalsIgnoreCompatibleCollation(fromValue, toValue)
432+
433+
case (StructType(fromFields), StructType(toFields)) =>
434+
fromFields.length == toFields.length &&
435+
fromFields.zip(toFields).forall { case (fromField, toField) =>
436+
fromField.name == toField.name &&
437+
fromField.nullable == toField.nullable &&
438+
fromField.metadata == toField.metadata &&
439+
equalsIgnoreCompatibleCollation(fromField.dataType, toField.dataType)
440+
}
441+
442+
case (fromDataType, toDataType) => fromDataType == toDataType
443+
}
444+
}
445+
411446
/**
412447
* Returns true if the two data types share the same "shape", i.e. the types
413448
* are the same, but the field names don't need to be the same.

sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala

Lines changed: 9 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -2687,6 +2687,15 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase with Compilat
26872687
Map("tableName" -> toSQLId(tableName), "columnName" -> toSQLId(columnName))
26882688
)
26892689
}
2690+
2691+
def cannotAlterCollationBucketColumn(tableName: String, columnName: String): Throwable = {
2692+
new AnalysisException(
2693+
errorClass = "CANNOT_ALTER_COLLATION_BUCKET_COLUMN",
2694+
messageParameters =
2695+
Map("tableName" -> toSQLId(tableName), "columnName" -> toSQLId(columnName))
2696+
)
2697+
}
2698+
26902699
def cannotFindColumnError(name: String, fieldNames: Array[String]): Throwable = {
26912700
new AnalysisException(
26922701
errorClass = "_LEGACY_ERROR_TEMP_1246",

sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeSuite.scala

Lines changed: 109 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -689,6 +689,115 @@ class DataTypeSuite extends SparkFunSuite {
689689
false,
690690
caseSensitive = true)
691691

692+
def checkEqualsIgnoreCompatibleCollation(
693+
from: DataType,
694+
to: DataType,
695+
expected: Boolean): Unit = {
696+
val testName = s"equalsIgnoreCompatibleCollation: (from: $from, to: $to)"
697+
698+
test(testName) {
699+
assert(DataType.equalsIgnoreCompatibleCollation(from, to) === expected)
700+
}
701+
}
702+
703+
// Simple types.
704+
checkEqualsIgnoreCompatibleCollation(IntegerType, IntegerType, expected = true)
705+
checkEqualsIgnoreCompatibleCollation(BooleanType, BooleanType, expected = true)
706+
checkEqualsIgnoreCompatibleCollation(StringType, StringType, expected = true)
707+
checkEqualsIgnoreCompatibleCollation(IntegerType, BooleanType, expected = false)
708+
checkEqualsIgnoreCompatibleCollation(BooleanType, IntegerType, expected = false)
709+
checkEqualsIgnoreCompatibleCollation(StringType, BooleanType, expected = false)
710+
checkEqualsIgnoreCompatibleCollation(BooleanType, StringType, expected = false)
711+
checkEqualsIgnoreCompatibleCollation(StringType, IntegerType, expected = false)
712+
checkEqualsIgnoreCompatibleCollation(IntegerType, StringType, expected = false)
713+
// Collated `StringType`.
714+
checkEqualsIgnoreCompatibleCollation(StringType, StringType("UTF8_BINARY_LCASE"),
715+
expected = true)
716+
checkEqualsIgnoreCompatibleCollation(
717+
StringType("UTF8_BINARY"), StringType("UTF8_BINARY_LCASE"), expected = true)
718+
// Complex types.
719+
checkEqualsIgnoreCompatibleCollation(
720+
ArrayType(StringType),
721+
ArrayType(StringType("UTF8_BINARY_LCASE")),
722+
expected = true
723+
)
724+
checkEqualsIgnoreCompatibleCollation(
725+
ArrayType(StringType),
726+
ArrayType(ArrayType(StringType("UTF8_BINARY_LCASE"))),
727+
expected = false
728+
)
729+
checkEqualsIgnoreCompatibleCollation(
730+
ArrayType(ArrayType(StringType)),
731+
ArrayType(ArrayType(StringType("UTF8_BINARY_LCASE"))),
732+
expected = true
733+
)
734+
checkEqualsIgnoreCompatibleCollation(
735+
MapType(StringType, StringType),
736+
MapType(StringType, StringType("UTF8_BINARY_LCASE")),
737+
expected = true
738+
)
739+
checkEqualsIgnoreCompatibleCollation(
740+
MapType(StringType("UTF8_BINARY_LCASE"), StringType),
741+
MapType(StringType, StringType),
742+
expected = false
743+
)
744+
checkEqualsIgnoreCompatibleCollation(
745+
MapType(StringType("UTF8_BINARY_LCASE"), ArrayType(StringType)),
746+
MapType(StringType("UTF8_BINARY_LCASE"), ArrayType(StringType("UTF8_BINARY_LCASE"))),
747+
expected = true
748+
)
749+
checkEqualsIgnoreCompatibleCollation(
750+
MapType(ArrayType(StringType), IntegerType),
751+
MapType(ArrayType(StringType("UTF8_BINARY_LCASE")), IntegerType),
752+
expected = false
753+
)
754+
checkEqualsIgnoreCompatibleCollation(
755+
MapType(ArrayType(StringType("UTF8_BINARY_LCASE")), IntegerType),
756+
MapType(ArrayType(StringType("UTF8_BINARY_LCASE")), IntegerType),
757+
expected = true
758+
)
759+
checkEqualsIgnoreCompatibleCollation(
760+
StructType(StructField("a", StringType) :: Nil),
761+
StructType(StructField("a", StringType("UTF8_BINARY_LCASE")) :: Nil),
762+
expected = true
763+
)
764+
checkEqualsIgnoreCompatibleCollation(
765+
StructType(StructField("a", ArrayType(StringType)) :: Nil),
766+
StructType(StructField("a", ArrayType(StringType("UTF8_BINARY_LCASE"))) :: Nil),
767+
expected = true
768+
)
769+
checkEqualsIgnoreCompatibleCollation(
770+
StructType(StructField("a", MapType(StringType, IntegerType)) :: Nil),
771+
StructType(StructField("a", MapType(StringType("UTF8_BINARY_LCASE"), IntegerType)) :: Nil),
772+
expected = false
773+
)
774+
checkEqualsIgnoreCompatibleCollation(
775+
StructType(StructField("a", StringType) :: Nil),
776+
StructType(StructField("b", StringType("UTF8_BINARY_LCASE")) :: Nil),
777+
expected = false
778+
)
779+
// Null compatibility checks.
780+
checkEqualsIgnoreCompatibleCollation(
781+
ArrayType(StringType, containsNull = true),
782+
ArrayType(StringType, containsNull = false),
783+
expected = false
784+
)
785+
checkEqualsIgnoreCompatibleCollation(
786+
ArrayType(StringType, containsNull = true),
787+
ArrayType(StringType("UTF8_BINARY_LCASE"), containsNull = false),
788+
expected = false
789+
)
790+
checkEqualsIgnoreCompatibleCollation(
791+
MapType(StringType, StringType, valueContainsNull = true),
792+
MapType(StringType, StringType, valueContainsNull = false),
793+
expected = false
794+
)
795+
checkEqualsIgnoreCompatibleCollation(
796+
StructType(StructField("a", StringType) :: Nil),
797+
StructType(StructField("a", StringType, nullable = false) :: Nil),
798+
expected = false
799+
)
800+
692801
test("SPARK-25031: MapType should produce current formatted string for complex types") {
693802
val keyType: DataType = StructType(Seq(
694803
StructField("a", DataTypes.IntegerType),

sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala

Lines changed: 37 additions & 13 deletions
Original file line numberDiff line numberDiff line change
@@ -356,8 +356,8 @@ case class AlterTableUnsetPropertiesCommand(
356356

357357

358358
/**
359-
* A command to change the column for a table, only support changing the comment of a non-partition
360-
* column for now.
359+
* A command to change the column for a table, only support changing the comment or collation of
360+
* the data type or nested types (recursively) of a non-partition column for now.
361361
*
362362
* The syntax of using this command in SQL is:
363363
* {{{
@@ -387,32 +387,45 @@ case class AlterTableChangeColumnCommand(
387387
}
388388
// Find the origin column from dataSchema by column name.
389389
val originColumn = findColumnByName(table.dataSchema, columnName, resolver)
390-
// Throw an AnalysisException if the column name/dataType is changed.
391-
if (!columnEqual(originColumn, newColumn, resolver)) {
390+
val validType = canEvolveType(originColumn, newColumn)
391+
// Throw an AnalysisException on attempt to change collation of bucket column.
392+
if (validType && originColumn.dataType != newColumn.dataType) {
393+
val isBucketColumn = table.bucketSpec match {
394+
case Some(bucketSpec) => bucketSpec.bucketColumnNames.exists(resolver(columnName, _))
395+
case _ => false
396+
}
397+
if (isBucketColumn) {
398+
throw QueryCompilationErrors.cannotAlterCollationBucketColumn(
399+
table.qualifiedName, columnName)
400+
}
401+
}
402+
// Throw an AnalysisException if the column name is changed or we cannot evolve the data type.
403+
// Only changes in collation of column data type or its nested types (recursively) are allowed.
404+
if (!validType || !namesEqual(originColumn, newColumn, resolver)) {
392405
throw QueryCompilationErrors.alterTableChangeColumnNotSupportedForColumnTypeError(
393406
toSQLId(table.identifier.nameParts), originColumn, newColumn, this.origin)
394407
}
395408

396409
val newDataSchema = table.dataSchema.fields.map { field =>
397410
if (field.name == originColumn.name) {
398-
// Create a new column from the origin column with the new comment.
399-
val withNewComment: StructField =
400-
addComment(field, newColumn.getComment())
411+
// Create a new column from the origin column with the new type and new comment.
412+
val withNewTypeAndComment: StructField =
413+
addComment(withNewType(field, newColumn.dataType), newColumn.getComment())
401414
// Create a new column from the origin column with the new current default value.
402415
if (newColumn.getCurrentDefaultValue().isDefined) {
403416
if (newColumn.getCurrentDefaultValue().get.nonEmpty) {
404417
val result: StructField =
405-
addCurrentDefaultValue(withNewComment, newColumn.getCurrentDefaultValue())
418+
addCurrentDefaultValue(withNewTypeAndComment, newColumn.getCurrentDefaultValue())
406419
// Check that the proposed default value parses and analyzes correctly, and that the
407420
// type of the resulting expression is equivalent or coercible to the destination column
408421
// type.
409422
ResolveDefaultColumns.analyze(result, "ALTER TABLE ALTER COLUMN")
410423
result
411424
} else {
412-
withNewComment.clearCurrentDefaultValue()
425+
withNewTypeAndComment.clearCurrentDefaultValue()
413426
}
414427
} else {
415-
withNewComment
428+
withNewTypeAndComment
416429
}
417430
} else {
418431
field
@@ -432,6 +445,10 @@ case class AlterTableChangeColumnCommand(
432445
}.getOrElse(throw QueryCompilationErrors.cannotFindColumnError(name, schema.fieldNames))
433446
}
434447

448+
// Change the dataType of the column.
449+
private def withNewType(column: StructField, dataType: DataType): StructField =
450+
column.copy(dataType = dataType)
451+
435452
// Add the comment to a column, if comment is empty, return the original column.
436453
private def addComment(column: StructField, comment: Option[String]): StructField =
437454
comment.map(column.withComment).getOrElse(column)
@@ -442,10 +459,17 @@ case class AlterTableChangeColumnCommand(
442459
value.map(column.withCurrentDefaultValue).getOrElse(column)
443460

444461
// Compare a [[StructField]] to another, return true if they have the same column
445-
// name(by resolver) and dataType.
446-
private def columnEqual(
462+
// name(by resolver).
463+
private def namesEqual(
447464
field: StructField, other: StructField, resolver: Resolver): Boolean = {
448-
resolver(field.name, other.name) && field.dataType == other.dataType
465+
resolver(field.name, other.name)
466+
}
467+
468+
// Compare dataType of [[StructField]] to another, return true if it is valid to evolve the type
469+
// when altering column. Only changes in collation of data type or its nested types (recursively)
470+
// are allowed.
471+
private def canEvolveType(from: StructField, to: StructField): Boolean = {
472+
DataType.equalsIgnoreCompatibleCollation(from.dataType, to.dataType)
449473
}
450474
}
451475

sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala

Lines changed: 94 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -2437,6 +2437,100 @@ abstract class DDLSuite extends QueryTest with DDLSuiteBase {
24372437
)
24382438
}
24392439
}
2440+
2441+
test("Change column collation") {
2442+
withTable("t1", "t2", "t3", "t4") {
2443+
// Plain `StringType`.
2444+
sql("CREATE TABLE t1(col STRING) USING parquet")
2445+
sql("INSERT INTO t1 VALUES ('a')")
2446+
checkAnswer(sql("SELECT COLLATION(col) FROM t1"), Row("UTF8_BINARY"))
2447+
sql("ALTER TABLE t1 ALTER COLUMN col TYPE STRING COLLATE UTF8_BINARY_LCASE")
2448+
checkAnswer(sql("SELECT COLLATION(col) FROM t1"), Row("UTF8_BINARY_LCASE"))
2449+
2450+
// Invalid "ALTER COLUMN" to Integer.
2451+
val alterInt = "ALTER TABLE t1 ALTER COLUMN col TYPE INTEGER"
2452+
checkError(
2453+
exception = intercept[AnalysisException] {
2454+
sql(alterInt)
2455+
},
2456+
errorClass = "NOT_SUPPORTED_CHANGE_COLUMN",
2457+
parameters = Map(
2458+
"originType" -> "\"STRING COLLATE UTF8_BINARY_LCASE\"",
2459+
"originName" -> "`col`",
2460+
"table" -> "`spark_catalog`.`default`.`t1`",
2461+
"newType" -> "\"INT\"",
2462+
"newName" -> "`col`"
2463+
),
2464+
context = ExpectedContext(fragment = alterInt, start = 0, stop = alterInt.length - 1)
2465+
)
2466+
2467+
// `ArrayType` with collation.
2468+
sql("CREATE TABLE t2(col ARRAY<STRING>) USING parquet")
2469+
sql("INSERT INTO t2 VALUES (ARRAY('a'))")
2470+
checkAnswer(sql("SELECT COLLATION(col[0]) FROM t2"), Row("UTF8_BINARY"))
2471+
sql("ALTER TABLE t2 ALTER COLUMN col TYPE ARRAY<STRING COLLATE UTF8_BINARY_LCASE>")
2472+
checkAnswer(sql("SELECT COLLATION(col[0]) FROM t2"), Row("UTF8_BINARY_LCASE"))
2473+
2474+
// `MapType` with collation.
2475+
sql("CREATE TABLE t3(col MAP<STRING, STRING>) USING parquet")
2476+
sql("INSERT INTO t3 VALUES (MAP('k', 'v'))")
2477+
checkAnswer(sql("SELECT COLLATION(col['k']) FROM t3"), Row("UTF8_BINARY"))
2478+
sql(
2479+
"""
2480+
|ALTER TABLE t3 ALTER COLUMN col TYPE
2481+
|MAP<STRING, STRING COLLATE UTF8_BINARY_LCASE>""".stripMargin)
2482+
checkAnswer(sql("SELECT COLLATION(col['k']) FROM t3"), Row("UTF8_BINARY_LCASE"))
2483+
2484+
// Invalid change of map key collation.
2485+
val alterMap =
2486+
"ALTER TABLE t3 ALTER COLUMN col TYPE " +
2487+
"MAP<STRING COLLATE UTF8_BINARY_LCASE, STRING>"
2488+
checkError(
2489+
exception = intercept[AnalysisException] {
2490+
sql(alterMap)
2491+
},
2492+
errorClass = "NOT_SUPPORTED_CHANGE_COLUMN",
2493+
parameters = Map(
2494+
"originType" -> "\"MAP<STRING, STRING COLLATE UTF8_BINARY_LCASE>\"",
2495+
"originName" -> "`col`",
2496+
"table" -> "`spark_catalog`.`default`.`t3`",
2497+
"newType" -> "\"MAP<STRING COLLATE UTF8_BINARY_LCASE, STRING>\"",
2498+
"newName" -> "`col`"
2499+
),
2500+
context = ExpectedContext(fragment = alterMap, start = 0, stop = alterMap.length - 1)
2501+
)
2502+
2503+
// `StructType` with collation.
2504+
sql("CREATE TABLE t4(col STRUCT<a:STRING>) USING parquet")
2505+
sql("INSERT INTO t4 VALUES (NAMED_STRUCT('a', 'value'))")
2506+
checkAnswer(sql("SELECT COLLATION(col.a) FROM t4"), Row("UTF8_BINARY"))
2507+
sql("ALTER TABLE t4 ALTER COLUMN col TYPE STRUCT<a:STRING COLLATE UTF8_BINARY_LCASE>")
2508+
checkAnswer(sql("SELECT COLLATION(col.a) FROM t4"), Row("UTF8_BINARY_LCASE"))
2509+
}
2510+
}
2511+
2512+
test("Invalid collation change on partition and bucket columns") {
2513+
withTable("t1", "t2") {
2514+
sql("CREATE TABLE t1(col STRING, i INTEGER) USING parquet PARTITIONED BY (col)")
2515+
checkError(
2516+
exception = intercept[AnalysisException] {
2517+
sql("ALTER TABLE t1 ALTER COLUMN col TYPE STRING COLLATE UTF8_BINARY_LCASE")
2518+
},
2519+
errorClass = "CANNOT_ALTER_PARTITION_COLUMN",
2520+
sqlState = "428FR",
2521+
parameters = Map("tableName" -> "`spark_catalog`.`default`.`t1`", "columnName" -> "`col`")
2522+
)
2523+
sql("CREATE TABLE t2(col STRING) USING parquet CLUSTERED BY (col) INTO 1 BUCKETS")
2524+
checkError(
2525+
exception = intercept[AnalysisException] {
2526+
sql("ALTER TABLE t2 ALTER COLUMN col TYPE STRING COLLATE UTF8_BINARY_LCASE")
2527+
},
2528+
errorClass = "CANNOT_ALTER_COLLATION_BUCKET_COLUMN",
2529+
sqlState = "428FR",
2530+
parameters = Map("tableName" -> "`spark_catalog`.`default`.`t2`", "columnName" -> "`col`")
2531+
)
2532+
}
2533+
}
24402534
}
24412535

24422536
object FakeLocalFsFileSystem {

0 commit comments

Comments
 (0)