Skip to content

Commit 6f4e051

Browse files
zedtangallisonport-db
authored andcommitted
[Spark] Support ALTER TABLE CLUSTER BY
This PR adds support for ALTER TABLE CLUSTER BY syntax for clustered tables: * `ALTER TABLE CLUSTER BY (col1, col2, ...)` to change the clustering columns * `ALTER TABLE CLUSTER BY NONE` to remove the clustering columns Closes #2556 GitOrigin-RevId: 7cc2ff2abe6fdd1cba6150648c71f27fc7432be1
1 parent dc574eb commit 6f4e051

File tree

11 files changed

+324
-7
lines changed

11 files changed

+324
-7
lines changed

spark/src/main/antlr4/io/delta/sql/parser/DeltaSqlBase.g4

Lines changed: 4 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -88,6 +88,8 @@ statement
8888
DROP CONSTRAINT (IF EXISTS)? name=identifier #dropTableConstraint
8989
| ALTER TABLE table=qualifiedName
9090
DROP FEATURE featureName=featureNameValue (TRUNCATE HISTORY)? #alterTableDropFeature
91+
| ALTER TABLE table=qualifiedName
92+
(clusterBySpec | CLUSTER BY NONE) #alterTableClusterBy
9193
| OPTIMIZE (path=STRING | table=qualifiedName)
9294
(WHERE partitionPredicate=predicateToken)?
9395
(zorderSpec)? #optimizeTable
@@ -231,7 +233,7 @@ nonReserved
231233
| NO | STATISTICS
232234
| CLONE | SHALLOW
233235
| FEATURE | TRUNCATE
234-
| CLUSTER
236+
| CLUSTER | NONE
235237
;
236238

237239
// Define how the keywords above should appear in a user's SQL statement.
@@ -269,6 +271,7 @@ LIMIT: 'LIMIT';
269271
LOCATION: 'LOCATION';
270272
MINUS: '-';
271273
NO: 'NO';
274+
NONE: 'NONE';
272275
NOT: 'NOT' | '!';
273276
NULL: 'NULL';
274277
OF: 'OF';

spark/src/main/resources/error/delta-error-classes.json

Lines changed: 6 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -71,6 +71,12 @@
7171
],
7272
"sqlState" : "42837"
7373
},
74+
"DELTA_ALTER_TABLE_CLUSTER_BY_NOT_ALLOWED" : {
75+
"message" : [
76+
"ALTER TABLE CLUSTER BY is supported only for Delta table with clustering."
77+
],
78+
"sqlState" : "42000"
79+
},
7480
"DELTA_ALTER_TABLE_SET_CLUSTERING_TABLE_FEATURE_NOT_ALLOWED" : {
7581
"message" : [
7682
"Cannot enable <tableFeature> table feature using ALTER TABLE SET TBLPROPERTIES. Please use CREATE OR REPLACE TABLE CLUSTER BY to create a Delta table with clustering."

spark/src/main/scala/io/delta/sql/parser/DeltaSqlParser.scala

Lines changed: 20 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -43,7 +43,7 @@ import java.util.Locale
4343
import scala.collection.JavaConverters._
4444

4545
import org.apache.spark.sql.catalyst.TimeTravel
46-
import org.apache.spark.sql.delta.skipping.clustering.temp.{ClusterByParserUtils, ClusterByPlan, ClusterBySpec}
46+
import org.apache.spark.sql.delta.skipping.clustering.temp.{AlterTableClusterBy, ClusterByParserUtils, ClusterByPlan, ClusterBySpec}
4747

4848
import org.apache.spark.sql.delta._
4949
import org.apache.spark.sql.delta.commands._
@@ -583,6 +583,25 @@ class DeltaSqlAstBuilder extends DeltaSqlBaseBaseVisitor[AnyRef] {
583583
truncateHistory)
584584
}
585585

586+
/**
587+
* Parse an ALTER TABLE CLUSTER BY command.
588+
*/
589+
override def visitAlterTableClusterBy(ctx: AlterTableClusterByContext): LogicalPlan = {
590+
val table =
591+
createUnresolvedTable(ctx.table.identifier.asScala.map(_.getText).toSeq,
592+
"ALTER TABLE ... CLUSTER BY")
593+
if (ctx.NONE() != null) {
594+
AlterTableClusterBy(table, None)
595+
} else {
596+
assert(ctx.clusterBySpec() != null)
597+
val columnNames =
598+
ctx.clusterBySpec().interleave.asScala
599+
.map(_.identifier.asScala.map(_.getText).toSeq)
600+
.map(_.asInstanceOf[Seq[String]]).toSeq
601+
AlterTableClusterBy(table, Some(ClusterBySpec(columnNames)))
602+
}
603+
}
604+
586605
protected def typedVisit[T](ctx: ParseTree): T = {
587606
ctx.accept(this).asInstanceOf[T]
588607
}

spark/src/main/scala/org/apache/spark/sql/delta/DeltaErrors.scala

Lines changed: 12 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -3195,6 +3195,18 @@ trait DeltaErrorsBase
31953195
messageParameters = Array(s"${zOrderBy.map(_.name).mkString(", ")}"))
31963196
}
31973197

