Skip to content
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

[Spark] Current Date/Time resolution in constraints #2766

Merged
merged 4 commits into from
Mar 21, 2024
Merged
Changes from 2 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
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,16 @@ 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)

child.execute().mapPartitionsInternal { rows =>
val boundRefs = newInvariantsPlan.expressions
andreaschat-db marked this conversation as resolved.
Show resolved Hide resolved
.asInstanceOf[Seq[CheckDeltaInvariant]]
.map(_.withBoundReferences(child.output))
val assertions = UnsafeProjection.create(boundRefs)
rows.map { row =>
assertions(row)
Expand Down
Loading