This is an automated email from the ASF dual-hosted git repository. wenchen pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git
The following commit(s) were added to refs/heads/master by this push: new 0895d195f77c [SPARK-51614][SQL] Introduce ResolveUnresolvedHaving rule in the Analyzer 0895d195f77c is described below commit 0895d195f77cf3d46b8bb45077d6376c1068539c Author: mihailoale-db <mihailo.alek...@databricks.com> AuthorDate: Fri Mar 28 23:40:17 2025 +0800 [SPARK-51614][SQL] Introduce ResolveUnresolvedHaving rule in the Analyzer ### What changes were proposed in this pull request? Introduce `ResolveUnresolvedHaving` rule in the Analyzer to handle `UnresolvedHaving` nodes with resolved `condition` and `child`. ### Why are the changes needed? Fix the case to work. ### Does this PR introduce _any_ user-facing change? Queries that failed now pass. ### How was this patch tested? Added test. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #50409 from mihailoale-db/unresolvedhavingwithgenerate. Authored-by: mihailoale-db <mihailo.alek...@databricks.com> Signed-off-by: Wenchen Fan <wenc...@databricks.com> --- .../apache/spark/sql/catalyst/analysis/Analyzer.scala | 18 +++++++++++++++++- .../spark/sql/catalyst/rules/RuleIdCollection.scala | 1 + .../scala/org/apache/spark/sql/SQLQuerySuite.scala | 13 +++++++++++++ 3 files changed, 31 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 91451795cc42..c22d28a0cd5b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -460,7 +460,8 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor Batch("HandleSpecialCommand", Once, HandleSpecialCommand), Batch("Remove watermark for batch query", Once, - EliminateEventTimeWatermark) + EliminateEventTimeWatermark), + Batch("ResolveUnresolvedHaving", Once, ResolveUnresolvedHaving) ) /** @@ -4206,3 +4207,18 @@ object RemoveTempResolvedColumn extends Rule[LogicalPlan] { } } } + +/** + * Rule that's used to handle `UnresolvedHaving` nodes with resolved `condition` and `child`. + * It's placed outside the main batch to avoid conflicts with other rules that resolve + * `UnresolvedHaving` in the main batch. + */ +object ResolveUnresolvedHaving extends Rule[LogicalPlan] { + override def apply(plan: LogicalPlan): LogicalPlan = { + plan.resolveOperatorsWithPruning(_.containsPattern(UNRESOLVED_HAVING), ruleId) { + case u @ UnresolvedHaving(havingCondition, child) + if havingCondition.resolved && child.resolved => + Filter(condition = havingCondition, child = child) + } + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleIdCollection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleIdCollection.scala index 8cf13168466f..dea7fc33b184 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleIdCollection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleIdCollection.scala @@ -111,6 +111,7 @@ object RuleIdCollection { "org.apache.spark.sql.catalyst.analysis.ResolveUpdateEventTimeWatermarkColumn" :: "org.apache.spark.sql.catalyst.expressions.EliminatePipeOperators" :: "org.apache.spark.sql.catalyst.expressions.ValidateAndStripPipeExpressions" :: + "org.apache.spark.sql.catalyst.analysis.ResolveUnresolvedHaving" :: // Catalyst Optimizer rules "org.apache.spark.sql.catalyst.optimizer.BooleanSimplification" :: "org.apache.spark.sql.catalyst.optimizer.CollapseProject" :: diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 3147a60bad5e..ce6440e90103 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -4943,6 +4943,19 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark Row(Array(0), Array(0)), Row(Array(1), Array(1)), Row(Array(2), Array(2))) checkAnswer(df, expectedAnswer) } + + test("SPARK-51614: Having operator is properly resolved when there's generator in condition") { + val df = sql( + """select + | explode(packages) as package + |from + | values(array('a')) t(packages) + |group by all + |having package in ('a')""".stripMargin + ) + + checkAnswer(df, Row("a")) + } } case class Foo(bar: Option[String]) --------------------------------------------------------------------- To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org