3198+
def alterClusterByNotOnDeltaTableException(): Throwable = {
3199+
new DeltaAnalysisException(
3200+
errorClass = "DELTA_ONLY_OPERATION",
3201+
messageParameters = Array("ALTER TABLE CLUSTER BY"))
3202+
}
3203+
3204+
def alterClusterByNotAllowedException(): Throwable = {
3205+
new DeltaAnalysisException(
3206+
errorClass = "DELTA_ALTER_TABLE_CLUSTER_BY_NOT_ALLOWED",
3207+
messageParameters = Array.empty)
3208+
}
3209+
31983210
def clusteringTablePreviewDisabledException(): Throwable = {
31993211
val msg = s"""
32003212
|A clustered table is currently in preview and is disabled by default. Please set

spark/src/main/scala/org/apache/spark/sql/delta/DeltaOperations.scala

Lines changed: 8 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -552,6 +552,14 @@ object DeltaOperations {
552552
override val operationMetrics: Set[String] = DeltaOperationMetrics.OPTIMIZE
553553
}
554554

555+
/** Recorded when clustering columns are changed on clustered tables. */
556+
case class ClusterBy(
557+
oldClusteringColumns: String,
558+
newClusteringColumns: String) extends Operation("CLUSTER BY") {
559+
override val parameters: Map[String, Any] = Map(
560+
"oldClusteringColumns" -> oldClusteringColumns,
561+
"newClusteringColumns" -> newClusteringColumns)
562+
}
555563

556564
private def structFieldToMap(colPath: Seq[String], field: StructField): Map[String, Any] = {
557565
Map(

spark/src/main/scala/org/apache/spark/sql/delta/catalog/DeltaCatalog.scala

Lines changed: 15 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -25,7 +25,7 @@ import scala.collection.JavaConverters._
2525
import scala.collection.mutable
2626

2727
import org.apache.spark.sql.delta.skipping.clustering.ClusteredTableUtils
28-
import org.apache.spark.sql.delta.skipping.clustering.temp.ClusterBySpec
28+
import org.apache.spark.sql.delta.skipping.clustering.temp.{ClusterBy, ClusterBySpec}
2929
import org.apache.spark.sql.delta.skipping.clustering.temp.{ClusterByTransform => TempClusterByTransform}
3030
import org.apache.spark.sql.delta.{DeltaConfigs, DeltaErrors, DeltaTableUtils}
3131
import org.apache.spark.sql.delta.{DeltaLog, DeltaOptions}
@@ -589,6 +589,8 @@ class DeltaCatalog extends DelegatingCatalogExtension
589589
}
590590
val table = loadTable(ident) match {
591591
case deltaTable: DeltaTableV2 => deltaTable
592+
case _ if changes.exists(_.isInstanceOf[ClusterBy]) =>
593+
throw DeltaErrors.alterClusterByNotOnDeltaTableException()
592594
case _ => return super.alterTable(ident, changes: _*)
593595
}
594596

@@ -755,6 +757,18 @@ class DeltaCatalog extends DelegatingCatalogExtension
755757
AlterTableDropFeatureDeltaCommand(
756758
table, featureName, truncateHistory = truncateHistory).run(spark)
757759

760+
case (t, clusterBy) if t == classOf[ClusterBy] =>
761+
clusterBy.asInstanceOf[Seq[ClusterBy]].foreach { c =>
762+
if (c.clusteringColumns.nonEmpty) {
763+
val clusterBySpec = ClusterBySpec(c.clusteringColumns.toSeq)
764+
validateClusterBySpec(Some(clusterBySpec), table.schema())
765+
}
766+
if (!ClusteredTableUtils.isSupported(table.initialSnapshot.protocol)) {
767+
throw DeltaErrors.alterClusterByNotAllowedException()
768+
}
769+
AlterTableClusterByDeltaCommand(
770+
table, c.clusteringColumns.map(_.fieldNames().toSeq).toSeq).run(spark)
771+
}
758772
}
759773

760774
columnUpdates.foreach { case (fieldNames, (newField, newPositionOpt)) =>

spark/src/main/scala/org/apache/spark/sql/delta/commands/alterDeltaTableCommands.scala

Lines changed: 57 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -22,6 +22,7 @@ import java.util.concurrent.TimeUnit
2222

2323
import scala.util.control.NonFatal
2424

25+
import org.apache.spark.sql.delta.skipping.clustering.ClusteredTableUtils
2526
import org.apache.spark.sql.delta.skipping.clustering.ClusteringColumnInfo
2627
import org.apache.spark.sql.delta._
2728
import org.apache.spark.sql.delta.actions.Protocol
@@ -42,6 +43,7 @@ import org.apache.spark.sql.catalyst.plans.logical.{IgnoreCachedData, QualifiedC
4243
import org.apache.spark.sql.catalyst.util.CharVarcharUtils
4344
import org.apache.spark.sql.connector.catalog.TableCatalog
4445
import org.apache.spark.sql.connector.catalog.TableChange.{After, ColumnPosition, First}
46+
import org.apache.spark.sql.connector.expressions.FieldReference
4547
import org.apache.spark.sql.execution.command.LeafRunnableCommand
4648
import org.apache.spark.sql.types._
4749

@@ -930,3 +932,58 @@ case class AlterTableDropConstraintDeltaCommand(
930932
}
931933
}
932934

935+
/**
936+
* Command for altering clustering columns for clustered tables.
937+
* - ALTER TABLE .. CLUSTER BY (col1, col2, ...)
938+
* - ALTER TABLE .. CLUSTER BY NONE
939+
*
940+
* Note that the given `clusteringColumns` are empty when CLUSTER BY NONE is specified.
941+
* Also, `clusteringColumns` are validated (e.g., duplication / existence check) in
942+
* DeltaCatalog.alterTable().
943+
*/
944+
case class AlterTableClusterByDeltaCommand(
945+
table: DeltaTableV2,
946+
clusteringColumns: Seq[Seq[String]])
947+
extends LeafRunnableCommand with AlterDeltaTableCommand with IgnoreCachedData {
948+
override def run(sparkSession: SparkSession): Seq[Row] = {
949+
val deltaLog = table.deltaLog
950+
ClusteredTableUtils.validateNumClusteringColumns(clusteringColumns, Some(deltaLog))
951+
recordDeltaOperation(deltaLog, "delta.ddl.alter.clusterBy") {
952+
val txn = startTransaction()
953+
954+
val clusteringColsLogicalNames = ClusteringColumnInfo.extractLogicalNames(txn.snapshot)
955+
val oldLogicalClusteringColumnsString = clusteringColsLogicalNames.mkString(",")
956+
val oldColumnsCount = clusteringColsLogicalNames.size
957+
958+
val newLogicalClusteringColumns = clusteringColumns.map(FieldReference(_).toString)
959+
ClusteredTableUtils.validateClusteringColumnsInStatsSchema(
960+
txn.snapshot, newLogicalClusteringColumns)
961+
962+
val newDomainMetadata =
963+
ClusteredTableUtils
964+
.getClusteringDomainMetadataForAlterTableClusterBy(newLogicalClusteringColumns, txn)
965+
966+
recordDeltaEvent(
967+
deltaLog,
968+
"delta.ddl.alter.clusterBy",
969+
data = Map(
970+
"isNewClusteredTable" -> !ClusteredTableUtils.isSupported(txn.protocol),
971+
"oldColumnsCount" -> oldColumnsCount, "newColumnsCount" -> clusteringColumns.size))
972+
// Add clustered table properties if the current table is not clustered.
973+
// [[DeltaCatalog.alterTable]] already ensures that the table is not partitioned.
974+
if (!ClusteredTableUtils.isSupported(txn.protocol)) {
975+
txn.updateMetadata(
976+
txn.metadata.copy(
977+
configuration = txn.metadata.configuration ++
978+
ClusteredTableUtils.getTableFeatureProperties(txn.metadata.configuration)
979+
))
980+
}
981+
txn.commit(
982+
newDomainMetadata,
983+
DeltaOperations.ClusterBy(
984+
oldLogicalClusteringColumnsString,
985+
newLogicalClusteringColumns.mkString(",")))
986+
}
987+
Seq.empty[Row]
988+
}
989+
}

spark/src/main/scala/org/apache/spark/sql/delta/skipping/clustering/ClusteredTableUtils.scala

Lines changed: 14 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -205,6 +205,20 @@ trait ClusteredTableUtilsBase extends DeltaLogging {
205205
ClusteringMetadataDomain.fromSnapshot(snapshot).map(_.toDomainMetadata).toSeq
206206
}
207207

208+
/**
209+
* Create new clustering [[DomainMetadata]] actions given updated column names for
210+
* 'ALTER TABLE ... CLUSTER BY'.
211+
*/
212+
def getClusteringDomainMetadataForAlterTableClusterBy(
213+
newLogicalClusteringColumns: Seq[String],
214+
txn: OptimisticTransaction): Seq[DomainMetadata] = {
215+
val newClusteringColumns =
216+
newLogicalClusteringColumns.map(ClusteringColumn(txn.metadata.schema, _))
217+
val clusteringMetadataDomainOpt =
218+
Some(ClusteringMetadataDomain.fromClusteringColumns(newClusteringColumns).toDomainMetadata)
219+
clusteringMetadataDomainOpt.toSeq
220+
}
221+
208222
/**
209223
* Validate stats will be collected for all clustering columns.
210224
*/
Lines changed: 39 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,39 @@
1+
/*
2+
* Copyright (2021) The Delta Lake Project Authors.
3+
*
4+
* Licensed under the Apache License, Version 2.0 (the "License");
5+
* you may not use this file except in compliance with the License.
6+
* You may obtain a copy of the License at
7+
*
8+
* http://www.apache.org/licenses/LICENSE-2.0
9+
*
10+
* Unless required by applicable law or agreed to in writing, software
11+
* distributed under the License is distributed on an "AS IS" BASIS,
12+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13+
* See the License for the specific language governing permissions and
14+
* limitations under the License.
15+
*/
16+
17+
package org.apache.spark.sql.delta.skipping.clustering.temp
18+
19+
import org.apache.spark.sql.catalyst.plans.logical.{AlterTableCommand, LogicalPlan}
20+
import org.apache.spark.sql.connector.catalog.TableChange
21+
import org.apache.spark.sql.connector.expressions.NamedReference
22+
23+
/**
24+
* The logical plan of the following commands:
25+
* - ALTER TABLE ... CLUSTER BY (col1, col2, ...)
26+
* - ALTER TABLE ... CLUSTER BY NONE
27+
*/
28+
case class AlterTableClusterBy(
29+
table: LogicalPlan, clusterBySpec: Option[ClusterBySpec]) extends AlterTableCommand {
30+
override def changes: Seq[TableChange] =
31+
Seq(ClusterBy(clusterBySpec
32+
.map(_.columnNames) // CLUSTER BY (col1, col2, ...)
33+
.getOrElse(Seq.empty))) // CLUSTER BY NONE
34+
35+
protected def withNewChildInternal(newChild: LogicalPlan): LogicalPlan = copy(table = newChild)
36+
}
37+
38+
/** A TableChange to alter clustering columns for a table. */
39+
case class ClusterBy(clusteringColumns: Seq[NamedReference]) extends TableChange {}

spark/src/test/scala/org/apache/spark/sql/delta/clustering/ClusteringTableFeatureSuite.scala

Lines changed: 21 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -48,4 +48,25 @@ class ClusteringTableFeatureSuite extends SparkFunSuite with DeltaSQLCommandTest
4848
parameters = Map("tableFeature" -> "clustering"))
4949
}
5050
}
51+
52+
test("alter table cluster by non-clustered tables is not allowed.") {
53+
withTable("tbl") {
54+
sql("CREATE TABLE tbl(a INT, b STRING) USING DELTA")
55+
val e1 = intercept[DeltaAnalysisException] {
56+
sql("ALTER TABLE tbl CLUSTER BY (a)")
57+
}
58+
checkError(
59+
e1,
60+
"DELTA_ALTER_TABLE_CLUSTER_BY_NOT_ALLOWED",
61+
parameters = Map.empty)
62+
63+
val e2 = intercept[DeltaAnalysisException] {
64+
sql("ALTER TABLE tbl CLUSTER BY NONE")
65+
}
66+
checkError(
67+
e2,
68+
"DELTA_ALTER_TABLE_CLUSTER_BY_NOT_ALLOWED",
69+
parameters = Map.empty)
70+
}
71+
}
5172
}

0 commit comments

Comments
 (0)