-
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
Changes from all commits
2d295c1
418c568
f00c405
f1da34d
091d9ec
a558b55
d10b03d
e35ecaf
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,84 @@ | ||
| /* | ||
| * 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.{And, CheckInvariant, Expression, V2ExpressionUtils} | ||
| import org.apache.spark.sql.catalyst.plans.logical.{Filter, LogicalPlan, V2WriteCommand} | ||
| 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 ResolveTableConstraints(val catalogManager: CatalogManager) extends Rule[LogicalPlan] { | ||
|
|
||
| override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsWithPruning( | ||
| _.containsPattern(COMMAND), ruleId) { | ||
| case v2Write: V2WriteCommand | ||
| if v2Write.table.resolved && v2Write.query.resolved && | ||
| !containsCheckInvariant(v2Write.query) && v2Write.outputResolved => | ||
| v2Write.table match { | ||
| case r: DataSourceV2Relation | ||
| if r.table.constraints != null && r.table.constraints.nonEmpty => | ||
| // Check constraint is the only enforced constraint for DSV2 tables. | ||
| val checkInvariants = r.table.constraints.collect { | ||
| case c: Check => | ||
| val unresolvedExpr = buildCatalystExpression(c) | ||
| val columnExtractors = mutable.Map[String, Expression]() | ||
| buildColumnExtractors(unresolvedExpr, columnExtractors) | ||
| CheckInvariant(unresolvedExpr, columnExtractors.toSeq, c.name, c.predicateSql) | ||
| } | ||
| // Combine the check invariants into a single expression using conjunctive AND. | ||
| checkInvariants.reduceOption(And).fold(v2Write)( | ||
|
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Optional: I think it would be easier to read/understand if we use pattern matching (or something similar). I think reduceOption + fold takes a bit of time to process but it is a personal preference. |
||
| condition => v2Write.withNewQuery(Filter(condition, v2Write.query))) | ||
| case _ => | ||
| v2Write | ||
| } | ||
| } | ||
|
|
||
| private def containsCheckInvariant(plan: LogicalPlan): Boolean = { | ||
| plan match { | ||
| case Filter(condition, _) => | ||
| condition.exists(_.isInstanceOf[CheckInvariant]) | ||
|
|
||
| case _ => false | ||
| } | ||
| } | ||
|
|
||
| private def buildCatalystExpression(c: Check): Expression = { | ||
|
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I would still consider a better error message if the conversion fails and there is no SQL representation provided, potentially in a follow-up PR. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I will have a follow-up |
||
| Option(c.predicate()) | ||
| .flatMap(V2ExpressionUtils.toCatalyst) | ||
| .getOrElse(catalogManager.v1SessionCatalog.parser.parseExpression(c.predicateSql())) | ||
| } | ||
|
|
||
| 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 | ||
| case other => | ||
| other.children.foreach(buildColumnExtractors(_, columnExtractors)) | ||
| } | ||
| } | ||
| } | ||
| 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 true if column 'a' > 0") { | ||
| checkEvaluation(expr, true, InternalRow(1)) | ||
| } | ||
|
|
||
| test("CheckInvariant: return true if column 'a' is null") { | ||
| checkEvaluation(expr, true, InternalRow(null)) | ||
| } | ||
|
|
||
| test("CheckInvariant: throws exception if column 'a' <= 0") { | ||
| checkExceptionInExpression[SparkRuntimeException]( | ||
| expr, InternalRow(-1), expectedMessage("-1")) | ||
| } | ||
| } |
Uh oh!
There was an error while loading. Please reload this page.