From f73411bb1fec5059a1c0e280403d75a1cb11f0c4 Mon Sep 17 00:00:00 2001 From: Gengliang Wang <gengliang@apache.org> Date: Mon, 14 Apr 2025 13:38:06 -0700 Subject: [PATCH 01/19] add asConstraint --- .../catalyst/expressions/constraints.scala | 65 +++++++++++++++++++ 1 file changed, 65 insertions(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/constraints.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/constraints.scala index 1a52723669358..1e70aee00b2e3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/constraints.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/constraints.scala @@ -20,9 +20,14 @@ import java.util.UUID import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.catalyst.trees.CurrentOrigin +import org.apache.spark.sql.catalyst.util.V2ExpressionBuilder +import org.apache.spark.sql.connector.catalog.constraints.Constraint +import org.apache.spark.sql.connector.expressions.FieldReference import org.apache.spark.sql.types.{DataType, StringType} trait TableConstraint { + // Convert to a data source v2 constraint + def asConstraint(isCreateTable: Boolean): Constraint /** Returns the user-provided name of the constraint */ def userProvidedName: String @@ -112,6 +117,27 @@ case class CheckConstraint( with TableConstraint { // scalastyle:on line.size.limit + def asConstraint(isCreateTable: Boolean): Constraint = { + val predicate = new V2ExpressionBuilder(child, true).buildPredicate().orNull + val enforced = userProvidedCharacteristic.enforced.getOrElse(true) + val rely = userProvidedCharacteristic.rely.getOrElse(false) + // The validation status is set to UNVALIDATED for create table and + // VALID for alter table. + val validateStatus = if (isCreateTable) { + Constraint.ValidationStatus.UNVALIDATED + } else { + Constraint.ValidationStatus.VALID + } + Constraint + .check(name) + .predicateSql(condition) + .predicate(predicate) + .rely(rely) + .enforced(enforced) + .validationStatus(validateStatus) + .build() + } + override protected def withNewChildInternal(newChild: Expression): Expression = copy(child = newChild) @@ -140,6 +166,18 @@ case class PrimaryKeyConstraint( extends TableConstraint { // scalastyle:on line.size.limit + + override def asConstraint(isCreateTable: Boolean): Constraint = { + val enforced = userProvidedCharacteristic.enforced.getOrElse(false) + val rely = userProvidedCharacteristic.rely.getOrElse(false) + Constraint + .primaryKey(name, columns.map(FieldReference.column).toArray) + .rely(rely) + .enforced(enforced) + .validationStatus(Constraint.ValidationStatus.UNVALIDATED) + .build() + } + override protected def generateName(tableName: String): String = s"${tableName}_pk" override def withUserProvidedName(name: String): TableConstraint = copy(userProvidedName = name) @@ -161,6 +199,17 @@ case class UniqueConstraint( extends TableConstraint { // scalastyle:on line.size.limit + override def asConstraint(isCreateTable: Boolean): Constraint = { + val enforced = userProvidedCharacteristic.enforced.getOrElse(false) + val rely = userProvidedCharacteristic.rely.getOrElse(false) + Constraint + .unique(name, columns.map(FieldReference.column).toArray) + .rely(rely) + .enforced(enforced) + .validationStatus(Constraint.ValidationStatus.UNVALIDATED) + .build() + } + override protected def generateName(tableName: String): String = { s"${tableName}_uniq_$randomSuffix" } @@ -186,6 +235,22 @@ case class ForeignKeyConstraint( extends TableConstraint { // scalastyle:on line.size.limit + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ + + override def asConstraint(isCreateTable: Boolean): Constraint = { + val enforced = userProvidedCharacteristic.enforced.getOrElse(false) + val rely = userProvidedCharacteristic.rely.getOrElse(false) + Constraint + .foreignKey(name, + childColumns.map(FieldReference.column).toArray, + parentTableId.asIdentifier, + parentColumns.map(FieldReference.column).toArray) + .rely(rely) + .enforced(enforced) + .validationStatus(Constraint.ValidationStatus.UNVALIDATED) + .build() + } + override protected def generateName(tableName: String): String = s"${tableName}_${parentTableId.last}_fk_$randomSuffix" From 259de548f51386b1d6da8d1c204a4b15f5e79589 Mon Sep 17 00:00:00 2001 From: Gengliang Wang <gengliang@apache.org> Date: Mon, 14 Apr 2025 14:29:23 -0700 Subject: [PATCH 02/19] support create/replace table with constraint --- .../catalyst/analysis/ResolveTableSpec.scala | 58 +++++++++++++++++-- .../datasources/v2/CreateTableExec.scala | 1 + .../datasources/v2/ReplaceTableExec.scala | 1 + 3 files changed, 54 insertions(+), 6 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveTableSpec.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveTableSpec.scala index 05158fbee3de6..f3bffcae0e463 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveTableSpec.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveTableSpec.scala @@ -18,11 +18,12 @@ package org.apache.spark.sql.catalyst.analysis import org.apache.spark.SparkThrowable -import org.apache.spark.sql.catalyst.expressions.{Expression, Literal} +import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.optimizer.ComputeCurrentTime import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.trees.TreePattern.COMMAND +import org.apache.spark.sql.catalyst.util.ResolveDefaultColumns.DefaultColumnAnalyzer import org.apache.spark.sql.errors.QueryCompilationErrors import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{ArrayType, MapType, StructType} @@ -46,20 +47,55 @@ object ResolveTableSpec extends Rule[LogicalPlan] { preparedPlan.resolveOperatorsWithPruning(_.containsAnyPattern(COMMAND), ruleId) { case t: CreateTable => - resolveTableSpec(t, t.tableSpec, s => t.copy(tableSpec = s)) + resolveTableSpec(t, t.tableSpec, + fakeRelationFromColumns(t.columns), s => t.copy(tableSpec = s)) case t: CreateTableAsSelect => - resolveTableSpec(t, t.tableSpec, s => t.copy(tableSpec = s)) + resolveTableSpec(t, t.tableSpec, None, s => t.copy(tableSpec = s)) case t: ReplaceTable => - resolveTableSpec(t, t.tableSpec, s => t.copy(tableSpec = s)) + resolveTableSpec(t, t.tableSpec, + fakeRelationFromColumns(t.columns), s => t.copy(tableSpec = s)) case t: ReplaceTableAsSelect => - resolveTableSpec(t, t.tableSpec, s => t.copy(tableSpec = s)) + resolveTableSpec(t, t.tableSpec, None, s => t.copy(tableSpec = s)) } } + private def fakeRelationFromColumns(columns: Seq[ColumnDefinition]): Option[LogicalPlan] = { + val attributeList = columns.map { col => + AttributeReference(col.name, col.dataType)() + } + Some(LocalRelation(attributeList)) + } + + private def analyzeConstraints( + constraints: Seq[TableConstraint], + fakeRelation: LogicalPlan): Seq[TableConstraint] = { + val analyzedExpressions = constraints.map { + case c: CheckConstraint => + val alias = Alias(c.child, c.name)() + val project = Project(Seq(alias), fakeRelation) + val analyzed = DefaultColumnAnalyzer.execute(project) + DefaultColumnAnalyzer.checkAnalysis0(analyzed) + + val analyzedExpression = (analyzed collectFirst { + case Project(Seq(Alias(e: Expression, _)), _) => e + }).get + if (!analyzedExpression.deterministic) { + analyzedExpression.failAnalysis( + errorClass = "INVALID_CHECK_CONSTRAINT.NONDETERMINISTIC", + messageParameters = Map.empty + ) + } + c.withNewChildren(Seq(analyzedExpression)).asInstanceOf[CheckConstraint] + case other => other + } + analyzedExpressions + } + /** Helper method to resolve the table specification within a logical plan. */ private def resolveTableSpec( input: LogicalPlan, tableSpec: TableSpecBase, + fakeRelation: Option[LogicalPlan], withNewSpec: TableSpecBase => LogicalPlan): LogicalPlan = tableSpec match { case u: UnresolvedTableSpec if u.optionExpression.resolved => val newOptions: Seq[(String, String)] = u.optionExpression.options.map { @@ -86,6 +122,15 @@ object ResolveTableSpec extends Rule[LogicalPlan] { } (key, newValue) } + // Analyze constraints for the table spec: + // If a fake relation is provided (for CREATE/REPLACE TABLE with column definitions), + // analyze the constraints using the fake relation as context. + // Otherwise, use the original constraints unmodified + val newConstraints = if (fakeRelation.isDefined) { + analyzeConstraints(u.constraints, fakeRelation.get) + } else { + u.constraints + } val newTableSpec = TableSpec( properties = u.properties, provider = u.provider, @@ -94,7 +139,8 @@ object ResolveTableSpec extends Rule[LogicalPlan] { comment = u.comment, collation = u.collation, serde = u.serde, - external = u.external) + external = u.external, + constraints = newConstraints.map(_.asConstraint(isCreateTable = true))) withNewSpec(newTableSpec) case _ => input diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateTableExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateTableExec.scala index f0812245bcce0..3298a56d95992 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateTableExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateTableExec.scala @@ -47,6 +47,7 @@ case class CreateTableExec( .withColumns(columns) .withPartitions(partitioning.toArray) .withProperties(tableProperties.asJava) + .withConstraints(tableSpec.constraints.toArray) .build() catalog.createTable(identifier, tableInfo) } catch { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ReplaceTableExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ReplaceTableExec.scala index 51f5c848bd27b..a819f836a1dd6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ReplaceTableExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ReplaceTableExec.scala @@ -52,6 +52,7 @@ case class ReplaceTableExec( .withColumns(columns) .withPartitions(partitioning.toArray) .withProperties(tableProperties.asJava) + .withConstraints(tableSpec.constraints.toArray) .build() catalog.createTable(ident, tableInfo) Seq.empty From 3ab17fcd56249f282817baf762c00404f4f18628 Mon Sep 17 00:00:00 2001 From: Gengliang Wang <gengliang@apache.org> Date: Mon, 14 Apr 2025 15:13:17 -0700 Subject: [PATCH 03/19] implment alter table commands --- .../plans/logical/v2AlterTableCommands.scala | 16 +++++++++++++--- 1 file changed, 13 insertions(+), 3 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2AlterTableCommands.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2AlterTableCommands.scala index 2b2f9df6abf52..5c61f4721ef73 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2AlterTableCommands.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2AlterTableCommands.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.plans.logical -import org.apache.spark.sql.catalyst.analysis.{FieldName, FieldPosition, ResolvedFieldName, UnresolvedException} +import org.apache.spark.sql.catalyst.analysis.{FieldName, FieldPosition, ResolvedFieldName, ResolvedTable, UnresolvedException} import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.catalyst.catalog.ClusterBySpec import org.apache.spark.sql.catalyst.expressions.{Expression, TableConstraint, Unevaluable} @@ -295,7 +295,16 @@ case class AlterTableCollation( case class AddConstraint( table: LogicalPlan, tableConstraint: TableConstraint) extends AlterTableCommand { - override def changes: Seq[TableChange] = Seq.empty + override def changes: Seq[TableChange] = { + val constraint = tableConstraint.asConstraint(isCreateTable = false) + val validatedTableVersion = table match { + case t: ResolvedTable if constraint.enforced() => + t.table.currentVersion() + case _ => + null + } + Seq(TableChange.addConstraint(constraint, validatedTableVersion)) + } protected def withNewChildInternal(newChild: LogicalPlan): LogicalPlan = copy(table = newChild) } @@ -308,7 +317,8 @@ case class DropConstraint( name: String, ifExists: Boolean, cascade: Boolean) extends AlterTableCommand { - override def changes: Seq[TableChange] = Seq.empty + override def changes: Seq[TableChange] = + Seq(TableChange.dropConstraint(name, ifExists, cascade)) protected def withNewChildInternal(newChild: LogicalPlan): LogicalPlan = copy(table = newChild) } From 2fac01789e90dd74557b2dba65ea08c699e2bebe Mon Sep 17 00:00:00 2001 From: Gengliang Wang <gengliang@apache.org> Date: Mon, 14 Apr 2025 15:21:09 -0700 Subject: [PATCH 04/19] add tests --- .../command/v2/CheckConstraintSuite.scala | 206 ++++++++++++++++++ .../command/v2/CommandSuiteBase.scala | 9 +- .../command/v2/DropConstraintSuite.scala | 93 ++++++++ .../v2/ForeignKeyConstraintSuite.scala | 112 ++++++++++ .../v2/PrimaryKeyConstraintSuite.scala | 100 +++++++++ .../command/v2/UniqueConstraintSuite.scala | 100 +++++++++ 6 files changed, 619 insertions(+), 1 deletion(-) create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/CheckConstraintSuite.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/DropConstraintSuite.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ForeignKeyConstraintSuite.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/PrimaryKeyConstraintSuite.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/UniqueConstraintSuite.scala diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/CheckConstraintSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/CheckConstraintSuite.scala new file mode 100644 index 0000000000000..5d64281ff58c8 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/CheckConstraintSuite.scala @@ -0,0 +1,206 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.command.v2 + +import org.apache.spark.sql.{AnalysisException, QueryTest} +import org.apache.spark.sql.connector.catalog.Table +import org.apache.spark.sql.connector.catalog.constraints.Check +import org.apache.spark.sql.execution.command.DDLCommandTestUtils + +class CheckConstraintSuite extends QueryTest with CommandSuiteBase with DDLCommandTestUtils { + override protected def command: String = "ALTER TABLE .. ADD CONSTRAINT" + + test("Nondeterministic expression -- alter table") { + withTable("t") { + sql("create table t(i double)") + val query = + """ + |ALTER TABLE t ADD CONSTRAINT c1 CHECK (i > rand(0)) + |""".stripMargin + val error = intercept[AnalysisException] { + sql(query) + } + checkError( + exception = error, + condition = "INVALID_CHECK_CONSTRAINT.NONDETERMINISTIC", + sqlState = "42621", + parameters = Map.empty, + context = ExpectedContext( + fragment = "i > rand(0)", + start = 40, + stop = 50 + ) + ) + } + } + + test("Nondeterministic expression -- create table") { + Seq( + "create table t(i double check (i > rand(0)))", + "create table t(i double, constraint c1 check (i > rand(0)))", + "replace table t(i double check (i > rand(0)))", + "replace table t(i double, constraint c1 check (i > rand(0)))" + ).foreach { query => + withTable("t") { + val error = intercept[AnalysisException] { + sql(query) + } + checkError( + exception = error, + condition = "INVALID_CHECK_CONSTRAINT.NONDETERMINISTIC", + sqlState = "42621", + parameters = Map.empty, + context = ExpectedContext( + fragment = "i > rand(0)" + ) + ) + } + } + } + + test("Expression referring a column of another table -- alter table") { + withTable("t", "t2") { + sql("create table t(i double) using parquet") + sql("create table t2(j string) using parquet") + val query = + """ + |ALTER TABLE t ADD CONSTRAINT c1 CHECK (len(t2.j) > 0) + |""".stripMargin + val error = intercept[AnalysisException] { + sql(query) + } + checkError( + exception = error, + condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + sqlState = "42703", + parameters = Map("objectName" -> "`t2`.`j`", "proposal" -> "`t`.`i`"), + context = ExpectedContext( + fragment = "t2.j", + start = 44, + stop = 47 + ) + ) + } + } + + test("Expression referring a column of another table -- create and replace table") { + withTable("t", "t2") { + sql("create table t(i double) using parquet") + val query = "create table t2(j string check(t.i > 0)) using parquet" + val error = intercept[AnalysisException] { + sql(query) + } + checkError( + exception = error, + condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + sqlState = "42703", + parameters = Map("objectName" -> "`t`.`i`", "proposal" -> "`j`"), + context = ExpectedContext( + fragment = "t.i" + ) + ) + } + } + + private def getCheckConstraint(table: Table): Check = { + assert(table.constraints.length == 1) + assert(table.constraints.head.isInstanceOf[Check]) + table.constraints.head.asInstanceOf[Check] + table.constraints.head.asInstanceOf[Check] + } + + test("Predicate should be null if it can't be converted to V2 predicate") { + withNamespaceAndTable("ns", "tbl", catalog) { t => + sql(s"CREATE TABLE $t (id bigint, j string) $defaultUsing") + sql(s"ALTER TABLE $t ADD CONSTRAINT c1 CHECK (from_json(j, 'a INT').a > 1)") + val table = loadTable(catalog, "ns", "tbl") + val constraint = getCheckConstraint(table) + assert(constraint.name() == "c1") + assert(constraint.toDDL == + "CONSTRAINT c1 CHECK from_json(j, 'a INT').a > 1 ENFORCED VALID NORELY") + assert(constraint.predicateSql() == "from_json(j, 'a INT').a > 1") + assert(constraint.predicate() == null) + } + } + + def getConstraintCharacteristics(isCreateTable: Boolean): Seq[(String, String)] = { + val validStatus = if (isCreateTable) "UNVALIDATED" else "VALID" + Seq( + ("", s"ENFORCED $validStatus NORELY"), + ("NOT ENFORCED", s"NOT ENFORCED $validStatus NORELY"), + ("NOT ENFORCED NORELY", s"NOT ENFORCED $validStatus NORELY"), + ("NORELY NOT ENFORCED", s"NOT ENFORCED $validStatus NORELY"), + ("NORELY", s"ENFORCED $validStatus NORELY"), + ("NOT ENFORCED RELY", s"NOT ENFORCED $validStatus RELY"), + ("RELY NOT ENFORCED", s"NOT ENFORCED $validStatus RELY"), + ("NOT ENFORCED RELY", s"NOT ENFORCED $validStatus RELY"), + ("RELY NOT ENFORCED", s"NOT ENFORCED $validStatus RELY"), + ("RELY", s"ENFORCED $validStatus RELY") + ) + } + + test("Create table with check constraint") { + getConstraintCharacteristics(true).foreach { case (characteristic, expectedDDL) => + withNamespaceAndTable("ns", "tbl", catalog) { t => + val constraintStr = s"CONSTRAINT c1 CHECK (id > 0) $characteristic" + sql(s"CREATE TABLE $t (id bigint, data string, $constraintStr) $defaultUsing") + val table = loadTable(catalog, "ns", "tbl") + val constraint = getCheckConstraint(table) + assert(constraint.name() == "c1") + assert(constraint.toDDL == s"CONSTRAINT c1 CHECK id > 0 $expectedDDL") + } + } + } + + test("Alter table add check constraint") { + getConstraintCharacteristics(false).foreach { case (characteristic, expectedDDL) => + withNamespaceAndTable("ns", "tbl", catalog) { t => + sql(s"CREATE TABLE $t (id bigint, data string) $defaultUsing") + assert(loadTable(catalog, "ns", "tbl").constraints.isEmpty) + + sql(s"ALTER TABLE $t ADD CONSTRAINT c1 CHECK (id > 0) $characteristic") + val table = loadTable(catalog, "ns", "tbl") + val constraint = getCheckConstraint(table) + assert(constraint.name() == "c1") + assert(constraint.toDDL == s"CONSTRAINT c1 CHECK id > 0 $expectedDDL") + } + } + } + + test("Add duplicated check constraint") { + withNamespaceAndTable("ns", "tbl", catalog) { t => + sql(s"CREATE TABLE $t (id bigint, data string) $defaultUsing") + assert(loadTable(catalog, "ns", "tbl").constraints.isEmpty) + + sql(s"ALTER TABLE $t ADD CONSTRAINT abc CHECK (id > 0)") + // Constraint names are case-insensitive + Seq("abc", "ABC").foreach { name => + val error = intercept[AnalysisException] { + sql(s"ALTER TABLE $t ADD CONSTRAINT $name CHECK (id>0)") + } + checkError( + exception = error, + condition = "CONSTRAINT_ALREADY_EXISTS", + sqlState = "42710", + parameters = Map("constraintName" -> "abc", + "oldConstraint" -> "CONSTRAINT abc CHECK id > 0 ENFORCED VALID NORELY") + ) + } + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/CommandSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/CommandSuiteBase.scala index 6ba60e245f9b4..15b95d35ff241 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/CommandSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/CommandSuiteBase.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.execution.command.v2 import org.apache.spark.SparkConf import org.apache.spark.sql.catalyst.analysis.ResolvePartitionSpec import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec -import org.apache.spark.sql.connector.catalog.{CatalogV2Implicits, Identifier, InMemoryCatalog, InMemoryPartitionTable, InMemoryPartitionTableCatalog, InMemoryTableCatalog} +import org.apache.spark.sql.connector.catalog._ import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.util.ArrayImplicits._ @@ -64,4 +64,11 @@ trait CommandSuiteBase extends SharedSparkSession { assert(partMetadata.containsKey("location")) assert(partMetadata.get("location") === expected) } + + def loadTable(catalog: String, schema : String, table: String): InMemoryTable = { + import CatalogV2Implicits._ + val catalogPlugin = spark.sessionState.catalogManager.catalog(catalog) + catalogPlugin.asTableCatalog.loadTable(Identifier.of(Array(schema), table)) + .asInstanceOf[InMemoryTable] + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/DropConstraintSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/DropConstraintSuite.scala new file mode 100644 index 0000000000000..f492e18a6e529 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/DropConstraintSuite.scala @@ -0,0 +1,93 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.command.v2 + +import org.apache.spark.sql.{AnalysisException, QueryTest} +import org.apache.spark.sql.connector.catalog.constraints.Check +import org.apache.spark.sql.execution.command.DDLCommandTestUtils + +class DropConstraintSuite extends QueryTest with CommandSuiteBase with DDLCommandTestUtils { + override protected def command: String = "ALTER TABLE .. DROP CONSTRAINT" + + test("Drop a non-exist constraint") { + withNamespaceAndTable("ns", "tbl", catalog) { t => + sql(s"CREATE TABLE $t (id bigint, data string) $defaultUsing") + val e = intercept[AnalysisException] { + sql(s"ALTER TABLE $t DROP CONSTRAINT c1") + } + checkError( + exception = e, + condition = "CONSTRAINT_DOES_NOT_EXIST", + sqlState = "42704", + parameters = Map("constraintName" -> "c1", "tableName" -> "test_catalog.ns.tbl") + ) + } + } + + test("Drop a non-exist constraint if exists") { + withNamespaceAndTable("ns", "tbl", catalog) { t => + sql(s"CREATE TABLE $t (id bigint, data string) $defaultUsing") + sql(s"ALTER TABLE $t DROP CONSTRAINT IF EXISTS c1") + } + } + + test("Drop a constraint on a non-exist table") { + Seq("", "IF EXISTS").foreach { ifExists => + val e = intercept[AnalysisException] { + sql(s"ALTER TABLE test_catalog.ns.tbl DROP CONSTRAINT $ifExists c1") + } + checkError( + exception = e, + condition = "TABLE_OR_VIEW_NOT_FOUND", + sqlState = "42P01", + parameters = Map("relationName" -> "`test_catalog`.`ns`.`tbl`"), + context = ExpectedContext( + fragment = "test_catalog.ns.tbl", + start = 12, + stop = 30 + ) + ) + } + } + + test("Drop existing constraints") { + Seq("", "IF EXISTS").foreach { ifExists => + withNamespaceAndTable("ns", "tbl", catalog) { t => + sql(s"CREATE TABLE $t (id bigint, data string) $defaultUsing") + sql(s"ALTER TABLE $t ADD CONSTRAINT c1 CHECK (id > 0)") + sql(s"ALTER TABLE $t ADD CONSTRAINT c2 CHECK (len(data) > 0)") + sql(s"ALTER TABLE $t DROP CONSTRAINT $ifExists c1") + val table = loadTable(catalog, "ns", "tbl") + assert(table.constraints.length == 1) + val constraint = table.constraints.head.asInstanceOf[Check] + assert(constraint.name() == "c2") + + sql(s"ALTER TABLE $t DROP CONSTRAINT $ifExists c2") + val table2 = loadTable(catalog, "ns", "tbl") + assert(table2.constraints.length == 0) + } + } + } + + test("Drop constraint is case insensitive") { + withNamespaceAndTable("ns", "tbl", catalog) { t => + sql(s"CREATE TABLE $t (id bigint, data string) $defaultUsing") + sql(s"ALTER TABLE $t ADD CONSTRAINT abc CHECK (id > 0)") + sql(s"ALTER TABLE $t DROP CONSTRAINT aBC") + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ForeignKeyConstraintSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ForeignKeyConstraintSuite.scala new file mode 100644 index 0000000000000..0a309e652a75b --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ForeignKeyConstraintSuite.scala @@ -0,0 +1,112 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.command.v2 + +import org.apache.spark.sql.{AnalysisException, QueryTest} +import org.apache.spark.sql.execution.command.DDLCommandTestUtils + +class ForeignKeyConstraintSuite extends QueryTest with CommandSuiteBase with DDLCommandTestUtils { + override protected def command: String = "ALTER TABLE .. ADD CONSTRAINT" + + private val validConstraintCharacteristics = Seq( + ("", "NOT ENFORCED UNVALIDATED NORELY"), + ("NOT ENFORCED", "NOT ENFORCED UNVALIDATED NORELY"), + ("NOT ENFORCED NORELY", "NOT ENFORCED UNVALIDATED NORELY"), + ("NORELY NOT ENFORCED", "NOT ENFORCED UNVALIDATED NORELY"), + ("NORELY", "NOT ENFORCED UNVALIDATED NORELY"), + ("RELY", "NOT ENFORCED UNVALIDATED RELY") + ) + + test("Add foreign key constraint") { + validConstraintCharacteristics.foreach { case (characteristic, expectedDDL) => + withNamespaceAndTable("ns", "tbl", catalog) { t => + sql(s"CREATE TABLE $t (id bigint, fk bigint, data string) $defaultUsing") + sql(s"CREATE TABLE ${t}_ref (id bigint, data string) $defaultUsing") + assert(loadTable(catalog, "ns", "tbl").constraints.isEmpty) + + sql(s"ALTER TABLE $t ADD CONSTRAINT fk1 FOREIGN KEY (fk) " + + s"REFERENCES ${t}_ref(id) $characteristic") + val table = loadTable(catalog, "ns", "tbl") + assert(table.constraints.length == 1) + val constraint = table.constraints.head + assert(constraint.name() == "fk1") + assert(constraint.toDDL == s"CONSTRAINT fk1 FOREIGN KEY (fk) " + + s"REFERENCES test_catalog.ns.tbl_ref (id) $expectedDDL") + } + } + } + + test("Create table with foreign key constraint") { + validConstraintCharacteristics.foreach { case (characteristic, expectedDDL) => + withNamespaceAndTable("ns", "tbl", catalog) { t => + sql(s"CREATE TABLE ${t}_ref (id bigint, data string) $defaultUsing") + val constraintStr = s"CONSTRAINT fk1 FOREIGN KEY (fk) " + + s"REFERENCES ${t}_ref(id) $characteristic" + sql(s"CREATE TABLE $t (id bigint, fk bigint, data string, $constraintStr) $defaultUsing") + val table = loadTable(catalog, "ns", "tbl") + assert(table.constraints.length == 1) + val constraint = table.constraints.head + assert(constraint.name() == "fk1") + assert(constraint.toDDL == s"CONSTRAINT fk1 FOREIGN KEY (fk) " + + s"REFERENCES test_catalog.ns.tbl_ref (id) $expectedDDL") + } + } + } + + test("Add duplicated foreign key constraint") { + withNamespaceAndTable("ns", "tbl", catalog) { t => + sql(s"CREATE TABLE $t (id bigint, fk bigint, data string) $defaultUsing") + sql(s"CREATE TABLE ${t}_ref (id bigint, data string) $defaultUsing") + assert(loadTable(catalog, "ns", "tbl").constraints.isEmpty) + + sql(s"ALTER TABLE $t ADD CONSTRAINT fk1 FOREIGN KEY (fk) REFERENCES ${t}_ref(id)") + // Constraint names are case-insensitive + Seq("fk1", "FK1").foreach { name => + val error = intercept[AnalysisException] { + sql(s"ALTER TABLE $t ADD CONSTRAINT $name FOREIGN KEY (fk) REFERENCES ${t}_ref(id)") + } + checkError( + exception = error, + condition = "CONSTRAINT_ALREADY_EXISTS", + sqlState = "42710", + parameters = Map("constraintName" -> "fk1", + "oldConstraint" -> + ("CONSTRAINT fk1 FOREIGN KEY (fk) " + + "REFERENCES test_catalog.ns.tbl_ref (id) NOT ENFORCED UNVALIDATED NORELY")) + ) + } + } + } + + test("Add foreign key constraint with multiple columns") { + withNamespaceAndTable("ns", "tbl", catalog) { t => + sql(s"CREATE TABLE $t (id1 bigint, id2 bigint, fk1 bigint, fk2 bigint, data string) " + + s"$defaultUsing") + sql(s"CREATE TABLE ${t}_ref (id1 bigint, id2 bigint, data string) $defaultUsing") + assert(loadTable(catalog, "ns", "tbl").constraints.isEmpty) + + sql(s"ALTER TABLE $t ADD CONSTRAINT fk1 FOREIGN KEY (fk1, fk2) REFERENCES ${t}_ref(id1, id2)") + val table = loadTable(catalog, "ns", "tbl") + assert(table.constraints.length == 1) + val constraint = table.constraints.head + assert(constraint.name() == "fk1") + assert(constraint.toDDL == + s"CONSTRAINT fk1 FOREIGN KEY (fk1, fk2) " + + s"REFERENCES test_catalog.ns.tbl_ref (id1, id2) NOT ENFORCED UNVALIDATED NORELY") + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/PrimaryKeyConstraintSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/PrimaryKeyConstraintSuite.scala new file mode 100644 index 0000000000000..ae404aff274ac --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/PrimaryKeyConstraintSuite.scala @@ -0,0 +1,100 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.command.v2 + +import org.apache.spark.sql.{AnalysisException, QueryTest} +import org.apache.spark.sql.execution.command.DDLCommandTestUtils + +class PrimaryKeyConstraintSuite extends QueryTest with CommandSuiteBase with DDLCommandTestUtils { + override protected def command: String = "ALTER TABLE .. ADD CONSTRAINT" + + private val validConstraintCharacteristics = Seq( + ("", "NOT ENFORCED UNVALIDATED NORELY"), + ("NOT ENFORCED", "NOT ENFORCED UNVALIDATED NORELY"), + ("NOT ENFORCED NORELY", "NOT ENFORCED UNVALIDATED NORELY"), + ("NORELY NOT ENFORCED", "NOT ENFORCED UNVALIDATED NORELY"), + ("NORELY", "NOT ENFORCED UNVALIDATED NORELY"), + ("RELY", "NOT ENFORCED UNVALIDATED RELY") + ) + + test("Add primary key constraint") { + validConstraintCharacteristics.foreach { case (characteristic, expectedDDL) => + withNamespaceAndTable("ns", "tbl", catalog) { t => + sql(s"CREATE TABLE $t (id bigint, data string) $defaultUsing") + assert(loadTable(catalog, "ns", "tbl").constraints.isEmpty) + + sql(s"ALTER TABLE $t ADD CONSTRAINT pk1 PRIMARY KEY (id) $characteristic") + val table = loadTable(catalog, "ns", "tbl") + assert(table.constraints.length == 1) + val constraint = table.constraints.head + assert(constraint.name() == "pk1") + assert(constraint.toDDL == s"CONSTRAINT pk1 PRIMARY KEY (id) $expectedDDL") + } + } + } + + test("Create table with primary key constraint") { + validConstraintCharacteristics.foreach { case (characteristic, expectedDDL) => + withNamespaceAndTable("ns", "tbl", catalog) { t => + val constraintStr = s"CONSTRAINT pk1 PRIMARY KEY (id) $characteristic" + sql(s"CREATE TABLE $t (id bigint, data string, $constraintStr) $defaultUsing") + val table = loadTable(catalog, "ns", "tbl") + assert(table.constraints.length == 1) + val constraint = table.constraints.head + assert(constraint.name() == "pk1") + assert(constraint.toDDL == s"CONSTRAINT pk1 PRIMARY KEY (id) $expectedDDL") + } + } + } + + test("Add duplicated primary key constraint") { + withNamespaceAndTable("ns", "tbl", catalog) { t => + sql(s"CREATE TABLE $t (id bigint, data string) $defaultUsing") + assert(loadTable(catalog, "ns", "tbl").constraints.isEmpty) + + sql(s"ALTER TABLE $t ADD CONSTRAINT pk1 PRIMARY KEY (id)") + // Constraint names are case-insensitive + Seq("pk1", "PK1").foreach { name => + val error = intercept[AnalysisException] { + sql(s"ALTER TABLE $t ADD CONSTRAINT $name PRIMARY KEY (id)") + } + checkError( + exception = error, + condition = "CONSTRAINT_ALREADY_EXISTS", + sqlState = "42710", + parameters = Map("constraintName" -> "pk1", + "oldConstraint" -> "CONSTRAINT pk1 PRIMARY KEY (id) NOT ENFORCED UNVALIDATED NORELY") + ) + } + } + } + + test("Add primary key constraint with multiple columns") { + withNamespaceAndTable("ns", "tbl", catalog) { t => + sql(s"CREATE TABLE $t (id1 bigint, id2 bigint, data string) $defaultUsing") + assert(loadTable(catalog, "ns", "tbl").constraints.isEmpty) + + sql(s"ALTER TABLE $t ADD CONSTRAINT pk1 PRIMARY KEY (id1, id2)") + val table = loadTable(catalog, "ns", "tbl") + assert(table.constraints.length == 1) + val constraint = table.constraints.head + assert(constraint.name() == "pk1") + assert(constraint.toDDL == + "CONSTRAINT pk1 PRIMARY KEY (id1, id2) NOT ENFORCED UNVALIDATED NORELY") + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/UniqueConstraintSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/UniqueConstraintSuite.scala new file mode 100644 index 0000000000000..4eee2c248cfde --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/UniqueConstraintSuite.scala @@ -0,0 +1,100 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.command.v2 + +import org.apache.spark.sql.{AnalysisException, QueryTest} +import org.apache.spark.sql.execution.command.DDLCommandTestUtils + +class UniqueConstraintSuite extends QueryTest with CommandSuiteBase with DDLCommandTestUtils { + override protected def command: String = "ALTER TABLE .. ADD CONSTRAINT" + + private val validConstraintCharacteristics = Seq( + ("", "NOT ENFORCED UNVALIDATED NORELY"), + ("NOT ENFORCED", "NOT ENFORCED UNVALIDATED NORELY"), + ("NOT ENFORCED NORELY", "NOT ENFORCED UNVALIDATED NORELY"), + ("NORELY NOT ENFORCED", "NOT ENFORCED UNVALIDATED NORELY"), + ("NORELY", "NOT ENFORCED UNVALIDATED NORELY"), + ("RELY", "NOT ENFORCED UNVALIDATED RELY") + ) + + test("Add unique constraint") { + validConstraintCharacteristics.foreach { case (characteristic, expectedDDL) => + withNamespaceAndTable("ns", "tbl", catalog) { t => + sql(s"CREATE TABLE $t (id bigint, data string) $defaultUsing") + assert(loadTable(catalog, "ns", "tbl").constraints.isEmpty) + + sql(s"ALTER TABLE $t ADD CONSTRAINT uk1 UNIQUE (id) $characteristic") + val table = loadTable(catalog, "ns", "tbl") + assert(table.constraints.length == 1) + val constraint = table.constraints.head + assert(constraint.name() == "uk1") + assert(constraint.toDDL == s"CONSTRAINT uk1 UNIQUE (id) $expectedDDL") + } + } + } + + test("Create table with unique constraint") { + validConstraintCharacteristics.foreach { case (characteristic, expectedDDL) => + withNamespaceAndTable("ns", "tbl", catalog) { t => + val constraintStr = s"CONSTRAINT uk1 UNIQUE (id) $characteristic" + sql(s"CREATE TABLE $t (id bigint, data string, $constraintStr) $defaultUsing") + val table = loadTable(catalog, "ns", "tbl") + assert(table.constraints.length == 1) + val constraint = table.constraints.head + assert(constraint.name() == "uk1") + assert(constraint.toDDL == s"CONSTRAINT uk1 UNIQUE (id) $expectedDDL") + } + } + } + + test("Add duplicated unique constraint") { + withNamespaceAndTable("ns", "tbl", catalog) { t => + sql(s"CREATE TABLE $t (id bigint, data string) $defaultUsing") + assert(loadTable(catalog, "ns", "tbl").constraints.isEmpty) + + sql(s"ALTER TABLE $t ADD CONSTRAINT uk1 UNIQUE (id)") + // Constraint names are case-insensitive + Seq("uk1", "UK1").foreach { name => + val error = intercept[AnalysisException] { + sql(s"ALTER TABLE $t ADD CONSTRAINT $name UNIQUE (id)") + } + checkError( + exception = error, + condition = "CONSTRAINT_ALREADY_EXISTS", + sqlState = "42710", + parameters = Map("constraintName" -> "uk1", + "oldConstraint" -> "CONSTRAINT uk1 UNIQUE (id) NOT ENFORCED UNVALIDATED NORELY") + ) + } + } + } + + test("Add unique constraint with multiple columns") { + withNamespaceAndTable("ns", "tbl", catalog) { t => + sql(s"CREATE TABLE $t (id1 bigint, id2 bigint, data string) $defaultUsing") + assert(loadTable(catalog, "ns", "tbl").constraints.isEmpty) + + sql(s"ALTER TABLE $t ADD CONSTRAINT uk1 UNIQUE (id1, id2)") + val table = loadTable(catalog, "ns", "tbl") + assert(table.constraints.length == 1) + val constraint = table.constraints.head + assert(constraint.name() == "uk1") + assert(constraint.toDDL == + "CONSTRAINT uk1 UNIQUE (id1, id2) NOT ENFORCED UNVALIDATED NORELY") + } + } +} From 76508df2d0c9acbc2d453558b80073e3b22b0223 Mon Sep 17 00:00:00 2001 From: Gengliang Wang <gengliang@apache.org> Date: Thu, 17 Apr 2025 10:21:26 -0700 Subject: [PATCH 05/19] add tests --- .../resources/error/error-conditions.json | 6 ++++ .../sql/catalyst/analysis/CheckAnalysis.scala | 9 +++++ .../catalyst/analysis/ResolveTableSpec.scala | 4 +-- .../command/v2/CheckConstraintSuite.scala | 34 +++++++++---------- .../command/v2/CommandSuiteBase.scala | 3 +- .../v2/ForeignKeyConstraintSuite.scala | 6 ++-- .../v2/PrimaryKeyConstraintSuite.scala | 20 +++++------ .../command/v2/UniqueConstraintSuite.scala | 4 +-- 8 files changed, 51 insertions(+), 35 deletions(-) diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index 6591a62151d4d..888f505017ec1 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -4280,6 +4280,12 @@ ], "sqlState" : "38000" }, + "NON_DETERMINISTIC_CHECK_CONSTRAINT" : { + "message" : [ + "The check constraint `<checkCondition>` is non-deterministic. Check constraints must only contain deterministic expressions." + ], + "sqlState" : "42621" + }, "NULLABLE_COLUMN_OR_FIELD" : { "message" : [ "Column or field <name> is nullable while it's required to be non-nullable." diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index 61d4820884614..dc2a0b87316f8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -1138,6 +1138,15 @@ trait CheckAnalysis extends LookupCatalog with QueryErrorsBase with PlanToString case RenameColumn(table: ResolvedTable, col: ResolvedFieldName, newName) => checkColumnNotExists("rename", col.path :+ newName, table.schema) + + case AddConstraint(_: ResolvedTable, check: CheckConstraint) => + if (!check.deterministic) { + check.child.failAnalysis( + errorClass = "NON_DETERMINISTIC_CHECK_CONSTRAINT", + messageParameters = Map("checkCondition" -> check.condition) + ) + } + case AlterColumns(table: ResolvedTable, specs) => val groupedColumns = specs.groupBy(_.column.name) groupedColumns.collect { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveTableSpec.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveTableSpec.scala index f3bffcae0e463..0cdf9a7d4ad4c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveTableSpec.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveTableSpec.scala @@ -81,8 +81,8 @@ object ResolveTableSpec extends Rule[LogicalPlan] { }).get if (!analyzedExpression.deterministic) { analyzedExpression.failAnalysis( - errorClass = "INVALID_CHECK_CONSTRAINT.NONDETERMINISTIC", - messageParameters = Map.empty + errorClass = "NON_DETERMINISTIC_CHECK_CONSTRAINT", + messageParameters = Map("checkCondition" -> c.condition) ) } c.withNewChildren(Seq(analyzedExpression)).asInstanceOf[CheckConstraint] diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/CheckConstraintSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/CheckConstraintSuite.scala index 5d64281ff58c8..f68046ed49dc6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/CheckConstraintSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/CheckConstraintSuite.scala @@ -37,9 +37,9 @@ class CheckConstraintSuite extends QueryTest with CommandSuiteBase with DDLComma } checkError( exception = error, - condition = "INVALID_CHECK_CONSTRAINT.NONDETERMINISTIC", + condition = "NON_DETERMINISTIC_CHECK_CONSTRAINT", sqlState = "42621", - parameters = Map.empty, + parameters = Map("checkCondition" -> "i > rand(0)"), context = ExpectedContext( fragment = "i > rand(0)", start = 40, @@ -62,9 +62,9 @@ class CheckConstraintSuite extends QueryTest with CommandSuiteBase with DDLComma } checkError( exception = error, - condition = "INVALID_CHECK_CONSTRAINT.NONDETERMINISTIC", + condition = "NON_DETERMINISTIC_CHECK_CONSTRAINT", sqlState = "42621", - parameters = Map.empty, + parameters = Map("checkCondition" -> "i > rand(0)"), context = ExpectedContext( fragment = "i > rand(0)" ) @@ -125,14 +125,14 @@ class CheckConstraintSuite extends QueryTest with CommandSuiteBase with DDLComma } test("Predicate should be null if it can't be converted to V2 predicate") { - withNamespaceAndTable("ns", "tbl", catalog) { t => + withNamespaceAndTable("ns", "tbl", nonPartitionCatalog) { t => sql(s"CREATE TABLE $t (id bigint, j string) $defaultUsing") sql(s"ALTER TABLE $t ADD CONSTRAINT c1 CHECK (from_json(j, 'a INT').a > 1)") - val table = loadTable(catalog, "ns", "tbl") + val table = loadTable(nonPartitionCatalog, "ns", "tbl") val constraint = getCheckConstraint(table) assert(constraint.name() == "c1") assert(constraint.toDDL == - "CONSTRAINT c1 CHECK from_json(j, 'a INT').a > 1 ENFORCED VALID NORELY") + "CONSTRAINT c1 CHECK (from_json(j, 'a INT').a > 1) ENFORCED VALID NORELY") assert(constraint.predicateSql() == "from_json(j, 'a INT').a > 1") assert(constraint.predicate() == null) } @@ -156,36 +156,36 @@ class CheckConstraintSuite extends QueryTest with CommandSuiteBase with DDLComma test("Create table with check constraint") { getConstraintCharacteristics(true).foreach { case (characteristic, expectedDDL) => - withNamespaceAndTable("ns", "tbl", catalog) { t => + withNamespaceAndTable("ns", "tbl", nonPartitionCatalog) { t => val constraintStr = s"CONSTRAINT c1 CHECK (id > 0) $characteristic" sql(s"CREATE TABLE $t (id bigint, data string, $constraintStr) $defaultUsing") - val table = loadTable(catalog, "ns", "tbl") + val table = loadTable(nonPartitionCatalog, "ns", "tbl") val constraint = getCheckConstraint(table) assert(constraint.name() == "c1") - assert(constraint.toDDL == s"CONSTRAINT c1 CHECK id > 0 $expectedDDL") + assert(constraint.toDDL == s"CONSTRAINT c1 CHECK (id > 0) $expectedDDL") } } } test("Alter table add check constraint") { getConstraintCharacteristics(false).foreach { case (characteristic, expectedDDL) => - withNamespaceAndTable("ns", "tbl", catalog) { t => + withNamespaceAndTable("ns", "tbl", nonPartitionCatalog) { t => sql(s"CREATE TABLE $t (id bigint, data string) $defaultUsing") - assert(loadTable(catalog, "ns", "tbl").constraints.isEmpty) + assert(loadTable(nonPartitionCatalog, "ns", "tbl").constraints.isEmpty) sql(s"ALTER TABLE $t ADD CONSTRAINT c1 CHECK (id > 0) $characteristic") - val table = loadTable(catalog, "ns", "tbl") + val table = loadTable(nonPartitionCatalog, "ns", "tbl") val constraint = getCheckConstraint(table) assert(constraint.name() == "c1") - assert(constraint.toDDL == s"CONSTRAINT c1 CHECK id > 0 $expectedDDL") + assert(constraint.toDDL == s"CONSTRAINT c1 CHECK (id > 0) $expectedDDL") } } } test("Add duplicated check constraint") { - withNamespaceAndTable("ns", "tbl", catalog) { t => + withNamespaceAndTable("ns", "tbl", nonPartitionCatalog) { t => sql(s"CREATE TABLE $t (id bigint, data string) $defaultUsing") - assert(loadTable(catalog, "ns", "tbl").constraints.isEmpty) + assert(loadTable(nonPartitionCatalog, "ns", "tbl").constraints.isEmpty) sql(s"ALTER TABLE $t ADD CONSTRAINT abc CHECK (id > 0)") // Constraint names are case-insensitive @@ -198,7 +198,7 @@ class CheckConstraintSuite extends QueryTest with CommandSuiteBase with DDLComma condition = "CONSTRAINT_ALREADY_EXISTS", sqlState = "42710", parameters = Map("constraintName" -> "abc", - "oldConstraint" -> "CONSTRAINT abc CHECK id > 0 ENFORCED VALID NORELY") + "oldConstraint" -> "CONSTRAINT abc CHECK (id > 0) ENFORCED VALID NORELY") ) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/CommandSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/CommandSuiteBase.scala index 15b95d35ff241..76928faec3189 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/CommandSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/CommandSuiteBase.scala @@ -34,12 +34,13 @@ trait CommandSuiteBase extends SharedSparkSession { def catalogVersion: String = "V2" // The catalog version is added to test names def commandVersion: String = "V2" // The command version is added to test names def catalog: String = "test_catalog" // The default V2 catalog for testing + def nonPartitionCatalog: String = "non_part_test_catalog" // Catalog for non-partitioned tables def defaultUsing: String = "USING _" // The clause is used in creating v2 tables under testing // V2 catalogs created and used especially for testing override def sparkConf: SparkConf = super.sparkConf .set(s"spark.sql.catalog.$catalog", classOf[InMemoryPartitionTableCatalog].getName) - .set(s"spark.sql.catalog.non_part_$catalog", classOf[InMemoryTableCatalog].getName) + .set(s"spark.sql.catalog.$nonPartitionCatalog", classOf[InMemoryTableCatalog].getName) .set(s"spark.sql.catalog.fun_$catalog", classOf[InMemoryCatalog].getName) def checkLocation( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ForeignKeyConstraintSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ForeignKeyConstraintSuite.scala index 0a309e652a75b..65bc670658c84 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ForeignKeyConstraintSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ForeignKeyConstraintSuite.scala @@ -52,17 +52,17 @@ class ForeignKeyConstraintSuite extends QueryTest with CommandSuiteBase with DDL test("Create table with foreign key constraint") { validConstraintCharacteristics.foreach { case (characteristic, expectedDDL) => - withNamespaceAndTable("ns", "tbl", catalog) { t => + withNamespaceAndTable("ns", "tbl", nonPartitionCatalog) { t => sql(s"CREATE TABLE ${t}_ref (id bigint, data string) $defaultUsing") val constraintStr = s"CONSTRAINT fk1 FOREIGN KEY (fk) " + s"REFERENCES ${t}_ref(id) $characteristic" sql(s"CREATE TABLE $t (id bigint, fk bigint, data string, $constraintStr) $defaultUsing") - val table = loadTable(catalog, "ns", "tbl") + val table = loadTable(nonPartitionCatalog, "ns", "tbl") assert(table.constraints.length == 1) val constraint = table.constraints.head assert(constraint.name() == "fk1") assert(constraint.toDDL == s"CONSTRAINT fk1 FOREIGN KEY (fk) " + - s"REFERENCES test_catalog.ns.tbl_ref (id) $expectedDDL") + s"REFERENCES non_part_test_catalog.ns.tbl_ref (id) $expectedDDL") } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/PrimaryKeyConstraintSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/PrimaryKeyConstraintSuite.scala index ae404aff274ac..332cd3df6be05 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/PrimaryKeyConstraintSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/PrimaryKeyConstraintSuite.scala @@ -33,12 +33,12 @@ class PrimaryKeyConstraintSuite extends QueryTest with CommandSuiteBase with DDL test("Add primary key constraint") { validConstraintCharacteristics.foreach { case (characteristic, expectedDDL) => - withNamespaceAndTable("ns", "tbl", catalog) { t => + withNamespaceAndTable("ns", "tbl", nonPartitionCatalog) { t => sql(s"CREATE TABLE $t (id bigint, data string) $defaultUsing") - assert(loadTable(catalog, "ns", "tbl").constraints.isEmpty) + assert(loadTable(nonPartitionCatalog, "ns", "tbl").constraints.isEmpty) sql(s"ALTER TABLE $t ADD CONSTRAINT pk1 PRIMARY KEY (id) $characteristic") - val table = loadTable(catalog, "ns", "tbl") + val table = loadTable(nonPartitionCatalog, "ns", "tbl") assert(table.constraints.length == 1) val constraint = table.constraints.head assert(constraint.name() == "pk1") @@ -49,10 +49,10 @@ class PrimaryKeyConstraintSuite extends QueryTest with CommandSuiteBase with DDL test("Create table with primary key constraint") { validConstraintCharacteristics.foreach { case (characteristic, expectedDDL) => - withNamespaceAndTable("ns", "tbl", catalog) { t => + withNamespaceAndTable("ns", "tbl", nonPartitionCatalog) { t => val constraintStr = s"CONSTRAINT pk1 PRIMARY KEY (id) $characteristic" sql(s"CREATE TABLE $t (id bigint, data string, $constraintStr) $defaultUsing") - val table = loadTable(catalog, "ns", "tbl") + val table = loadTable(nonPartitionCatalog, "ns", "tbl") assert(table.constraints.length == 1) val constraint = table.constraints.head assert(constraint.name() == "pk1") @@ -62,9 +62,9 @@ class PrimaryKeyConstraintSuite extends QueryTest with CommandSuiteBase with DDL } test("Add duplicated primary key constraint") { - withNamespaceAndTable("ns", "tbl", catalog) { t => + withNamespaceAndTable("ns", "tbl", nonPartitionCatalog) { t => sql(s"CREATE TABLE $t (id bigint, data string) $defaultUsing") - assert(loadTable(catalog, "ns", "tbl").constraints.isEmpty) + assert(loadTable(nonPartitionCatalog, "ns", "tbl").constraints.isEmpty) sql(s"ALTER TABLE $t ADD CONSTRAINT pk1 PRIMARY KEY (id)") // Constraint names are case-insensitive @@ -84,12 +84,12 @@ class PrimaryKeyConstraintSuite extends QueryTest with CommandSuiteBase with DDL } test("Add primary key constraint with multiple columns") { - withNamespaceAndTable("ns", "tbl", catalog) { t => + withNamespaceAndTable("ns", "tbl", nonPartitionCatalog) { t => sql(s"CREATE TABLE $t (id1 bigint, id2 bigint, data string) $defaultUsing") - assert(loadTable(catalog, "ns", "tbl").constraints.isEmpty) + assert(loadTable(nonPartitionCatalog, "ns", "tbl").constraints.isEmpty) sql(s"ALTER TABLE $t ADD CONSTRAINT pk1 PRIMARY KEY (id1, id2)") - val table = loadTable(catalog, "ns", "tbl") + val table = loadTable(nonPartitionCatalog, "ns", "tbl") assert(table.constraints.length == 1) val constraint = table.constraints.head assert(constraint.name() == "pk1") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/UniqueConstraintSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/UniqueConstraintSuite.scala index 4eee2c248cfde..8971cec2eb013 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/UniqueConstraintSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/UniqueConstraintSuite.scala @@ -49,10 +49,10 @@ class UniqueConstraintSuite extends QueryTest with CommandSuiteBase with DDLComm test("Create table with unique constraint") { validConstraintCharacteristics.foreach { case (characteristic, expectedDDL) => - withNamespaceAndTable("ns", "tbl", catalog) { t => + withNamespaceAndTable("ns", "tbl", nonPartitionCatalog) { t => val constraintStr = s"CONSTRAINT uk1 UNIQUE (id) $characteristic" sql(s"CREATE TABLE $t (id bigint, data string, $constraintStr) $defaultUsing") - val table = loadTable(catalog, "ns", "tbl") + val table = loadTable(nonPartitionCatalog, "ns", "tbl") assert(table.constraints.length == 1) val constraint = table.constraints.head assert(constraint.name() == "uk1") From a6507e01a253ce2cf8462e8bcc3a1278959d8932 Mon Sep 17 00:00:00 2001 From: Gengliang Wang <gengliang@apache.org> Date: Thu, 17 Apr 2025 16:59:10 -0700 Subject: [PATCH 06/19] rename --- .../sql/catalyst/analysis/ResolveTableSpec.scala | 2 +- .../spark/sql/catalyst/expressions/constraints.scala | 11 +++++------ .../catalyst/plans/logical/v2AlterTableCommands.scala | 2 +- 3 files changed, 7 insertions(+), 8 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveTableSpec.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveTableSpec.scala index 0cdf9a7d4ad4c..331b796a0adab 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveTableSpec.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveTableSpec.scala @@ -140,7 +140,7 @@ object ResolveTableSpec extends Rule[LogicalPlan] { collation = u.collation, serde = u.serde, external = u.external, - constraints = newConstraints.map(_.asConstraint(isCreateTable = true))) + constraints = newConstraints.map(_.toV2Constraint(isCreateTable = true))) withNewSpec(newTableSpec) case _ => input diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/constraints.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/constraints.scala index 1e70aee00b2e3..8e1c539bc3ce4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/constraints.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/constraints.scala @@ -27,7 +27,7 @@ import org.apache.spark.sql.types.{DataType, StringType} trait TableConstraint { // Convert to a data source v2 constraint - def asConstraint(isCreateTable: Boolean): Constraint + def toV2Constraint(isCreateTable: Boolean): Constraint /** Returns the user-provided name of the constraint */ def userProvidedName: String @@ -117,7 +117,7 @@ case class CheckConstraint( with TableConstraint { // scalastyle:on line.size.limit - def asConstraint(isCreateTable: Boolean): Constraint = { + def toV2Constraint(isCreateTable: Boolean): Constraint = { val predicate = new V2ExpressionBuilder(child, true).buildPredicate().orNull val enforced = userProvidedCharacteristic.enforced.getOrElse(true) val rely = userProvidedCharacteristic.rely.getOrElse(false) @@ -166,8 +166,7 @@ case class PrimaryKeyConstraint( extends TableConstraint { // scalastyle:on line.size.limit - - override def asConstraint(isCreateTable: Boolean): Constraint = { + override def toV2Constraint(isCreateTable: Boolean): Constraint = { val enforced = userProvidedCharacteristic.enforced.getOrElse(false) val rely = userProvidedCharacteristic.rely.getOrElse(false) Constraint @@ -199,7 +198,7 @@ case class UniqueConstraint( extends TableConstraint { // scalastyle:on line.size.limit - override def asConstraint(isCreateTable: Boolean): Constraint = { + override def toV2Constraint(isCreateTable: Boolean): Constraint = { val enforced = userProvidedCharacteristic.enforced.getOrElse(false) val rely = userProvidedCharacteristic.rely.getOrElse(false) Constraint @@ -237,7 +236,7 @@ case class ForeignKeyConstraint( import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ - override def asConstraint(isCreateTable: Boolean): Constraint = { + override def toV2Constraint(isCreateTable: Boolean): Constraint = { val enforced = userProvidedCharacteristic.enforced.getOrElse(false) val rely = userProvidedCharacteristic.rely.getOrElse(false) Constraint diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2AlterTableCommands.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2AlterTableCommands.scala index 5c61f4721ef73..eec0a4a3c3502 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2AlterTableCommands.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2AlterTableCommands.scala @@ -296,7 +296,7 @@ case class AddConstraint( table: LogicalPlan, tableConstraint: TableConstraint) extends AlterTableCommand { override def changes: Seq[TableChange] = { - val constraint = tableConstraint.asConstraint(isCreateTable = false) + val constraint = tableConstraint.toV2Constraint(isCreateTable = false) val validatedTableVersion = table match { case t: ResolvedTable if constraint.enforced() => t.table.currentVersion() From b78c4f0a5e6f05a4cae5a9d68a349fe6f79833e3 Mon Sep 17 00:00:00 2001 From: Gengliang Wang <gengliang@apache.org> Date: Thu, 17 Apr 2025 17:25:42 -0700 Subject: [PATCH 07/19] add replace table test cases --- .../command/v2/CheckConstraintSuite.scala | 14 ++++++++++++++ .../command/v2/ForeignKeyConstraintSuite.scala | 18 ++++++++++++++++++ .../command/v2/PrimaryKeyConstraintSuite.scala | 15 +++++++++++++++ .../command/v2/UniqueConstraintSuite.scala | 15 +++++++++++++++ 4 files changed, 62 insertions(+) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/CheckConstraintSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/CheckConstraintSuite.scala index f68046ed49dc6..70d4a5e92b6f2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/CheckConstraintSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/CheckConstraintSuite.scala @@ -167,6 +167,20 @@ class CheckConstraintSuite extends QueryTest with CommandSuiteBase with DDLComma } } + test("Replace table with check constraint") { + getConstraintCharacteristics(true).foreach { case (characteristic, expectedDDL) => + withNamespaceAndTable("ns", "tbl", nonPartitionCatalog) { t => + val constraintStr = s"CONSTRAINT c1 CHECK (id > 0) $characteristic" + sql(s"CREATE TABLE $t (id bigint) $defaultUsing") + sql(s"REPLACE TABLE $t (id bigint, data string, $constraintStr) $defaultUsing") + val table = loadTable(nonPartitionCatalog, "ns", "tbl") + val constraint = getCheckConstraint(table) + assert(constraint.name() == "c1") + assert(constraint.toDDL == s"CONSTRAINT c1 CHECK (id > 0) $expectedDDL") + } + } + } + test("Alter table add check constraint") { getConstraintCharacteristics(false).foreach { case (characteristic, expectedDDL) => withNamespaceAndTable("ns", "tbl", nonPartitionCatalog) { t => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ForeignKeyConstraintSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ForeignKeyConstraintSuite.scala index 65bc670658c84..5ef541074e1ae 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ForeignKeyConstraintSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ForeignKeyConstraintSuite.scala @@ -67,6 +67,24 @@ class ForeignKeyConstraintSuite extends QueryTest with CommandSuiteBase with DDL } } + test("REPLACE table with foreign key constraint") { + validConstraintCharacteristics.foreach { case (characteristic, expectedDDL) => + withNamespaceAndTable("ns", "tbl", nonPartitionCatalog) { t => + sql(s"CREATE TABLE $t (id bigint) $defaultUsing") + sql(s"CREATE TABLE ${t}_ref (id bigint, data string) $defaultUsing") + val constraintStr = s"CONSTRAINT fk1 FOREIGN KEY (fk) " + + s"REFERENCES ${t}_ref(id) $characteristic" + sql(s"REPLACE TABLE $t (id bigint, fk bigint, data string, $constraintStr) $defaultUsing") + val table = loadTable(nonPartitionCatalog, "ns", "tbl") + assert(table.constraints.length == 1) + val constraint = table.constraints.head + assert(constraint.name() == "fk1") + assert(constraint.toDDL == s"CONSTRAINT fk1 FOREIGN KEY (fk) " + + s"REFERENCES non_part_test_catalog.ns.tbl_ref (id) $expectedDDL") + } + } + } + test("Add duplicated foreign key constraint") { withNamespaceAndTable("ns", "tbl", catalog) { t => sql(s"CREATE TABLE $t (id bigint, fk bigint, data string) $defaultUsing") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/PrimaryKeyConstraintSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/PrimaryKeyConstraintSuite.scala index 332cd3df6be05..79763c059d151 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/PrimaryKeyConstraintSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/PrimaryKeyConstraintSuite.scala @@ -61,6 +61,21 @@ class PrimaryKeyConstraintSuite extends QueryTest with CommandSuiteBase with DDL } } + test("Replace table with primary key constraint") { + validConstraintCharacteristics.foreach { case (characteristic, expectedDDL) => + withNamespaceAndTable("ns", "tbl", nonPartitionCatalog) { t => + val constraintStr = s"CONSTRAINT pk1 PRIMARY KEY (id) $characteristic" + sql(s"CREATE TABLE $t (id bigint) $defaultUsing") + sql(s"REPLACE TABLE $t (id bigint, data string, $constraintStr) $defaultUsing") + val table = loadTable(nonPartitionCatalog, "ns", "tbl") + assert(table.constraints.length == 1) + val constraint = table.constraints.head + assert(constraint.name() == "pk1") + assert(constraint.toDDL == s"CONSTRAINT pk1 PRIMARY KEY (id) $expectedDDL") + } + } + } + test("Add duplicated primary key constraint") { withNamespaceAndTable("ns", "tbl", nonPartitionCatalog) { t => sql(s"CREATE TABLE $t (id bigint, data string) $defaultUsing") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/UniqueConstraintSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/UniqueConstraintSuite.scala index 8971cec2eb013..bd290657e3e21 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/UniqueConstraintSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/UniqueConstraintSuite.scala @@ -61,6 +61,21 @@ class UniqueConstraintSuite extends QueryTest with CommandSuiteBase with DDLComm } } + test("Replace table with unique constraint") { + validConstraintCharacteristics.foreach { case (characteristic, expectedDDL) => + withNamespaceAndTable("ns", "tbl", nonPartitionCatalog) { t => + val constraintStr = s"CONSTRAINT uk1 UNIQUE (id) $characteristic" + sql(s"CREATE TABLE $t (id bigint) $defaultUsing") + sql(s"REPLACE TABLE $t (id bigint, data string, $constraintStr) $defaultUsing") + val table = loadTable(nonPartitionCatalog, "ns", "tbl") + assert(table.constraints.length == 1) + val constraint = table.constraints.head + assert(constraint.name() == "uk1") + assert(constraint.toDDL == s"CONSTRAINT uk1 UNIQUE (id) $expectedDDL") + } + } + } + test("Add duplicated unique constraint") { withNamespaceAndTable("ns", "tbl", catalog) { t => sql(s"CREATE TABLE $t (id bigint, data string) $defaultUsing") From d54a64f9380dc44f2db5a03af592723e8a007415 Mon Sep 17 00:00:00 2001 From: Gengliang Wang <gengliang@apache.org> Date: Fri, 18 Apr 2025 09:16:10 -0700 Subject: [PATCH 08/19] fix formatting --- .../src/main/resources/error/error-conditions.json | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index 888f505017ec1..48b86589be426 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -4070,6 +4070,12 @@ ], "sqlState" : "HV091" }, + "NON_DETERMINISTIC_CHECK_CONSTRAINT" : { + "message" : [ + "The check constraint `<checkCondition>` is non-deterministic. Check constraints must only contain deterministic expressions." + ], + "sqlState" : "42621" + }, "NON_FOLDABLE_ARGUMENT" : { "message" : [ "The function <funcName> requires the parameter <paramName> to be a foldable expression of the type <paramType>, but the actual argument is a non-foldable." @@ -4280,12 +4286,6 @@ ], "sqlState" : "38000" }, - "NON_DETERMINISTIC_CHECK_CONSTRAINT" : { - "message" : [ - "The check constraint `<checkCondition>` is non-deterministic. Check constraints must only contain deterministic expressions." - ], - "sqlState" : "42621" - }, "NULLABLE_COLUMN_OR_FIELD" : { "message" : [ "Column or field <name> is nullable while it's required to be non-nullable." From 0e4b88a99d6c52c5dbcbf32665b99255e1f291d5 Mon Sep 17 00:00:00 2001 From: Gengliang Wang <gengliang@apache.org> Date: Wed, 23 Apr 2025 20:43:31 -0700 Subject: [PATCH 09/19] convert constraints to expressions --- .../resources/error/error-conditions.json | 6 ++++ .../catalyst/expressions/constraints.scala | 34 ++++++++++++++----- .../catalyst/plans/logical/v2Commands.scala | 19 +++++++---- 3 files changed, 44 insertions(+), 15 deletions(-) diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index 48b86589be426..216f683aa7fc1 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -825,6 +825,12 @@ ], "sqlState" : "42704" }, + "CONSTRAINT_DOES_NOT_HAVE_DATA_TYPE" : { + "message" : [ + "Table constraint expressions do not have a data type." + ], + "sqlState" : "0A000" + }, "CONVERSION_INVALID_INPUT" : { "message" : [ "The value <str> (<fmt>) cannot be converted to <targetType> because it is malformed. Correct the value as per the syntax, or change its format. Use <suggestion> to tolerate malformed input and return NULL instead." diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/constraints.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/constraints.scala index 8e1c539bc3ce4..05647e4b8306f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/constraints.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/constraints.scala @@ -18,14 +18,16 @@ package org.apache.spark.sql.catalyst.expressions import java.util.UUID +import org.apache.spark.SparkUnsupportedOperationException + import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.catalyst.trees.CurrentOrigin import org.apache.spark.sql.catalyst.util.V2ExpressionBuilder import org.apache.spark.sql.connector.catalog.constraints.Constraint import org.apache.spark.sql.connector.expressions.FieldReference -import org.apache.spark.sql.types.{DataType, StringType} +import org.apache.spark.sql.types.{ArrayType, DataType, StringType} -trait TableConstraint { +trait TableConstraint extends Expression with Unevaluable { // Convert to a data source v2 constraint def toV2Constraint(isCreateTable: Boolean): Constraint @@ -97,6 +99,11 @@ trait TableConstraint { ) } } + + override def nullable: Boolean = true + + override def dataType: DataType = + throw new SparkUnsupportedOperationException("CONSTRAINT_DOES_NOT_HAVE_DATA_TYPE") } case class ConstraintCharacteristic(enforced: Option[Boolean], rely: Option[Boolean]) @@ -113,7 +120,6 @@ case class CheckConstraint( override val tableName: String = null, override val userProvidedCharacteristic: ConstraintCharacteristic = ConstraintCharacteristic.empty) extends UnaryExpression - with Unevaluable with TableConstraint { // scalastyle:on line.size.limit @@ -147,8 +153,6 @@ case class CheckConstraint( override def sql: String = s"CONSTRAINT $userProvidedName CHECK ($condition)" - override def dataType: DataType = StringType - override def withUserProvidedName(name: String): TableConstraint = copy(userProvidedName = name) override def withTableName(tableName: String): TableConstraint = copy(tableName = tableName) @@ -163,7 +167,8 @@ case class PrimaryKeyConstraint( override val userProvidedName: String = null, override val tableName: String = null, override val userProvidedCharacteristic: ConstraintCharacteristic = ConstraintCharacteristic.empty) - extends TableConstraint { + extends LeafExpression + with TableConstraint { // scalastyle:on line.size.limit override def toV2Constraint(isCreateTable: Boolean): Constraint = { @@ -195,7 +200,8 @@ case class UniqueConstraint( override val userProvidedName: String = null, override val tableName: String = null, override val userProvidedCharacteristic: ConstraintCharacteristic = ConstraintCharacteristic.empty) - extends TableConstraint { + extends LeafExpression + with TableConstraint { // scalastyle:on line.size.limit override def toV2Constraint(isCreateTable: Boolean): Constraint = { @@ -231,7 +237,8 @@ case class ForeignKeyConstraint( override val userProvidedName: String = null, override val tableName: String = null, override val userProvidedCharacteristic: ConstraintCharacteristic = ConstraintCharacteristic.empty) - extends TableConstraint { + extends LeafExpression + with TableConstraint { // scalastyle:on line.size.limit import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ @@ -262,3 +269,14 @@ case class ForeignKeyConstraint( copy(userProvidedCharacteristic = c) } } + +case class Constraints(children: Seq[Expression]) extends Expression with Unevaluable { + + override def nullable: Boolean = true + + override def dataType: DataType = + throw new SparkUnsupportedOperationException("CONSTRAINTS_DO_NOT_HAVE_DATA_TYPE") + + override protected def withNewChildrenInternal(newChildren: IndexedSeq[Expression]): Expression = + copy(children = newChildren) +} \ No newline at end of file diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala index e0d44e7d248ed..782c2f29328c1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala @@ -1504,20 +1504,25 @@ case class UnresolvedTableSpec( collation: Option[String], serde: Option[SerdeInfo], external: Boolean, - constraints: Seq[TableConstraint]) - extends UnaryExpression with Unevaluable with TableSpecBase { + constraints: Constraints) + extends BinaryExpression with Unevaluable with TableSpecBase { override def dataType: DataType = throw new SparkUnsupportedOperationException("_LEGACY_ERROR_TEMP_3113") - override def child: Expression = optionExpression - - override protected def withNewChildInternal(newChild: Expression): Expression = - this.copy(optionExpression = newChild.asInstanceOf[OptionList]) - override def simpleString(maxFields: Int): String = { this.copy(properties = Utils.redact(properties).toMap).toString } + + override def left: Expression = optionExpression + + override def right: Expression = constraints + + override protected def withNewChildrenInternal( + newLeft: Expression, + newRight: Expression): Expression = + copy(optionExpression = newLeft.asInstanceOf[OptionList], + constraints = newRight.asInstanceOf[Constraints]) } /** From aa2b99b29b1869c50396131b86d71074ef37ed0d Mon Sep 17 00:00:00 2001 From: Gengliang Wang <gengliang@apache.org> Date: Wed, 23 Apr 2025 21:10:34 -0700 Subject: [PATCH 10/19] refactor ResolvedIdentifier --- .../spark/sql/catalyst/analysis/v2ResolutionPlans.scala | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/v2ResolutionPlans.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/v2ResolutionPlans.scala index 6f657e931a49e..b52091afc1333 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/v2ResolutionPlans.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/v2ResolutionPlans.scala @@ -252,8 +252,13 @@ case class ResolvedNonPersistentFunc( */ case class ResolvedIdentifier( catalog: CatalogPlugin, - identifier: Identifier) extends LeafNodeWithoutStats { - override def output: Seq[Attribute] = Nil + identifier: Identifier, + override val output: Seq[Attribute] = Nil) extends LeafNodeWithoutStats + +object ResolvedIdentifier { + def unapply(ri: ResolvedIdentifier): Option[(CatalogPlugin, Identifier)] = { + Some((ri.catalog, ri.identifier)) + } } // A fake v2 catalog to hold temp views. From f4932abafaae4e0234a5414ca24758380a767986 Mon Sep 17 00:00:00 2001 From: Gengliang Wang <gengliang@apache.org> Date: Wed, 23 Apr 2025 22:24:08 -0700 Subject: [PATCH 11/19] more refactor and fix compiling --- .../catalyst/analysis/ResolveCatalogs.scala | 33 +++++++-- .../catalyst/analysis/ResolveTableSpec.scala | 67 +++++-------------- .../catalyst/expressions/constraints.scala | 15 ++++- .../sql/catalyst/parser/AstBuilder.scala | 4 +- ...eateTablePartitioningValidationSuite.scala | 4 +- .../sql/catalyst/parser/DDLParserSuite.scala | 18 ++--- .../apache/spark/sql/classic/Catalog.scala | 4 +- .../spark/sql/classic/DataFrameWriter.scala | 8 +-- .../spark/sql/classic/DataFrameWriterV2.scala | 4 +- .../spark/sql/classic/DataStreamWriter.scala | 3 +- .../V2CommandsCaseSensitivitySuite.scala | 9 +-- .../command/CheckConstraintParseSuite.scala | 14 ++-- .../command/ConstraintParseSuiteBase.scala | 8 +-- 13 files changed, 95 insertions(+), 96 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala index ab9487aa66644..06dd8a3ffc974 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala @@ -22,6 +22,7 @@ import scala.jdk.CollectionConverters._ import org.apache.spark.SparkException import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.SqlScriptingLocalVariableManager +import org.apache.spark.sql.catalyst.expressions.AttributeReference import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.connector.catalog.{CatalogManager, CatalogPlugin, Identifier, LookupCatalog, SupportsNamespaces} @@ -77,14 +78,16 @@ class ResolveCatalogs(val catalogManager: CatalogManager) assertValidSessionVariableNameParts(nameParts, resolved) d.copy(name = resolved) + case c @ CreateTable(UnresolvedIdentifier(nameParts, allowTemp), columns, _, _, _) => + val resolvedIdentifier = resolveIdentifier(nameParts, allowTemp, columns) + c.copy(name = resolvedIdentifier) + + case r @ ReplaceTable(UnresolvedIdentifier(nameParts, allowTemp), columns, _, _, _) => + val resolvedIdentifier = resolveIdentifier(nameParts, allowTemp, columns) + r.copy(name = resolvedIdentifier) + case UnresolvedIdentifier(nameParts, allowTemp) => - if (allowTemp && catalogManager.v1SessionCatalog.isTempView(nameParts)) { - val ident = Identifier.of(nameParts.dropRight(1).toArray, nameParts.last) - ResolvedIdentifier(FakeSystemCatalog, ident) - } else { - val CatalogAndIdentifier(catalog, identifier) = nameParts - ResolvedIdentifier(catalog, identifier) - } + resolveIdentifier(nameParts, allowTemp, Seq.empty) case CurrentNamespace => ResolvedNamespace(currentCatalog, catalogManager.currentNamespace.toImmutableArraySeq) @@ -94,6 +97,22 @@ class ResolveCatalogs(val catalogManager: CatalogManager) resolveNamespace(catalog, ns, fetchMetadata) } + private def resolveIdentifier( + nameParts: Seq[String], + allowTemp: Boolean, + columns: Seq[ColumnDefinition]): ResolvedIdentifier = { + val columnOutput = columns.map { col => + AttributeReference(col.name, col.dataType, col.nullable, col.metadata)() + } + if (allowTemp && catalogManager.v1SessionCatalog.isTempView(nameParts)) { + val ident = Identifier.of(nameParts.dropRight(1).toArray, nameParts.last) + ResolvedIdentifier(FakeSystemCatalog, ident, columnOutput) + } else { + val CatalogAndIdentifier(catalog, identifier) = nameParts + ResolvedIdentifier(catalog, identifier, columnOutput) + } + } + private def resolveNamespace( catalog: CatalogPlugin, ns: Seq[String], diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveTableSpec.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveTableSpec.scala index 331b796a0adab..961556d267cc6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveTableSpec.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveTableSpec.scala @@ -23,7 +23,6 @@ import org.apache.spark.sql.catalyst.optimizer.ComputeCurrentTime import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.trees.TreePattern.COMMAND -import org.apache.spark.sql.catalyst.util.ResolveDefaultColumns.DefaultColumnAnalyzer import org.apache.spark.sql.errors.QueryCompilationErrors import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{ArrayType, MapType, StructType} @@ -47,57 +46,22 @@ object ResolveTableSpec extends Rule[LogicalPlan] { preparedPlan.resolveOperatorsWithPruning(_.containsAnyPattern(COMMAND), ruleId) { case t: CreateTable => - resolveTableSpec(t, t.tableSpec, - fakeRelationFromColumns(t.columns), s => t.copy(tableSpec = s)) + resolveTableSpec(t, t.tableSpec, s => t.copy(tableSpec = s)) case t: CreateTableAsSelect => - resolveTableSpec(t, t.tableSpec, None, s => t.copy(tableSpec = s)) + resolveTableSpec(t, t.tableSpec, s => t.copy(tableSpec = s)) case t: ReplaceTable => - resolveTableSpec(t, t.tableSpec, - fakeRelationFromColumns(t.columns), s => t.copy(tableSpec = s)) + resolveTableSpec(t, t.tableSpec, s => t.copy(tableSpec = s)) case t: ReplaceTableAsSelect => - resolveTableSpec(t, t.tableSpec, None, s => t.copy(tableSpec = s)) + resolveTableSpec(t, t.tableSpec, s => t.copy(tableSpec = s)) } } - private def fakeRelationFromColumns(columns: Seq[ColumnDefinition]): Option[LogicalPlan] = { - val attributeList = columns.map { col => - AttributeReference(col.name, col.dataType)() - } - Some(LocalRelation(attributeList)) - } - - private def analyzeConstraints( - constraints: Seq[TableConstraint], - fakeRelation: LogicalPlan): Seq[TableConstraint] = { - val analyzedExpressions = constraints.map { - case c: CheckConstraint => - val alias = Alias(c.child, c.name)() - val project = Project(Seq(alias), fakeRelation) - val analyzed = DefaultColumnAnalyzer.execute(project) - DefaultColumnAnalyzer.checkAnalysis0(analyzed) - - val analyzedExpression = (analyzed collectFirst { - case Project(Seq(Alias(e: Expression, _)), _) => e - }).get - if (!analyzedExpression.deterministic) { - analyzedExpression.failAnalysis( - errorClass = "NON_DETERMINISTIC_CHECK_CONSTRAINT", - messageParameters = Map("checkCondition" -> c.condition) - ) - } - c.withNewChildren(Seq(analyzedExpression)).asInstanceOf[CheckConstraint] - case other => other - } - analyzedExpressions - } - /** Helper method to resolve the table specification within a logical plan. */ private def resolveTableSpec( input: LogicalPlan, tableSpec: TableSpecBase, - fakeRelation: Option[LogicalPlan], withNewSpec: TableSpecBase => LogicalPlan): LogicalPlan = tableSpec match { - case u: UnresolvedTableSpec if u.optionExpression.resolved => + case u: UnresolvedTableSpec if u.optionExpression.resolved && u.constraints.resolved => val newOptions: Seq[(String, String)] = u.optionExpression.options.map { case (key: String, null) => (key, null) @@ -122,15 +86,18 @@ object ResolveTableSpec extends Rule[LogicalPlan] { } (key, newValue) } - // Analyze constraints for the table spec: - // If a fake relation is provided (for CREATE/REPLACE TABLE with column definitions), - // analyze the constraints using the fake relation as context. - // Otherwise, use the original constraints unmodified - val newConstraints = if (fakeRelation.isDefined) { - analyzeConstraints(u.constraints, fakeRelation.get) - } else { - u.constraints + + u.constraints.children.foreach { + case check: CheckConstraint => + if (!check.child.deterministic) { + check.child.failAnalysis( + errorClass = "NON_DETERMINISTIC_CHECK_CONSTRAINT", + messageParameters = Map("checkCondition" -> check.condition) + ) + } + case _ => } + val newTableSpec = TableSpec( properties = u.properties, provider = u.provider, @@ -140,7 +107,7 @@ object ResolveTableSpec extends Rule[LogicalPlan] { collation = u.collation, serde = u.serde, external = u.external, - constraints = newConstraints.map(_.toV2Constraint(isCreateTable = true))) + constraints = u.constraints.tableConstraints.map(_.toV2Constraint(isCreateTable = true))) withNewSpec(newTableSpec) case _ => input diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/constraints.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/constraints.scala index 05647e4b8306f..d13907eb4f47d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/constraints.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/constraints.scala @@ -19,13 +19,12 @@ package org.apache.spark.sql.catalyst.expressions import java.util.UUID import org.apache.spark.SparkUnsupportedOperationException - import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.catalyst.trees.CurrentOrigin import org.apache.spark.sql.catalyst.util.V2ExpressionBuilder import org.apache.spark.sql.connector.catalog.constraints.Constraint import org.apache.spark.sql.connector.expressions.FieldReference -import org.apache.spark.sql.types.{ArrayType, DataType, StringType} +import org.apache.spark.sql.types.DataType trait TableConstraint extends Expression with Unevaluable { // Convert to a data source v2 constraint @@ -271,6 +270,12 @@ case class ForeignKeyConstraint( } case class Constraints(children: Seq[Expression]) extends Expression with Unevaluable { + assert(children.forall(_.isInstanceOf[TableConstraint]), + "All children of Constraints must be TableConstraints") + + def tableConstraints: Seq[TableConstraint] = { + children.map(_.asInstanceOf[TableConstraint]) + } override def nullable: Boolean = true @@ -279,4 +284,8 @@ case class Constraints(children: Seq[Expression]) extends Expression with Uneval override protected def withNewChildrenInternal(newChildren: IndexedSeq[Expression]): Expression = copy(children = newChildren) -} \ No newline at end of file +} + +object Constraints { + val empty: Constraints = Constraints(Seq.empty) +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index cccab1b9b0e08..61e74ebdd9eca 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -4876,7 +4876,7 @@ class AstBuilder extends DataTypeAstBuilder val namedConstraints = constraints.map(c => c.withTableName(identifiers.last)) val tableSpec = UnresolvedTableSpec(properties, provider, options, location, comment, - collation, serdeInfo, external, namedConstraints) + collation, serdeInfo, external, Constraints(namedConstraints)) val identifier = withOrigin(identifierContext) { UnresolvedIdentifier(identifiers) } @@ -4955,7 +4955,7 @@ class AstBuilder extends DataTypeAstBuilder val namedConstraints = constraints.map(c => c.withTableName(identifiers.last)) val tableSpec = UnresolvedTableSpec(properties, provider, options, location, comment, - collation, serdeInfo, external = false, namedConstraints) + collation, serdeInfo, external = false, Constraints(namedConstraints)) val identifier = withOrigin(identifierContext) { UnresolvedIdentifier(identifiers) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/CreateTablePartitioningValidationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/CreateTablePartitioningValidationSuite.scala index 0afdffb8b5e7c..bf94758267fb9 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/CreateTablePartitioningValidationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/CreateTablePartitioningValidationSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.analysis import java.util -import org.apache.spark.sql.catalyst.expressions.AttributeReference +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Constraints} import org.apache.spark.sql.catalyst.plans.logical.{CreateTableAsSelect, LeafNode, OptionList, UnresolvedTableSpec} import org.apache.spark.sql.catalyst.types.DataTypeUtils import org.apache.spark.sql.connector.catalog.{InMemoryTableCatalog, Table, TableCapability, TableCatalog} @@ -31,7 +31,7 @@ import org.apache.spark.util.ArrayImplicits._ class CreateTablePartitioningValidationSuite extends AnalysisTest { val tableSpec = UnresolvedTableSpec(Map.empty, None, OptionList(Seq.empty), None, None, None, None, false, - Seq.empty) + Constraints.empty) test("CreateTableAsSelect: fail missing top-level column") { val plan = CreateTableAsSelect( UnresolvedIdentifier(Array("table_name").toImmutableArraySeq), diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala index 1589bcb8a3d7e..6d4df1432d879 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala @@ -22,7 +22,7 @@ import java.util.Locale import org.apache.spark.SparkThrowable import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.analysis._ -import org.apache.spark.sql.catalyst.expressions.{EqualTo, Hex, Literal} +import org.apache.spark.sql.catalyst.expressions.{Constraints, EqualTo, Hex, Literal} import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.connector.catalog.IdentityColumnSpec import org.apache.spark.sql.connector.catalog.TableChange.ColumnPosition.{after, first} @@ -2705,7 +2705,7 @@ class DDLParserSuite extends AnalysisTest { val createTableResult = CreateTable(UnresolvedIdentifier(Seq("my_tab")), columnsWithDefaultValue, Seq.empty[Transform], UnresolvedTableSpec(Map.empty[String, String], Some("parquet"), - OptionList(Seq.empty), None, None, None, None, false, Seq.empty), false) + OptionList(Seq.empty), None, None, None, None, false, Constraints.empty), false) // Parse the CREATE TABLE statement twice, swapping the order of the NOT NULL and DEFAULT // options, to make sure that the parser accepts any ordering of these options. comparePlans(parsePlan( @@ -2718,7 +2718,7 @@ class DDLParserSuite extends AnalysisTest { "b STRING NOT NULL DEFAULT 'abc') USING parquet"), ReplaceTable(UnresolvedIdentifier(Seq("my_tab")), columnsWithDefaultValue, Seq.empty[Transform], UnresolvedTableSpec(Map.empty[String, String], Some("parquet"), - OptionList(Seq.empty), None, None, None, None, false, Seq.empty), false)) + OptionList(Seq.empty), None, None, None, None, false, Constraints.empty), false)) // These ALTER TABLE statements should parse successfully. comparePlans( parsePlan("ALTER TABLE t1 ADD COLUMN x int NOT NULL DEFAULT 42"), @@ -2881,12 +2881,12 @@ class DDLParserSuite extends AnalysisTest { "CREATE TABLE my_tab(a INT, b INT NOT NULL GENERATED ALWAYS AS (a+1)) USING parquet"), CreateTable(UnresolvedIdentifier(Seq("my_tab")), columnsWithGenerationExpr, Seq.empty[Transform], UnresolvedTableSpec(Map.empty[String, String], Some("parquet"), - OptionList(Seq.empty), None, None, None, None, false, Seq.empty), false)) + OptionList(Seq.empty), None, None, None, None, false, Constraints.empty), false)) comparePlans(parsePlan( "REPLACE TABLE my_tab(a INT, b INT NOT NULL GENERATED ALWAYS AS (a+1)) USING parquet"), ReplaceTable(UnresolvedIdentifier(Seq("my_tab")), columnsWithGenerationExpr, Seq.empty[Transform], UnresolvedTableSpec(Map.empty[String, String], Some("parquet"), - OptionList(Seq.empty), None, None, None, None, false, Seq.empty), false)) + OptionList(Seq.empty), None, None, None, None, false, Constraints.empty), false)) // Two generation expressions checkError( exception = parseException("CREATE TABLE my_tab(a INT, " + @@ -2958,7 +2958,7 @@ class DDLParserSuite extends AnalysisTest { None, None, false, - Seq.empty + Constraints.empty ), false ) @@ -2982,7 +2982,7 @@ class DDLParserSuite extends AnalysisTest { None, None, false, - Seq.empty + Constraints.empty ), false ) @@ -3275,7 +3275,7 @@ class DDLParserSuite extends AnalysisTest { Seq(ColumnDefinition("c", StringType)), Seq.empty[Transform], UnresolvedTableSpec(Map.empty[String, String], Some("parquet"), OptionList(Seq.empty), - None, None, Some(collation), None, false, Seq.empty), false)) + None, None, Some(collation), None, false, Constraints.empty), false)) } } @@ -3287,7 +3287,7 @@ class DDLParserSuite extends AnalysisTest { Seq(ColumnDefinition("c", StringType)), Seq.empty[Transform], UnresolvedTableSpec(Map.empty[String, String], Some("parquet"), OptionList(Seq.empty), - None, None, Some(collation), None, false, Seq.empty), false)) + None, None, Some(collation), None, false, Constraints.empty), false)) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/classic/Catalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/classic/Catalog.scala index 16ac140f091a3..e7914f3201128 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/classic/Catalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/classic/Catalog.scala @@ -30,7 +30,7 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder -import org.apache.spark.sql.catalyst.expressions.{Expression, Literal} +import org.apache.spark.sql.catalyst.expressions.{Constraints, Expression, Literal} import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.catalyst.plans.logical.{ColumnDefinition, CreateTable, LocalRelation, LogicalPlan, OptionList, RecoverPartitions, ShowFunctions, ShowTables, UnresolvedTableSpec, View} import org.apache.spark.sql.catalyst.types.DataTypeUtils @@ -701,7 +701,7 @@ class Catalog(sparkSession: SparkSession) extends catalog.Catalog with Logging { collation = None, serde = None, external = tableType == CatalogTableType.EXTERNAL, - constraints = Seq.empty) + constraints = Constraints.empty) val plan = CreateTable( name = UnresolvedIdentifier(ident), diff --git a/sql/core/src/main/scala/org/apache/spark/sql/classic/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/classic/DataFrameWriter.scala index 501b4985128dd..6733f31ec0171 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/classic/DataFrameWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/classic/DataFrameWriter.scala @@ -27,7 +27,7 @@ import org.apache.spark.sql.SaveMode import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.{EliminateSubqueryAliases, NoSuchTableException, UnresolvedIdentifier, UnresolvedRelation} import org.apache.spark.sql.catalyst.catalog._ -import org.apache.spark.sql.catalyst.expressions.Literal +import org.apache.spark.sql.catalyst.expressions.{Constraints, Literal} import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap import org.apache.spark.sql.connector.catalog._ @@ -214,7 +214,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) extends sql.DataFram collation = extraOptions.get(TableCatalog.PROP_COLLATION), serde = None, external = false, - constraints = Seq.empty) + constraints = Constraints.empty) runCommand(df.sparkSession) { CreateTableAsSelect( UnresolvedIdentifier( @@ -480,7 +480,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) extends sql.DataFram collation = extraOptions.get(TableCatalog.PROP_COLLATION), serde = None, external = false, - constraints = Seq.empty) + constraints = Constraints.empty) ReplaceTableAsSelect( UnresolvedIdentifier(nameParts), partitioningAsV2, @@ -502,7 +502,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) extends sql.DataFram collation = extraOptions.get(TableCatalog.PROP_COLLATION), serde = None, external = false, - constraints = Seq.empty) + constraints = Constraints.empty) CreateTableAsSelect( UnresolvedIdentifier(nameParts), diff --git a/sql/core/src/main/scala/org/apache/spark/sql/classic/DataFrameWriterV2.scala b/sql/core/src/main/scala/org/apache/spark/sql/classic/DataFrameWriterV2.scala index c6eacfe8f1ed9..defd14767324e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/classic/DataFrameWriterV2.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/classic/DataFrameWriterV2.scala @@ -26,7 +26,7 @@ import org.apache.spark.annotation.Experimental import org.apache.spark.sql import org.apache.spark.sql.Column import org.apache.spark.sql.catalyst.analysis.{NoSuchTableException, UnresolvedFunction, UnresolvedIdentifier, UnresolvedRelation} -import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, Literal} +import org.apache.spark.sql.catalyst.expressions.{Attribute, Constraints, Expression, Literal} import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap import org.apache.spark.sql.connector.catalog.TableWritePrivilege._ @@ -170,7 +170,7 @@ final class DataFrameWriterV2[T] private[sql](table: String, ds: Dataset[T]) collation = None, serde = None, external = false, - constraints = Seq.empty) + constraints = Constraints.empty) } /** @inheritdoc */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/classic/DataStreamWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/classic/DataStreamWriter.scala index 471c5feadaabc..4eff6adb28b99 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/classic/DataStreamWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/classic/DataStreamWriter.scala @@ -30,6 +30,7 @@ import org.apache.spark.sql.{streaming, Dataset => DS, ForeachWriter} import org.apache.spark.sql.catalyst.analysis.UnresolvedIdentifier import org.apache.spark.sql.catalyst.catalog.{CatalogTable, CatalogTableType} import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder +import org.apache.spark.sql.catalyst.expressions.Constraints import org.apache.spark.sql.catalyst.plans.logical.{ColumnDefinition, CreateTable, OptionList, UnresolvedTableSpec} import org.apache.spark.sql.catalyst.streaming.InternalOutputModes import org.apache.spark.sql.catalyst.types.DataTypeUtils @@ -176,7 +177,7 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) extends streaming.D None, None, external = false, - constraints = Seq.empty) + constraints = Constraints.empty) val cmd = CreateTable( UnresolvedIdentifier(originalMultipartIdentifier), ds.schema.asNullable.map(ColumnDefinition.fromV1Column(_, parser)), diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/V2CommandsCaseSensitivitySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/V2CommandsCaseSensitivitySuite.scala index 300492577b1fe..1f012099bda46 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/V2CommandsCaseSensitivitySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/V2CommandsCaseSensitivitySuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.connector import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, CreateTablePartitioningValidationSuite, ResolvedTable, TestRelation2, TestTable2, UnresolvedFieldName, UnresolvedFieldPosition, UnresolvedIdentifier} +import org.apache.spark.sql.catalyst.expressions.Constraints import org.apache.spark.sql.catalyst.plans.logical.{AddColumns, AlterColumns, AlterColumnSpec, AlterTableCommand, CreateTableAsSelect, DropColumns, LogicalPlan, OptionList, QualifiedColType, RenameColumn, ReplaceColumns, ReplaceTableAsSelect, UnresolvedTableSpec} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.types.DataTypeUtils.toAttributes @@ -54,7 +55,7 @@ class V2CommandsCaseSensitivitySuite Seq("ID", "iD").foreach { ref => val tableSpec = UnresolvedTableSpec(Map.empty, None, OptionList(Seq.empty), - None, None, None, None, false, Seq.empty) + None, None, None, None, false, Constraints.empty) val plan = CreateTableAsSelect( UnresolvedIdentifier(Array("table_name").toImmutableArraySeq), Expressions.identity(ref) :: Nil, @@ -79,7 +80,7 @@ class V2CommandsCaseSensitivitySuite Seq("POINT.X", "point.X", "poInt.x", "poInt.X").foreach { ref => val tableSpec = UnresolvedTableSpec(Map.empty, None, OptionList(Seq.empty), - None, None, None, None, false, Seq.empty) + None, None, None, None, false, Constraints.empty) val plan = CreateTableAsSelect( UnresolvedIdentifier(Array("table_name").toImmutableArraySeq), Expressions.bucket(4, ref) :: Nil, @@ -105,7 +106,7 @@ class V2CommandsCaseSensitivitySuite Seq("ID", "iD").foreach { ref => val tableSpec = UnresolvedTableSpec(Map.empty, None, OptionList(Seq.empty), - None, None, None, None, false, Seq.empty) + None, None, None, None, false, Constraints.empty) val plan = ReplaceTableAsSelect( UnresolvedIdentifier(Array("table_name").toImmutableArraySeq), Expressions.identity(ref) :: Nil, @@ -130,7 +131,7 @@ class V2CommandsCaseSensitivitySuite Seq("POINT.X", "point.X", "poInt.x", "poInt.X").foreach { ref => val tableSpec = UnresolvedTableSpec(Map.empty, None, OptionList(Seq.empty), - None, None, None, None, false, Seq.empty) + None, None, None, None, false, Constraints.empty) val plan = ReplaceTableAsSelect( UnresolvedIdentifier(Array("table_name").toImmutableArraySeq), Expressions.bucket(4, ref) :: Nil, diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/CheckConstraintParseSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/CheckConstraintParseSuite.scala index a34d126f6f643..9391dacd513a4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/CheckConstraintParseSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/CheckConstraintParseSuite.scala @@ -270,9 +270,10 @@ class CheckConstraintParseSuite extends ConstraintParseSuiteBase { plan match { case c: CreateTable => val tableSpec = c.tableSpec.asInstanceOf[UnresolvedTableSpec] - assert(tableSpec.constraints.size == 1) - assert(tableSpec.constraints.head == constraint1.withUserProvidedName(null)) - assert(tableSpec.constraints.head.name.matches("t_chk_[0-9a-f]{7}")) + assert(tableSpec.constraints.tableConstraints.size == 1) + assert(tableSpec.constraints.tableConstraints.head == + constraint1.withUserProvidedName(null)) + assert(tableSpec.constraints.tableConstraints.head.name.matches("t_chk_[0-9a-f]{7}")) case other => fail(s"Expected CreateTable, but got: $other") @@ -289,9 +290,10 @@ class CheckConstraintParseSuite extends ConstraintParseSuiteBase { plan match { case c: ReplaceTable => val tableSpec = c.tableSpec.asInstanceOf[UnresolvedTableSpec] - assert(tableSpec.constraints.size == 1) - assert(tableSpec.constraints.head == constraint1.withUserProvidedName(null)) - assert(tableSpec.constraints.head.name.matches("t_chk_[0-9a-f]{7}")) + assert(tableSpec.constraints.tableConstraints.size == 1) + assert(tableSpec.constraints.tableConstraints.head == + constraint1.withUserProvidedName(null)) + assert(tableSpec.constraints.tableConstraints.head.name.matches("t_chk_[0-9a-f]{7}")) case other => fail(s"Expected ReplaceTable, but got: $other") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ConstraintParseSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ConstraintParseSuiteBase.scala index ea369489eb181..8ee84ae6f281b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ConstraintParseSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ConstraintParseSuiteBase.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.execution.command import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, UnresolvedIdentifier} -import org.apache.spark.sql.catalyst.expressions.{ConstraintCharacteristic, TableConstraint} +import org.apache.spark.sql.catalyst.expressions.{ConstraintCharacteristic, Constraints, TableConstraint} import org.apache.spark.sql.catalyst.parser.CatalystSqlParser.parsePlan import org.apache.spark.sql.catalyst.plans.logical.{ColumnDefinition, CreateTable, LogicalPlan, OptionList, ReplaceTable, UnresolvedTableSpec} import org.apache.spark.sql.test.SharedSparkSession @@ -56,12 +56,12 @@ abstract class ConstraintParseSuiteBase extends AnalysisTest with SharedSparkSes protected def createExpectedPlan( columns: Seq[ColumnDefinition], - constraints: Seq[TableConstraint], + tableConstraints: Seq[TableConstraint], isCreateTable: Boolean = true): LogicalPlan = { val tableId = UnresolvedIdentifier(Seq("t")) val tableSpec = UnresolvedTableSpec( Map.empty[String, String], Some("parquet"), OptionList(Seq.empty), - None, None, None, None, false, constraints) + None, None, None, None, false, Constraints(tableConstraints)) if (isCreateTable) { CreateTable(tableId, columns, Seq.empty, tableSpec, false) } else { @@ -79,7 +79,7 @@ abstract class ConstraintParseSuiteBase extends AnalysisTest with SharedSparkSes ColumnDefinition("b", StringType) ) val expected = createExpectedPlan( - columns = columns, constraints = constraints, isCreateTable = isCreateTable) + columns = columns, tableConstraints = constraints, isCreateTable = isCreateTable) comparePlans(parsed, expected) } } From d284204e350b7ff8c23068d7fb851caafaf37cc8 Mon Sep 17 00:00:00 2001 From: Gengliang Wang <gengliang@apache.org> Date: Wed, 23 Apr 2025 22:55:09 -0700 Subject: [PATCH 12/19] add comments --- .../spark/sql/catalyst/analysis/ResolveCatalogs.scala | 3 +++ .../apache/spark/sql/catalyst/expressions/constraints.scala | 6 ++++++ 2 files changed, 9 insertions(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala index 06dd8a3ffc974..2cf700976be4c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala @@ -78,6 +78,9 @@ class ResolveCatalogs(val catalogManager: CatalogManager) assertValidSessionVariableNameParts(nameParts, resolved) d.copy(name = resolved) + // For CREATE TABLE and REPLACE TABLE statements, resolve the table identifier and include + // the table columns as output. This allows expressions (e.g., constraints) referencing these + // columns to be resolved correctly. case c @ CreateTable(UnresolvedIdentifier(nameParts, allowTemp), columns, _, _, _) => val resolvedIdentifier = resolveIdentifier(nameParts, allowTemp, columns) c.copy(name = resolvedIdentifier) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/constraints.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/constraints.scala index d13907eb4f47d..f91a75faaea75 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/constraints.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/constraints.scala @@ -269,6 +269,12 @@ case class ForeignKeyConstraint( } } + +/** This class is used to represent a collection of constraints in a table. + * It is used for analysis purposes only. + * + * @param children The list of constraint expressions + */ case class Constraints(children: Seq[Expression]) extends Expression with Unevaluable { assert(children.forall(_.isInstanceOf[TableConstraint]), "All children of Constraints must be TableConstraints") From 722d0a10f957e6afea9d06212df0167171ac3759 Mon Sep 17 00:00:00 2001 From: Gengliang Wang <gengliang@apache.org> Date: Thu, 24 Apr 2025 00:56:30 -0700 Subject: [PATCH 13/19] simplify --- .../catalyst/analysis/ResolveTableSpec.scala | 6 ++-- .../catalyst/expressions/constraints.scala | 36 ++----------------- .../sql/catalyst/parser/AstBuilder.scala | 4 +-- .../catalyst/plans/logical/v2Commands.scala | 17 ++++----- ...eateTablePartitioningValidationSuite.scala | 4 +-- .../sql/catalyst/parser/DDLParserSuite.scala | 18 +++++----- .../apache/spark/sql/classic/Catalog.scala | 4 +-- .../spark/sql/classic/DataFrameWriter.scala | 8 ++--- .../spark/sql/classic/DataFrameWriterV2.scala | 4 +-- .../spark/sql/classic/DataStreamWriter.scala | 3 +- .../V2CommandsCaseSensitivitySuite.scala | 9 +++-- .../command/CheckConstraintParseSuite.scala | 12 +++---- .../command/ConstraintParseSuiteBase.scala | 4 +-- 13 files changed, 49 insertions(+), 80 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveTableSpec.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveTableSpec.scala index 961556d267cc6..7f841dd32af40 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveTableSpec.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveTableSpec.scala @@ -61,7 +61,7 @@ object ResolveTableSpec extends Rule[LogicalPlan] { input: LogicalPlan, tableSpec: TableSpecBase, withNewSpec: TableSpecBase => LogicalPlan): LogicalPlan = tableSpec match { - case u: UnresolvedTableSpec if u.optionExpression.resolved && u.constraints.resolved => + case u: UnresolvedTableSpec if u.childrenResolved => val newOptions: Seq[(String, String)] = u.optionExpression.options.map { case (key: String, null) => (key, null) @@ -87,7 +87,7 @@ object ResolveTableSpec extends Rule[LogicalPlan] { (key, newValue) } - u.constraints.children.foreach { + u.constraints.foreach { case check: CheckConstraint => if (!check.child.deterministic) { check.child.failAnalysis( @@ -107,7 +107,7 @@ object ResolveTableSpec extends Rule[LogicalPlan] { collation = u.collation, serde = u.serde, external = u.external, - constraints = u.constraints.tableConstraints.map(_.toV2Constraint(isCreateTable = true))) + constraints = u.constraints.map(_.toV2Constraint(isCreateTable = true))) withNewSpec(newTableSpec) case _ => input diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/constraints.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/constraints.scala index f91a75faaea75..d869d0ab4b127 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/constraints.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/constraints.scala @@ -166,8 +166,7 @@ case class PrimaryKeyConstraint( override val userProvidedName: String = null, override val tableName: String = null, override val userProvidedCharacteristic: ConstraintCharacteristic = ConstraintCharacteristic.empty) - extends LeafExpression - with TableConstraint { + extends LeafExpression with TableConstraint { // scalastyle:on line.size.limit override def toV2Constraint(isCreateTable: Boolean): Constraint = { @@ -199,8 +198,7 @@ case class UniqueConstraint( override val userProvidedName: String = null, override val tableName: String = null, override val userProvidedCharacteristic: ConstraintCharacteristic = ConstraintCharacteristic.empty) - extends LeafExpression - with TableConstraint { + extends LeafExpression with TableConstraint { // scalastyle:on line.size.limit override def toV2Constraint(isCreateTable: Boolean): Constraint = { @@ -236,8 +234,7 @@ case class ForeignKeyConstraint( override val userProvidedName: String = null, override val tableName: String = null, override val userProvidedCharacteristic: ConstraintCharacteristic = ConstraintCharacteristic.empty) - extends LeafExpression - with TableConstraint { + extends LeafExpression with TableConstraint { // scalastyle:on line.size.limit import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ @@ -268,30 +265,3 @@ case class ForeignKeyConstraint( copy(userProvidedCharacteristic = c) } } - - -/** This class is used to represent a collection of constraints in a table. - * It is used for analysis purposes only. - * - * @param children The list of constraint expressions - */ -case class Constraints(children: Seq[Expression]) extends Expression with Unevaluable { - assert(children.forall(_.isInstanceOf[TableConstraint]), - "All children of Constraints must be TableConstraints") - - def tableConstraints: Seq[TableConstraint] = { - children.map(_.asInstanceOf[TableConstraint]) - } - - override def nullable: Boolean = true - - override def dataType: DataType = - throw new SparkUnsupportedOperationException("CONSTRAINTS_DO_NOT_HAVE_DATA_TYPE") - - override protected def withNewChildrenInternal(newChildren: IndexedSeq[Expression]): Expression = - copy(children = newChildren) -} - -object Constraints { - val empty: Constraints = Constraints(Seq.empty) -} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 61e74ebdd9eca..cccab1b9b0e08 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -4876,7 +4876,7 @@ class AstBuilder extends DataTypeAstBuilder val namedConstraints = constraints.map(c => c.withTableName(identifiers.last)) val tableSpec = UnresolvedTableSpec(properties, provider, options, location, comment, - collation, serdeInfo, external, Constraints(namedConstraints)) + collation, serdeInfo, external, namedConstraints) val identifier = withOrigin(identifierContext) { UnresolvedIdentifier(identifiers) } @@ -4955,7 +4955,7 @@ class AstBuilder extends DataTypeAstBuilder val namedConstraints = constraints.map(c => c.withTableName(identifiers.last)) val tableSpec = UnresolvedTableSpec(properties, provider, options, location, comment, - collation, serdeInfo, external = false, Constraints(namedConstraints)) + collation, serdeInfo, external = false, namedConstraints) val identifier = withOrigin(identifierContext) { UnresolvedIdentifier(identifiers) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala index 782c2f29328c1..184ba22bd08fa 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala @@ -1504,8 +1504,8 @@ case class UnresolvedTableSpec( collation: Option[String], serde: Option[SerdeInfo], external: Boolean, - constraints: Constraints) - extends BinaryExpression with Unevaluable with TableSpecBase { + constraints: Seq[TableConstraint]) + extends Expression with Unevaluable with TableSpecBase { override def dataType: DataType = throw new SparkUnsupportedOperationException("_LEGACY_ERROR_TEMP_3113") @@ -1514,15 +1514,16 @@ case class UnresolvedTableSpec( this.copy(properties = Utils.redact(properties).toMap).toString } - override def left: Expression = optionExpression + override def nullable: Boolean = true - override def right: Expression = constraints + override def children: Seq[Expression] = optionExpression +: constraints override protected def withNewChildrenInternal( - newLeft: Expression, - newRight: Expression): Expression = - copy(optionExpression = newLeft.asInstanceOf[OptionList], - constraints = newRight.asInstanceOf[Constraints]) + newChildren: IndexedSeq[Expression]): Expression = { + copy( + optionExpression = newChildren.head.asInstanceOf[OptionList], + constraints = newChildren.tail.asInstanceOf[Seq[TableConstraint]]) + } } /** diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/CreateTablePartitioningValidationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/CreateTablePartitioningValidationSuite.scala index bf94758267fb9..0afdffb8b5e7c 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/CreateTablePartitioningValidationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/CreateTablePartitioningValidationSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.analysis import java.util -import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Constraints} +import org.apache.spark.sql.catalyst.expressions.AttributeReference import org.apache.spark.sql.catalyst.plans.logical.{CreateTableAsSelect, LeafNode, OptionList, UnresolvedTableSpec} import org.apache.spark.sql.catalyst.types.DataTypeUtils import org.apache.spark.sql.connector.catalog.{InMemoryTableCatalog, Table, TableCapability, TableCatalog} @@ -31,7 +31,7 @@ import org.apache.spark.util.ArrayImplicits._ class CreateTablePartitioningValidationSuite extends AnalysisTest { val tableSpec = UnresolvedTableSpec(Map.empty, None, OptionList(Seq.empty), None, None, None, None, false, - Constraints.empty) + Seq.empty) test("CreateTableAsSelect: fail missing top-level column") { val plan = CreateTableAsSelect( UnresolvedIdentifier(Array("table_name").toImmutableArraySeq), diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala index 6d4df1432d879..1589bcb8a3d7e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala @@ -22,7 +22,7 @@ import java.util.Locale import org.apache.spark.SparkThrowable import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.analysis._ -import org.apache.spark.sql.catalyst.expressions.{Constraints, EqualTo, Hex, Literal} +import org.apache.spark.sql.catalyst.expressions.{EqualTo, Hex, Literal} import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.connector.catalog.IdentityColumnSpec import org.apache.spark.sql.connector.catalog.TableChange.ColumnPosition.{after, first} @@ -2705,7 +2705,7 @@ class DDLParserSuite extends AnalysisTest { val createTableResult = CreateTable(UnresolvedIdentifier(Seq("my_tab")), columnsWithDefaultValue, Seq.empty[Transform], UnresolvedTableSpec(Map.empty[String, String], Some("parquet"), - OptionList(Seq.empty), None, None, None, None, false, Constraints.empty), false) + OptionList(Seq.empty), None, None, None, None, false, Seq.empty), false) // Parse the CREATE TABLE statement twice, swapping the order of the NOT NULL and DEFAULT // options, to make sure that the parser accepts any ordering of these options. comparePlans(parsePlan( @@ -2718,7 +2718,7 @@ class DDLParserSuite extends AnalysisTest { "b STRING NOT NULL DEFAULT 'abc') USING parquet"), ReplaceTable(UnresolvedIdentifier(Seq("my_tab")), columnsWithDefaultValue, Seq.empty[Transform], UnresolvedTableSpec(Map.empty[String, String], Some("parquet"), - OptionList(Seq.empty), None, None, None, None, false, Constraints.empty), false)) + OptionList(Seq.empty), None, None, None, None, false, Seq.empty), false)) // These ALTER TABLE statements should parse successfully. comparePlans( parsePlan("ALTER TABLE t1 ADD COLUMN x int NOT NULL DEFAULT 42"), @@ -2881,12 +2881,12 @@ class DDLParserSuite extends AnalysisTest { "CREATE TABLE my_tab(a INT, b INT NOT NULL GENERATED ALWAYS AS (a+1)) USING parquet"), CreateTable(UnresolvedIdentifier(Seq("my_tab")), columnsWithGenerationExpr, Seq.empty[Transform], UnresolvedTableSpec(Map.empty[String, String], Some("parquet"), - OptionList(Seq.empty), None, None, None, None, false, Constraints.empty), false)) + OptionList(Seq.empty), None, None, None, None, false, Seq.empty), false)) comparePlans(parsePlan( "REPLACE TABLE my_tab(a INT, b INT NOT NULL GENERATED ALWAYS AS (a+1)) USING parquet"), ReplaceTable(UnresolvedIdentifier(Seq("my_tab")), columnsWithGenerationExpr, Seq.empty[Transform], UnresolvedTableSpec(Map.empty[String, String], Some("parquet"), - OptionList(Seq.empty), None, None, None, None, false, Constraints.empty), false)) + OptionList(Seq.empty), None, None, None, None, false, Seq.empty), false)) // Two generation expressions checkError( exception = parseException("CREATE TABLE my_tab(a INT, " + @@ -2958,7 +2958,7 @@ class DDLParserSuite extends AnalysisTest { None, None, false, - Constraints.empty + Seq.empty ), false ) @@ -2982,7 +2982,7 @@ class DDLParserSuite extends AnalysisTest { None, None, false, - Constraints.empty + Seq.empty ), false ) @@ -3275,7 +3275,7 @@ class DDLParserSuite extends AnalysisTest { Seq(ColumnDefinition("c", StringType)), Seq.empty[Transform], UnresolvedTableSpec(Map.empty[String, String], Some("parquet"), OptionList(Seq.empty), - None, None, Some(collation), None, false, Constraints.empty), false)) + None, None, Some(collation), None, false, Seq.empty), false)) } } @@ -3287,7 +3287,7 @@ class DDLParserSuite extends AnalysisTest { Seq(ColumnDefinition("c", StringType)), Seq.empty[Transform], UnresolvedTableSpec(Map.empty[String, String], Some("parquet"), OptionList(Seq.empty), - None, None, Some(collation), None, false, Constraints.empty), false)) + None, None, Some(collation), None, false, Seq.empty), false)) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/classic/Catalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/classic/Catalog.scala index e7914f3201128..16ac140f091a3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/classic/Catalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/classic/Catalog.scala @@ -30,7 +30,7 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder -import org.apache.spark.sql.catalyst.expressions.{Constraints, Expression, Literal} +import org.apache.spark.sql.catalyst.expressions.{Expression, Literal} import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.catalyst.plans.logical.{ColumnDefinition, CreateTable, LocalRelation, LogicalPlan, OptionList, RecoverPartitions, ShowFunctions, ShowTables, UnresolvedTableSpec, View} import org.apache.spark.sql.catalyst.types.DataTypeUtils @@ -701,7 +701,7 @@ class Catalog(sparkSession: SparkSession) extends catalog.Catalog with Logging { collation = None, serde = None, external = tableType == CatalogTableType.EXTERNAL, - constraints = Constraints.empty) + constraints = Seq.empty) val plan = CreateTable( name = UnresolvedIdentifier(ident), diff --git a/sql/core/src/main/scala/org/apache/spark/sql/classic/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/classic/DataFrameWriter.scala index 6733f31ec0171..501b4985128dd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/classic/DataFrameWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/classic/DataFrameWriter.scala @@ -27,7 +27,7 @@ import org.apache.spark.sql.SaveMode import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.{EliminateSubqueryAliases, NoSuchTableException, UnresolvedIdentifier, UnresolvedRelation} import org.apache.spark.sql.catalyst.catalog._ -import org.apache.spark.sql.catalyst.expressions.{Constraints, Literal} +import org.apache.spark.sql.catalyst.expressions.Literal import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap import org.apache.spark.sql.connector.catalog._ @@ -214,7 +214,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) extends sql.DataFram collation = extraOptions.get(TableCatalog.PROP_COLLATION), serde = None, external = false, - constraints = Constraints.empty) + constraints = Seq.empty) runCommand(df.sparkSession) { CreateTableAsSelect( UnresolvedIdentifier( @@ -480,7 +480,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) extends sql.DataFram collation = extraOptions.get(TableCatalog.PROP_COLLATION), serde = None, external = false, - constraints = Constraints.empty) + constraints = Seq.empty) ReplaceTableAsSelect( UnresolvedIdentifier(nameParts), partitioningAsV2, @@ -502,7 +502,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) extends sql.DataFram collation = extraOptions.get(TableCatalog.PROP_COLLATION), serde = None, external = false, - constraints = Constraints.empty) + constraints = Seq.empty) CreateTableAsSelect( UnresolvedIdentifier(nameParts), diff --git a/sql/core/src/main/scala/org/apache/spark/sql/classic/DataFrameWriterV2.scala b/sql/core/src/main/scala/org/apache/spark/sql/classic/DataFrameWriterV2.scala index defd14767324e..c6eacfe8f1ed9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/classic/DataFrameWriterV2.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/classic/DataFrameWriterV2.scala @@ -26,7 +26,7 @@ import org.apache.spark.annotation.Experimental import org.apache.spark.sql import org.apache.spark.sql.Column import org.apache.spark.sql.catalyst.analysis.{NoSuchTableException, UnresolvedFunction, UnresolvedIdentifier, UnresolvedRelation} -import org.apache.spark.sql.catalyst.expressions.{Attribute, Constraints, Expression, Literal} +import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, Literal} import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap import org.apache.spark.sql.connector.catalog.TableWritePrivilege._ @@ -170,7 +170,7 @@ final class DataFrameWriterV2[T] private[sql](table: String, ds: Dataset[T]) collation = None, serde = None, external = false, - constraints = Constraints.empty) + constraints = Seq.empty) } /** @inheritdoc */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/classic/DataStreamWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/classic/DataStreamWriter.scala index 4eff6adb28b99..471c5feadaabc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/classic/DataStreamWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/classic/DataStreamWriter.scala @@ -30,7 +30,6 @@ import org.apache.spark.sql.{streaming, Dataset => DS, ForeachWriter} import org.apache.spark.sql.catalyst.analysis.UnresolvedIdentifier import org.apache.spark.sql.catalyst.catalog.{CatalogTable, CatalogTableType} import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder -import org.apache.spark.sql.catalyst.expressions.Constraints import org.apache.spark.sql.catalyst.plans.logical.{ColumnDefinition, CreateTable, OptionList, UnresolvedTableSpec} import org.apache.spark.sql.catalyst.streaming.InternalOutputModes import org.apache.spark.sql.catalyst.types.DataTypeUtils @@ -177,7 +176,7 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) extends streaming.D None, None, external = false, - constraints = Constraints.empty) + constraints = Seq.empty) val cmd = CreateTable( UnresolvedIdentifier(originalMultipartIdentifier), ds.schema.asNullable.map(ColumnDefinition.fromV1Column(_, parser)), diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/V2CommandsCaseSensitivitySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/V2CommandsCaseSensitivitySuite.scala index 1f012099bda46..300492577b1fe 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/V2CommandsCaseSensitivitySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/V2CommandsCaseSensitivitySuite.scala @@ -18,7 +18,6 @@ package org.apache.spark.sql.connector import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, CreateTablePartitioningValidationSuite, ResolvedTable, TestRelation2, TestTable2, UnresolvedFieldName, UnresolvedFieldPosition, UnresolvedIdentifier} -import org.apache.spark.sql.catalyst.expressions.Constraints import org.apache.spark.sql.catalyst.plans.logical.{AddColumns, AlterColumns, AlterColumnSpec, AlterTableCommand, CreateTableAsSelect, DropColumns, LogicalPlan, OptionList, QualifiedColType, RenameColumn, ReplaceColumns, ReplaceTableAsSelect, UnresolvedTableSpec} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.types.DataTypeUtils.toAttributes @@ -55,7 +54,7 @@ class V2CommandsCaseSensitivitySuite Seq("ID", "iD").foreach { ref => val tableSpec = UnresolvedTableSpec(Map.empty, None, OptionList(Seq.empty), - None, None, None, None, false, Constraints.empty) + None, None, None, None, false, Seq.empty) val plan = CreateTableAsSelect( UnresolvedIdentifier(Array("table_name").toImmutableArraySeq), Expressions.identity(ref) :: Nil, @@ -80,7 +79,7 @@ class V2CommandsCaseSensitivitySuite Seq("POINT.X", "point.X", "poInt.x", "poInt.X").foreach { ref => val tableSpec = UnresolvedTableSpec(Map.empty, None, OptionList(Seq.empty), - None, None, None, None, false, Constraints.empty) + None, None, None, None, false, Seq.empty) val plan = CreateTableAsSelect( UnresolvedIdentifier(Array("table_name").toImmutableArraySeq), Expressions.bucket(4, ref) :: Nil, @@ -106,7 +105,7 @@ class V2CommandsCaseSensitivitySuite Seq("ID", "iD").foreach { ref => val tableSpec = UnresolvedTableSpec(Map.empty, None, OptionList(Seq.empty), - None, None, None, None, false, Constraints.empty) + None, None, None, None, false, Seq.empty) val plan = ReplaceTableAsSelect( UnresolvedIdentifier(Array("table_name").toImmutableArraySeq), Expressions.identity(ref) :: Nil, @@ -131,7 +130,7 @@ class V2CommandsCaseSensitivitySuite Seq("POINT.X", "point.X", "poInt.x", "poInt.X").foreach { ref => val tableSpec = UnresolvedTableSpec(Map.empty, None, OptionList(Seq.empty), - None, None, None, None, false, Constraints.empty) + None, None, None, None, false, Seq.empty) val plan = ReplaceTableAsSelect( UnresolvedIdentifier(Array("table_name").toImmutableArraySeq), Expressions.bucket(4, ref) :: Nil, diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/CheckConstraintParseSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/CheckConstraintParseSuite.scala index 9391dacd513a4..124970a49a2c5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/CheckConstraintParseSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/CheckConstraintParseSuite.scala @@ -270,10 +270,10 @@ class CheckConstraintParseSuite extends ConstraintParseSuiteBase { plan match { case c: CreateTable => val tableSpec = c.tableSpec.asInstanceOf[UnresolvedTableSpec] - assert(tableSpec.constraints.tableConstraints.size == 1) - assert(tableSpec.constraints.tableConstraints.head == + assert(tableSpec.constraints.size == 1) + assert(tableSpec.constraints.head == constraint1.withUserProvidedName(null)) - assert(tableSpec.constraints.tableConstraints.head.name.matches("t_chk_[0-9a-f]{7}")) + assert(tableSpec.constraints.head.name.matches("t_chk_[0-9a-f]{7}")) case other => fail(s"Expected CreateTable, but got: $other") @@ -290,10 +290,10 @@ class CheckConstraintParseSuite extends ConstraintParseSuiteBase { plan match { case c: ReplaceTable => val tableSpec = c.tableSpec.asInstanceOf[UnresolvedTableSpec] - assert(tableSpec.constraints.tableConstraints.size == 1) - assert(tableSpec.constraints.tableConstraints.head == + assert(tableSpec.constraints.size == 1) + assert(tableSpec.constraints.head == constraint1.withUserProvidedName(null)) - assert(tableSpec.constraints.tableConstraints.head.name.matches("t_chk_[0-9a-f]{7}")) + assert(tableSpec.constraints.head.name.matches("t_chk_[0-9a-f]{7}")) case other => fail(s"Expected ReplaceTable, but got: $other") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ConstraintParseSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ConstraintParseSuiteBase.scala index 8ee84ae6f281b..8b5ddb506f786 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ConstraintParseSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ConstraintParseSuiteBase.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.execution.command import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, UnresolvedIdentifier} -import org.apache.spark.sql.catalyst.expressions.{ConstraintCharacteristic, Constraints, TableConstraint} +import org.apache.spark.sql.catalyst.expressions.{ConstraintCharacteristic, TableConstraint} import org.apache.spark.sql.catalyst.parser.CatalystSqlParser.parsePlan import org.apache.spark.sql.catalyst.plans.logical.{ColumnDefinition, CreateTable, LogicalPlan, OptionList, ReplaceTable, UnresolvedTableSpec} import org.apache.spark.sql.test.SharedSparkSession @@ -61,7 +61,7 @@ abstract class ConstraintParseSuiteBase extends AnalysisTest with SharedSparkSes val tableId = UnresolvedIdentifier(Seq("t")) val tableSpec = UnresolvedTableSpec( Map.empty[String, String], Some("parquet"), OptionList(Seq.empty), - None, None, None, None, false, Constraints(tableConstraints)) + None, None, None, None, false, tableConstraints) if (isCreateTable) { CreateTable(tableId, columns, Seq.empty, tableSpec, false) } else { From e7d98d9fdc5e50c0db1ea6c73014e0a5c10ddc20 Mon Sep 17 00:00:00 2001 From: Gengliang Wang <gengliang@apache.org> Date: Thu, 24 Apr 2025 11:57:24 -0700 Subject: [PATCH 14/19] change validation status of check --- .../sql/catalyst/expressions/constraints.scala | 9 ++------- .../command/v2/CheckConstraintSuite.scala | 14 +++++++------- 2 files changed, 9 insertions(+), 14 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/constraints.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/constraints.scala index d869d0ab4b127..65c0c255a303e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/constraints.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/constraints.scala @@ -126,13 +126,8 @@ case class CheckConstraint( val predicate = new V2ExpressionBuilder(child, true).buildPredicate().orNull val enforced = userProvidedCharacteristic.enforced.getOrElse(true) val rely = userProvidedCharacteristic.rely.getOrElse(false) - // The validation status is set to UNVALIDATED for create table and - // VALID for alter table. - val validateStatus = if (isCreateTable) { - Constraint.ValidationStatus.UNVALIDATED - } else { - Constraint.ValidationStatus.VALID - } + // TODO(SPARK-51903): Change the status to VALIDATED when we support validation on ALTER TABLE + val validateStatus = Constraint.ValidationStatus.UNVALIDATED Constraint .check(name) .predicateSql(condition) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/CheckConstraintSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/CheckConstraintSuite.scala index 70d4a5e92b6f2..4b64b4c26ffe8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/CheckConstraintSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/CheckConstraintSuite.scala @@ -132,14 +132,14 @@ class CheckConstraintSuite extends QueryTest with CommandSuiteBase with DDLComma val constraint = getCheckConstraint(table) assert(constraint.name() == "c1") assert(constraint.toDDL == - "CONSTRAINT c1 CHECK (from_json(j, 'a INT').a > 1) ENFORCED VALID NORELY") + "CONSTRAINT c1 CHECK (from_json(j, 'a INT').a > 1) ENFORCED UNVALIDATED NORELY") assert(constraint.predicateSql() == "from_json(j, 'a INT').a > 1") assert(constraint.predicate() == null) } } - def getConstraintCharacteristics(isCreateTable: Boolean): Seq[(String, String)] = { - val validStatus = if (isCreateTable) "UNVALIDATED" else "VALID" + def getConstraintCharacteristics(): Seq[(String, String)] = { + val validStatus = "UNVALIDATED" Seq( ("", s"ENFORCED $validStatus NORELY"), ("NOT ENFORCED", s"NOT ENFORCED $validStatus NORELY"), @@ -155,7 +155,7 @@ class CheckConstraintSuite extends QueryTest with CommandSuiteBase with DDLComma } test("Create table with check constraint") { - getConstraintCharacteristics(true).foreach { case (characteristic, expectedDDL) => + getConstraintCharacteristics().foreach { case (characteristic, expectedDDL) => withNamespaceAndTable("ns", "tbl", nonPartitionCatalog) { t => val constraintStr = s"CONSTRAINT c1 CHECK (id > 0) $characteristic" sql(s"CREATE TABLE $t (id bigint, data string, $constraintStr) $defaultUsing") @@ -168,7 +168,7 @@ class CheckConstraintSuite extends QueryTest with CommandSuiteBase with DDLComma } test("Replace table with check constraint") { - getConstraintCharacteristics(true).foreach { case (characteristic, expectedDDL) => + getConstraintCharacteristics().foreach { case (characteristic, expectedDDL) => withNamespaceAndTable("ns", "tbl", nonPartitionCatalog) { t => val constraintStr = s"CONSTRAINT c1 CHECK (id > 0) $characteristic" sql(s"CREATE TABLE $t (id bigint) $defaultUsing") @@ -182,7 +182,7 @@ class CheckConstraintSuite extends QueryTest with CommandSuiteBase with DDLComma } test("Alter table add check constraint") { - getConstraintCharacteristics(false).foreach { case (characteristic, expectedDDL) => + getConstraintCharacteristics().foreach { case (characteristic, expectedDDL) => withNamespaceAndTable("ns", "tbl", nonPartitionCatalog) { t => sql(s"CREATE TABLE $t (id bigint, data string) $defaultUsing") assert(loadTable(nonPartitionCatalog, "ns", "tbl").constraints.isEmpty) @@ -212,7 +212,7 @@ class CheckConstraintSuite extends QueryTest with CommandSuiteBase with DDLComma condition = "CONSTRAINT_ALREADY_EXISTS", sqlState = "42710", parameters = Map("constraintName" -> "abc", - "oldConstraint" -> "CONSTRAINT abc CHECK (id > 0) ENFORCED VALID NORELY") + "oldConstraint" -> "CONSTRAINT abc CHECK (id > 0) ENFORCED UNVALIDATED NORELY") ) } } From 5101f2e4fcccfd4a39ca9cf85f0f6c942155ba6b Mon Sep 17 00:00:00 2001 From: Gengliang Wang <gengliang@apache.org> Date: Mon, 28 Apr 2025 10:35:56 -0700 Subject: [PATCH 15/19] address comments --- .../catalyst/analysis/ResolveCatalogs.scala | 2 +- .../catalyst/analysis/ResolveTableSpec.scala | 2 +- .../catalyst/expressions/constraints.scala | 19 +++++++++---------- .../plans/logical/v2AlterTableCommands.scala | 2 +- .../command/CheckConstraintParseSuite.scala | 19 ++++++++----------- 5 files changed, 20 insertions(+), 24 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala index 2cf700976be4c..70ee71bbbfb6d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala @@ -90,7 +90,7 @@ class ResolveCatalogs(val catalogManager: CatalogManager) r.copy(name = resolvedIdentifier) case UnresolvedIdentifier(nameParts, allowTemp) => - resolveIdentifier(nameParts, allowTemp, Seq.empty) + resolveIdentifier(nameParts, allowTemp, Nil) case CurrentNamespace => ResolvedNamespace(currentCatalog, catalogManager.currentNamespace.toImmutableArraySeq) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveTableSpec.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveTableSpec.scala index 7f841dd32af40..2b66ef503822c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveTableSpec.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveTableSpec.scala @@ -107,7 +107,7 @@ object ResolveTableSpec extends Rule[LogicalPlan] { collation = u.collation, serde = u.serde, external = u.external, - constraints = u.constraints.map(_.toV2Constraint(isCreateTable = true))) + constraints = u.constraints.map(_.toV2Constraint)) withNewSpec(newTableSpec) case _ => input diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/constraints.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/constraints.scala index 65c0c255a303e..3127ef1359bea 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/constraints.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/constraints.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.catalyst.expressions import java.util.UUID -import org.apache.spark.SparkUnsupportedOperationException +import org.apache.spark.sql.catalyst.analysis.UnresolvedException import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.catalyst.trees.CurrentOrigin import org.apache.spark.sql.catalyst.util.V2ExpressionBuilder @@ -27,8 +27,8 @@ import org.apache.spark.sql.connector.expressions.FieldReference import org.apache.spark.sql.types.DataType trait TableConstraint extends Expression with Unevaluable { - // Convert to a data source v2 constraint - def toV2Constraint(isCreateTable: Boolean): Constraint + /** Convert to a data source v2 constraint */ + def toV2Constraint: Constraint /** Returns the user-provided name of the constraint */ def userProvidedName: String @@ -99,10 +99,9 @@ trait TableConstraint extends Expression with Unevaluable { } } - override def nullable: Boolean = true + override def nullable: Boolean = false - override def dataType: DataType = - throw new SparkUnsupportedOperationException("CONSTRAINT_DOES_NOT_HAVE_DATA_TYPE") + override def dataType: DataType = throw new UnresolvedException("dataType") } case class ConstraintCharacteristic(enforced: Option[Boolean], rely: Option[Boolean]) @@ -122,7 +121,7 @@ case class CheckConstraint( with TableConstraint { // scalastyle:on line.size.limit - def toV2Constraint(isCreateTable: Boolean): Constraint = { + def toV2Constraint: Constraint = { val predicate = new V2ExpressionBuilder(child, true).buildPredicate().orNull val enforced = userProvidedCharacteristic.enforced.getOrElse(true) val rely = userProvidedCharacteristic.rely.getOrElse(false) @@ -164,7 +163,7 @@ case class PrimaryKeyConstraint( extends LeafExpression with TableConstraint { // scalastyle:on line.size.limit - override def toV2Constraint(isCreateTable: Boolean): Constraint = { + override def toV2Constraint: Constraint = { val enforced = userProvidedCharacteristic.enforced.getOrElse(false) val rely = userProvidedCharacteristic.rely.getOrElse(false) Constraint @@ -196,7 +195,7 @@ case class UniqueConstraint( extends LeafExpression with TableConstraint { // scalastyle:on line.size.limit - override def toV2Constraint(isCreateTable: Boolean): Constraint = { + override def toV2Constraint: Constraint = { val enforced = userProvidedCharacteristic.enforced.getOrElse(false) val rely = userProvidedCharacteristic.rely.getOrElse(false) Constraint @@ -234,7 +233,7 @@ case class ForeignKeyConstraint( import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ - override def toV2Constraint(isCreateTable: Boolean): Constraint = { + override def toV2Constraint: Constraint = { val enforced = userProvidedCharacteristic.enforced.getOrElse(false) val rely = userProvidedCharacteristic.rely.getOrElse(false) Constraint diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2AlterTableCommands.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2AlterTableCommands.scala index eec0a4a3c3502..bc3e86351b11b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2AlterTableCommands.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2AlterTableCommands.scala @@ -296,7 +296,7 @@ case class AddConstraint( table: LogicalPlan, tableConstraint: TableConstraint) extends AlterTableCommand { override def changes: Seq[TableChange] = { - val constraint = tableConstraint.toV2Constraint(isCreateTable = false) + val constraint = tableConstraint.toV2Constraint val validatedTableVersion = table match { case t: ResolvedTable if constraint.enforced() => t.table.currentVersion() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/CheckConstraintParseSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/CheckConstraintParseSuite.scala index 124970a49a2c5..2df42b1b429e8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/CheckConstraintParseSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/CheckConstraintParseSuite.scala @@ -39,6 +39,8 @@ class CheckConstraintParseSuite extends ConstraintParseSuiteBase { userProvidedName = "c2", tableName = "t") + val unnamedConstraint = constraint1.withUserProvidedName(null) + test("Create table with one check constraint - table level") { val sql = "CREATE TABLE t (a INT, b STRING, CONSTRAINT c1 CHECK (a > 0)) USING parquet" verifyConstraints(sql, Seq(constraint1)) @@ -56,8 +58,7 @@ class CheckConstraintParseSuite extends ConstraintParseSuiteBase { case (enforcedStr, relyStr, characteristic) => val sql = s"CREATE TABLE t (a INT, b STRING, CONSTRAINT c1 CHECK (a > 0) " + s"$enforcedStr $relyStr) USING parquet" - val constraint = constraint1.withUserProvidedName("c1") - .withUserProvidedCharacteristic(characteristic) + val constraint = constraint1.withUserProvidedCharacteristic(characteristic) verifyConstraints(sql, Seq(constraint)) } } @@ -102,8 +103,7 @@ class CheckConstraintParseSuite extends ConstraintParseSuiteBase { case (enforcedStr, relyStr, characteristic) => val sql = s"CREATE TABLE t (a INT CONSTRAINT c1 CHECK (a > 0)" + s" $enforcedStr $relyStr, b STRING) USING parquet" - val constraint = constraint1.withUserProvidedName("c1") - .withUserProvidedCharacteristic(characteristic) + val constraint = constraint1.withUserProvidedCharacteristic(characteristic) verifyConstraints(sql, Seq(constraint)) } } @@ -151,8 +151,7 @@ class CheckConstraintParseSuite extends ConstraintParseSuiteBase { case (enforcedStr, relyStr, characteristic) => val sql = s"REPLACE TABLE t (a INT, b STRING, CONSTRAINT c1 CHECK (a > 0) " + s"$enforcedStr $relyStr) USING parquet" - val constraint = constraint1.withUserProvidedName("c1") - .withUserProvidedCharacteristic(characteristic) + val constraint = constraint1.withUserProvidedCharacteristic(characteristic) verifyConstraints(sql, Seq(constraint), isCreateTable = false) } } @@ -271,8 +270,7 @@ class CheckConstraintParseSuite extends ConstraintParseSuiteBase { case c: CreateTable => val tableSpec = c.tableSpec.asInstanceOf[UnresolvedTableSpec] assert(tableSpec.constraints.size == 1) - assert(tableSpec.constraints.head == - constraint1.withUserProvidedName(null)) + assert(tableSpec.constraints.head == unnamedConstraint) assert(tableSpec.constraints.head.name.matches("t_chk_[0-9a-f]{7}")) case other => @@ -291,8 +289,7 @@ class CheckConstraintParseSuite extends ConstraintParseSuiteBase { case c: ReplaceTable => val tableSpec = c.tableSpec.asInstanceOf[UnresolvedTableSpec] assert(tableSpec.constraints.size == 1) - assert(tableSpec.constraints.head == - constraint1.withUserProvidedName(null)) + assert(tableSpec.constraints.head == unnamedConstraint) assert(tableSpec.constraints.head.name.matches("t_chk_[0-9a-f]{7}")) case other => @@ -311,7 +308,7 @@ class CheckConstraintParseSuite extends ConstraintParseSuiteBase { case a: AddConstraint => val table = a.table.asInstanceOf[UnresolvedTable] assert(table.multipartIdentifier == Seq("a", "b", "t")) - assert(a.tableConstraint == constraint1.withUserProvidedName(null)) + assert(a.tableConstraint == unnamedConstraint) assert(a.tableConstraint.name.matches("t_chk_[0-9a-f]{7}")) case other => From a87d4eb5f2d03c334df68d67703136cc93ec6e77 Mon Sep 17 00:00:00 2001 From: Gengliang Wang <gengliang@apache.org> Date: Mon, 28 Apr 2025 10:58:37 -0700 Subject: [PATCH 16/19] address comment --- .../org/apache/spark/sql/catalyst/expressions/constraints.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/constraints.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/constraints.scala index 3127ef1359bea..15b8b7e2e7319 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/constraints.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/constraints.scala @@ -99,7 +99,7 @@ trait TableConstraint extends Expression with Unevaluable { } } - override def nullable: Boolean = false + override def nullable: Boolean = throw new UnresolvedException("nullable") override def dataType: DataType = throw new UnresolvedException("dataType") } From 59551c8ebe680a11ce46782705fad96cc318a3b4 Mon Sep 17 00:00:00 2001 From: Gengliang Wang <gengliang@apache.org> Date: Mon, 28 Apr 2025 15:59:11 -0700 Subject: [PATCH 17/19] fix test and add tests --- .../catalyst/analysis/ResolveCatalogs.scala | 8 ++++++- .../TableOptionsConstantFoldingSuite.scala | 10 ++++---- .../command/v2/CheckConstraintSuite.scala | 23 ++++++++++++++++++- .../v2/ForeignKeyConstraintSuite.scala | 2 +- .../v2/PrimaryKeyConstraintSuite.scala | 2 +- .../command/v2/UniqueConstraintSuite.scala | 2 +- 6 files changed, 38 insertions(+), 9 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala index 70ee71bbbfb6d..6423b96a37c1e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala @@ -28,6 +28,7 @@ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.connector.catalog.{CatalogManager, CatalogPlugin, Identifier, LookupCatalog, SupportsNamespaces} import org.apache.spark.sql.errors.DataTypeErrors.toSQLId import org.apache.spark.sql.errors.QueryCompilationErrors +import org.apache.spark.sql.types.{CharType, StringType, VarcharType} import org.apache.spark.util.ArrayImplicits._ /** @@ -105,7 +106,12 @@ class ResolveCatalogs(val catalogManager: CatalogManager) allowTemp: Boolean, columns: Seq[ColumnDefinition]): ResolvedIdentifier = { val columnOutput = columns.map { col => - AttributeReference(col.name, col.dataType, col.nullable, col.metadata)() + val dataType = col.dataType match { + case _: CharType | _: VarcharType if !conf.preserveCharVarcharTypeInfo => + StringType + case _ => col.dataType + } + AttributeReference(col.name, dataType, col.nullable, col.metadata)() } if (allowTemp && catalogManager.v1SessionCatalog.isTempView(nameParts)) { val ident = Identifier.of(nameParts.dropRight(1).toArray, nameParts.last) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/TableOptionsConstantFoldingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/TableOptionsConstantFoldingSuite.scala index aa82ac57089f0..a98a7faf4884f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/TableOptionsConstantFoldingSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/TableOptionsConstantFoldingSuite.scala @@ -70,16 +70,18 @@ class TableOptionsConstantFoldingSuite extends QueryTest with SharedSparkSession checkError( exception = intercept[AnalysisException]( sql(s"$prefix ('k' = 1 + 2 + unresolvedAttribute)")), - condition = "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", parameters = Map( - "objectName" -> "`unresolvedAttribute`"), + "objectName" -> "`unresolvedAttribute`", + "proposal" -> "`col`"), queryContext = Array(ExpectedContext("", "", 60, 78, "unresolvedAttribute"))) checkError( exception = intercept[AnalysisException]( sql(s"$prefix ('k' = true or false or unresolvedAttribute)")), - condition = "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", + condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION", parameters = Map( - "objectName" -> "`unresolvedAttribute`"), + "objectName" -> "`unresolvedAttribute`", + "proposal" -> "`col`"), queryContext = Array(ExpectedContext("", "", 69, 87, "unresolvedAttribute"))) checkError( exception = intercept[AnalysisException]( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/CheckConstraintSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/CheckConstraintSuite.scala index 4b64b4c26ffe8..ee83614eb6e5f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/CheckConstraintSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/CheckConstraintSuite.scala @@ -21,9 +21,10 @@ import org.apache.spark.sql.{AnalysisException, QueryTest} import org.apache.spark.sql.connector.catalog.Table import org.apache.spark.sql.connector.catalog.constraints.Check import org.apache.spark.sql.execution.command.DDLCommandTestUtils +import org.apache.spark.sql.internal.SQLConf class CheckConstraintSuite extends QueryTest with CommandSuiteBase with DDLCommandTestUtils { - override protected def command: String = "ALTER TABLE .. ADD CONSTRAINT" + override protected def command: String = "Check CONSTRAINT" test("Nondeterministic expression -- alter table") { withTable("t") { @@ -167,6 +168,26 @@ class CheckConstraintSuite extends QueryTest with CommandSuiteBase with DDLComma } } + test("Create table with check constraint: char/varchar type") { + Seq(true, false).foreach { preserveCharVarcharTypeInfo => + withSQLConf(SQLConf.PRESERVE_CHAR_VARCHAR_TYPE_INFO.key -> + preserveCharVarcharTypeInfo.toString) { + Seq("CHAR(10)", "VARCHAR(10)").foreach { dt => + withNamespaceAndTable("ns", "tbl", nonPartitionCatalog) { t => + val constraintStr = "CONSTRAINT c1 CHECK (LENGTH(name) > 0)" + sql(s"CREATE TABLE $t (id bigint, name $dt, $constraintStr) $defaultUsing") + val table = loadTable(nonPartitionCatalog, "ns", "tbl") + val constraint = getCheckConstraint(table) + assert(constraint.name() == "c1") + assert(constraint.toDDL == + s"CONSTRAINT c1 CHECK (LENGTH(name) > 0) ENFORCED UNVALIDATED NORELY") + assert(constraint.predicateSql() == "LENGTH(name) > 0") + } + } + } + } + } + test("Replace table with check constraint") { getConstraintCharacteristics().foreach { case (characteristic, expectedDDL) => withNamespaceAndTable("ns", "tbl", nonPartitionCatalog) { t => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ForeignKeyConstraintSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ForeignKeyConstraintSuite.scala index 5ef541074e1ae..02646a3cfcbb0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ForeignKeyConstraintSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ForeignKeyConstraintSuite.scala @@ -20,7 +20,7 @@ import org.apache.spark.sql.{AnalysisException, QueryTest} import org.apache.spark.sql.execution.command.DDLCommandTestUtils class ForeignKeyConstraintSuite extends QueryTest with CommandSuiteBase with DDLCommandTestUtils { - override protected def command: String = "ALTER TABLE .. ADD CONSTRAINT" + override protected def command: String = "FOREIGN KEY CONSTRAINT" private val validConstraintCharacteristics = Seq( ("", "NOT ENFORCED UNVALIDATED NORELY"), diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/PrimaryKeyConstraintSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/PrimaryKeyConstraintSuite.scala index 79763c059d151..a4785e953a2d8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/PrimaryKeyConstraintSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/PrimaryKeyConstraintSuite.scala @@ -20,7 +20,7 @@ import org.apache.spark.sql.{AnalysisException, QueryTest} import org.apache.spark.sql.execution.command.DDLCommandTestUtils class PrimaryKeyConstraintSuite extends QueryTest with CommandSuiteBase with DDLCommandTestUtils { - override protected def command: String = "ALTER TABLE .. ADD CONSTRAINT" + override protected def command: String = "PRIMARY KEY CONSTRAINT" private val validConstraintCharacteristics = Seq( ("", "NOT ENFORCED UNVALIDATED NORELY"), diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/UniqueConstraintSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/UniqueConstraintSuite.scala index bd290657e3e21..9446cbc6ade22 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/UniqueConstraintSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/UniqueConstraintSuite.scala @@ -20,7 +20,7 @@ import org.apache.spark.sql.{AnalysisException, QueryTest} import org.apache.spark.sql.execution.command.DDLCommandTestUtils class UniqueConstraintSuite extends QueryTest with CommandSuiteBase with DDLCommandTestUtils { - override protected def command: String = "ALTER TABLE .. ADD CONSTRAINT" + override protected def command: String = "UNIQUE CONSTRAINT" private val validConstraintCharacteristics = Seq( ("", "NOT ENFORCED UNVALIDATED NORELY"), From 8044c9b5164be76308daca6f2b397e12d3320ab9 Mon Sep 17 00:00:00 2001 From: Gengliang Wang <gengliang@apache.org> Date: Mon, 28 Apr 2025 21:59:20 -0700 Subject: [PATCH 18/19] recursively replace --- .../sql/catalyst/analysis/ResolveCatalogs.scala | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala index 6423b96a37c1e..b426ad6165739 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala @@ -25,10 +25,10 @@ import org.apache.spark.sql.catalyst.SqlScriptingLocalVariableManager import org.apache.spark.sql.catalyst.expressions.AttributeReference import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.connector.catalog.{CatalogManager, CatalogPlugin, Identifier, LookupCatalog, SupportsNamespaces} +import org.apache.spark.sql.catalyst.util.SparkCharVarcharUtils.replaceCharVarcharWithString +import org.apache.spark.sql.connector.catalog._ import org.apache.spark.sql.errors.DataTypeErrors.toSQLId import org.apache.spark.sql.errors.QueryCompilationErrors -import org.apache.spark.sql.types.{CharType, StringType, VarcharType} import org.apache.spark.util.ArrayImplicits._ /** @@ -106,10 +106,10 @@ class ResolveCatalogs(val catalogManager: CatalogManager) allowTemp: Boolean, columns: Seq[ColumnDefinition]): ResolvedIdentifier = { val columnOutput = columns.map { col => - val dataType = col.dataType match { - case _: CharType | _: VarcharType if !conf.preserveCharVarcharTypeInfo => - StringType - case _ => col.dataType + val dataType = if (conf.preserveCharVarcharTypeInfo) { + col.dataType + } else { + replaceCharVarcharWithString(col.dataType) } AttributeReference(col.name, dataType, col.nullable, col.metadata)() } From a997c9b0692e353263434c6aa1f632bc36835e28 Mon Sep 17 00:00:00 2001 From: Gengliang Wang <gengliang@apache.org> Date: Tue, 29 Apr 2025 09:45:53 -0700 Subject: [PATCH 19/19] address comments --- .../main/resources/error/error-conditions.json | 6 ------ .../sql/catalyst/analysis/CheckAnalysis.scala | 13 +++++-------- .../command/v2/CheckConstraintSuite.scala | 16 ++++++++-------- 3 files changed, 13 insertions(+), 22 deletions(-) diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index 216f683aa7fc1..48b86589be426 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -825,12 +825,6 @@ ], "sqlState" : "42704" }, - "CONSTRAINT_DOES_NOT_HAVE_DATA_TYPE" : { - "message" : [ - "Table constraint expressions do not have a data type." - ], - "sqlState" : "0A000" - }, "CONVERSION_INVALID_INPUT" : { "message" : [ "The value <str> (<fmt>) cannot be converted to <targetType> because it is malformed. Correct the value as per the syntax, or change its format. Use <suggestion> to tolerate malformed input and return NULL instead." diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index dc2a0b87316f8..6c0ba6c74d8d3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -1138,14 +1138,11 @@ trait CheckAnalysis extends LookupCatalog with QueryErrorsBase with PlanToString case RenameColumn(table: ResolvedTable, col: ResolvedFieldName, newName) => checkColumnNotExists("rename", col.path :+ newName, table.schema) - - case AddConstraint(_: ResolvedTable, check: CheckConstraint) => - if (!check.deterministic) { - check.child.failAnalysis( - errorClass = "NON_DETERMINISTIC_CHECK_CONSTRAINT", - messageParameters = Map("checkCondition" -> check.condition) - ) - } + case AddConstraint(_: ResolvedTable, check: CheckConstraint) if !check.deterministic => + check.child.failAnalysis( + errorClass = "NON_DETERMINISTIC_CHECK_CONSTRAINT", + messageParameters = Map("checkCondition" -> check.condition) + ) case AlterColumns(table: ResolvedTable, specs) => val groupedColumns = specs.groupBy(_.column.name) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/CheckConstraintSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/CheckConstraintSuite.scala index ee83614eb6e5f..3b48430604e86 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/CheckConstraintSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/CheckConstraintSuite.scala @@ -52,10 +52,10 @@ class CheckConstraintSuite extends QueryTest with CommandSuiteBase with DDLComma test("Nondeterministic expression -- create table") { Seq( - "create table t(i double check (i > rand(0)))", - "create table t(i double, constraint c1 check (i > rand(0)))", - "replace table t(i double check (i > rand(0)))", - "replace table t(i double, constraint c1 check (i > rand(0)))" + "CREATE TABLE t(i DOUBLE CHECK (i > rand(0)))", + "CREATE TABLE t(i DOUBLE, CONSTRAINT c1 CHECK (i > rand(0)))", + "REPLACE TABLE t(i DOUBLE CHECK (i > rand(0)))", + "REPLACE TABLE t(i DOUBLE, CONSTRAINT c1 CHECK (i > rand(0)))" ).foreach { query => withTable("t") { val error = intercept[AnalysisException] { @@ -76,8 +76,8 @@ class CheckConstraintSuite extends QueryTest with CommandSuiteBase with DDLComma test("Expression referring a column of another table -- alter table") { withTable("t", "t2") { - sql("create table t(i double) using parquet") - sql("create table t2(j string) using parquet") + sql("CREATE TABLE t(i DOUBLE) USING parquet") + sql("CREATE TABLE t2(j STRING) USING parquet") val query = """ |ALTER TABLE t ADD CONSTRAINT c1 CHECK (len(t2.j) > 0) @@ -101,8 +101,8 @@ class CheckConstraintSuite extends QueryTest with CommandSuiteBase with DDLComma test("Expression referring a column of another table -- create and replace table") { withTable("t", "t2") { - sql("create table t(i double) using parquet") - val query = "create table t2(j string check(t.i > 0)) using parquet" + sql("CREATE TABLE t(i double) USING parquet") + val query = "CREATE TABLE t2(j string check(t.i > 0)) USING parquet" val error = intercept[AnalysisException] { sql(query) }