-
Notifications
You must be signed in to change notification settings - Fork 28.9k
[SPARK-51902][SQL] Enforce check constraint on table insertion #50761
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Closed
Closed
Changes from 4 commits
Commits
Show all changes
8 commits
Select commit
Hold shift + click to select a range
2d295c1
check enforcement
gengliangwang 418c568
add test cases of insert overwrite
gengliangwang f00c405
fix tests
gengliangwang f1da34d
null is valid
gengliangwang 091d9ec
address comments
gengliangwang a558b55
refactor; avoid introducing validate operator
gengliangwang d10b03d
enhance comments
gengliangwang e35ecaf
address comments
gengliangwang File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
70 changes: 70 additions & 0 deletions
70
...talyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveTableConstraint.scala
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,70 @@ | ||
| /* | ||
| * 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.catalyst.analysis | ||
|
|
||
| import scala.collection.mutable | ||
|
|
||
| import org.apache.spark.sql.catalyst.expressions.{CheckInvariant, Expression} | ||
| import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, V2WriteCommand, Validate} | ||
| import org.apache.spark.sql.catalyst.rules.Rule | ||
| import org.apache.spark.sql.catalyst.trees.TreePattern.COMMAND | ||
| import org.apache.spark.sql.connector.catalog.CatalogManager | ||
| import org.apache.spark.sql.connector.catalog.constraints.Check | ||
| import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation | ||
|
|
||
| class ResolveTableConstraint(val catalogManager: CatalogManager) extends Rule[LogicalPlan] { | ||
gengliangwang marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
|
|
||
| override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsWithPruning( | ||
| _.containsPattern(COMMAND), ruleId) { | ||
| case v2Write: V2WriteCommand | ||
gengliangwang marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| if v2Write.table.resolved && v2Write.query.resolved && | ||
| !v2Write.query.isInstanceOf[Validate] && v2Write.outputResolved => | ||
| v2Write.table match { | ||
| case r: DataSourceV2Relation | ||
| if r.table.constraints() != null && r.table.constraints().nonEmpty => | ||
gengliangwang marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
| val checks = r.table.constraints().collect { | ||
|
||
| case c: Check => c | ||
| } | ||
| val checkInvariants = checks.map { c => | ||
| val parsed = | ||
gengliangwang marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
| catalogManager.v1SessionCatalog.parser.parseExpression(c.predicateSql()) | ||
| val columnExtractors = mutable.Map[String, Expression]() | ||
gengliangwang marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
| buildColumnExtractors(parsed, columnExtractors) | ||
| CheckInvariant(parsed, columnExtractors.toSeq, c.name(), c.predicateSql()) | ||
| }.toSeq | ||
| v2Write.withNewQuery(Validate(checkInvariants, v2Write.query)) | ||
| case _ => | ||
| v2Write | ||
| } | ||
| } | ||
|
|
||
| private def buildColumnExtractors( | ||
| expr: Expression, | ||
| columnExtractors: mutable.Map[String, Expression]): Unit = { | ||
| expr match { | ||
| case u: UnresolvedExtractValue => | ||
| // When extracting a value from a Map or Array type, we display only the specific extracted | ||
| // value rather than the entire Map or Array structure for clarity and readability. | ||
| columnExtractors(u.sql) = u | ||
| case u: UnresolvedAttribute => | ||
| columnExtractors(u.name) = u | ||
|
|
||
gengliangwang marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
| case other => | ||
| other.children.foreach(buildColumnExtractors(_, columnExtractors)) | ||
| } | ||
| } | ||
| } | ||
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
45 changes: 45 additions & 0 deletions
45
.../src/test/scala/org/apache/spark/sql/catalyst/expressions/ConstraintExpressionSuite.scala
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,45 @@ | ||
| /* | ||
| * 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.catalyst.expressions | ||
|
|
||
| import org.apache.spark.{SparkFunSuite, SparkRuntimeException} | ||
| import org.apache.spark.sql.catalyst.InternalRow | ||
| import org.apache.spark.sql.types.IntegerType | ||
|
|
||
| class ConstraintExpressionSuite extends SparkFunSuite with ExpressionEvalHelper { | ||
| private val boundRef = BoundReference(0, IntegerType, nullable = true) | ||
| private val expr = | ||
| CheckInvariant(GreaterThan(boundRef, Literal(0)), Seq(("a", boundRef)), "c1", "a > 0") | ||
|
|
||
| def expectedMessage(value: String): String = | ||
| s"""|[CHECK_CONSTRAINT_VIOLATION] CHECK constraint c1 a > 0 violated by row with values: | ||
| | - a : $value | ||
| | SQLSTATE: 23001""".stripMargin | ||
|
|
||
| test("CheckInvariant: returns null if column 'a' > 0") { | ||
| checkEvaluation(expr, null, InternalRow(1)) | ||
| } | ||
|
|
||
| test("CheckInvariant: return null if column 'a' is null") { | ||
| checkEvaluation(expr, null, InternalRow(null)) | ||
| } | ||
|
|
||
| test("CheckInvariant: throws exception if column 'a' <= 0") { | ||
| checkExceptionInExpression[SparkRuntimeException]( | ||
| expr, InternalRow(-1), expectedMessage("-1")) | ||
| } | ||
| } |
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
49 changes: 49 additions & 0 deletions
49
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ValidateExec.scala
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,49 @@ | ||
| /* | ||
| * 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.datasources.v2 | ||
|
|
||
| import org.apache.spark.rdd.RDD | ||
| import org.apache.spark.sql.catalyst.InternalRow | ||
| import org.apache.spark.sql.catalyst.expressions.{Attribute, CheckInvariant, SortOrder, UnsafeProjection} | ||
| import org.apache.spark.sql.catalyst.plans.physical.Partitioning | ||
| import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode} | ||
|
|
||
| case class ValidateExec( | ||
| child: SparkPlan, | ||
| constraints: Seq[CheckInvariant]) extends UnaryExecNode { | ||
|
|
||
| override def output: Seq[Attribute] = child.output | ||
|
|
||
| override protected def doExecute(): RDD[InternalRow] = { | ||
| if (constraints.isEmpty) return child.execute() | ||
|
|
||
| child.execute().mapPartitionsInternal { rows => | ||
| val assertions = UnsafeProjection.create(constraints, child.output) | ||
| rows.map { row => | ||
| assertions(row) | ||
| row | ||
| } | ||
| } | ||
| } | ||
|
|
||
| override def outputOrdering: Seq[SortOrder] = child.outputOrdering | ||
|
|
||
| override def outputPartitioning: Partitioning = child.outputPartitioning | ||
|
|
||
| override protected def withNewChildInternal(newChild: SparkPlan): ValidateExec = | ||
| copy(child = newChild) | ||
| } |
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
Uh oh!
There was an error while loading. Please reload this page.