Skip to content

Commit

Permalink
[Spark] Current Date/Time resolution in constraints (#2766)
Browse files Browse the repository at this point in the history
<!--
Thanks for sending a pull request!  Here are some tips for you:
1. If this is your first time, please read our contributor guidelines:
https://github.com/delta-io/delta/blob/master/CONTRIBUTING.md
2. If the PR is unfinished, add '[WIP]' in your PR title, e.g., '[WIP]
Your PR title ...'.
  3. Be sure to keep the PR description updated to reflect all changes.
  4. Please write your PR title to summarize what this PR proposes.
5. If possible, provide a concise example to reproduce the issue for a
faster review.
6. If applicable, include the corresponding issue number in the PR title
and link it in the body.
-->

#### Which Delta project/connector is this regarding?
<!--
Please add the component selected below to the beginning of the pull
request title
For example: [Spark] Title of my pull request
-->

- [x] Spark
- [ ] Standalone
- [ ] Flink
- [ ] Kernel
- [ ] Other (fill in here)

## Description

<!--
- Describe what this PR changes.
- Describe why we need the change.
 
If this PR resolves an issue be sure to include "Resolves #XXX" to
correctly link and close the issue upon merge.
-->

Heading towards the removal of codegenFallback from Date/Time
expressions (apache/spark#44261), delta
constraints need to resolve current_datetime expressions during the
analysis of the invariants. The proposed changes work for both Spark 3.5
and Spark master.

## How was this patch tested?

<!--
If tests were added, say they were added here. Please make sure to test
the changes thoroughly including negative and positive cases if
possible.
If the changes were tested in any way other than unit tests, please
clarify how you tested step by step (ideally copy and paste-able, so
that other reviewers can test and check, and descendants can verify in
the future).
If the changes were not tested, please explain why.
-->
Existing tests. In particular, `CheckConstaintsSuite` covers constraints
with `current_timestamp()` expressions. Added an extra test to cover
`current_date()`.

## Does this PR introduce _any_ user-facing changes?

<!--
If yes, please clarify the previous behavior and the change this PR
proposes - provide the console output, description and/or an example to
show the behavior difference if possible.
If possible, please also clarify if this is a user-facing change
compared to the released Delta Lake versions or within the unreleased
branches such as master.
If no, write 'No'.
-->
No.
  • Loading branch information
andreaschat-db authored Mar 21, 2024
1 parent 8cd0f94 commit 9f040d4
Show file tree
Hide file tree
Showing 2 changed files with 18 additions and 6 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -27,10 +27,12 @@ import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.analysis._
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.optimizer
import org.apache.spark.sql.catalyst.optimizer.ReplaceExpressions
import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, UnaryNode}
import org.apache.spark.sql.catalyst.plans.physical.Partitioning
import org.apache.spark.sql.catalyst.rules.RuleExecutor
import org.apache.spark.sql.delta.util.AnalysisHelper
import org.apache.spark.sql.execution.{SparkPlan, SparkStrategy, UnaryExecNode}
import org.apache.spark.sql.types.StructType

Expand Down Expand Up @@ -71,9 +73,17 @@ case class DeltaInvariantCheckerExec(
if (constraints.isEmpty) return child.execute()
val invariantChecks =
DeltaInvariantCheckerExec.buildInvariantChecks(child.output, constraints, session)
val boundRefs = invariantChecks.map(_.withBoundReferences(child.output))

// Resolve current_date()/current_time() expressions.
// We resolve currentTime for all invariants together to make sure we use the same timestamp.
val invariantsFakePlan = AnalysisHelper.FakeLogicalPlan(invariantChecks, Nil)
val newInvariantsPlan = optimizer.ComputeCurrentTime(invariantsFakePlan)
val localOutput = child.output

child.execute().mapPartitionsInternal { rows =>
val boundRefs = newInvariantsPlan.expressions
.asInstanceOf[Seq[CheckDeltaInvariant]]
.map(_.withBoundReferences(localOutput))
val assertions = UnsafeProjection.create(boundRefs)
rows.map { row =>
assertions(row)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -277,18 +277,20 @@ class CheckConstraintsSuite extends QueryTest
}
}

testQuietly("constraint with analyzer-evaluated expressions") {
for (expression <- Seq("year(current_date())", "unix_timestamp()"))
testQuietly(s"constraint with analyzer-evaluated expressions. Expression: $expression") {
withTestTable { table =>
// We use current_timestamp() as the most convenient analyzer-evaluated expression - of course
// in a realistic use case it'd probably not be right to add a constraint on a
// We use current_timestamp()/current_date() as the most convenient
// analyzer-evaluated expressions - of course in a realistic use case
// it'd probably not be right to add a constraint on a
// nondeterministic expression.
sql(s"ALTER TABLE $table ADD CONSTRAINT maxWithAnalyzerEval " +
s"CHECK (num < unix_timestamp())")
s"CHECK (num < $expression)")
val e = intercept[InvariantViolationException] {
sql(s"INSERT INTO $table VALUES (${Int.MaxValue}, 'data')")
}
errorContains(e.getMessage,
"maxwithanalyzereval (num < unix_timestamp()) violated by row")
s"maxwithanalyzereval (num < $expression) violated by row")
}
}

Expand Down

0 comments on commit 9f040d4

Please sign in to comment.