Github user marmbrus commented on a diff in the pull request:
https://github.com/apache/spark/pull/1497#discussion_r15157732
--- Diff:
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
---
@@ -152,6 +155,34 @@ class Analyzer(catalog: Catalog, registry:
FunctionRegistry, caseSensitive: Bool
}
/**
+ * This rule finds expressions in HAVING clause filters that depend on
+ * unresolved attributes. It pushes these expressions down to the
underlying
+ * aggregates and then projects them away above the filter.
+ */
+ object UnresolvedHavingClauseAttributes extends Rule[LogicalPlan] {
+ def apply(plan: LogicalPlan): LogicalPlan = plan transform {
+ case pl @ Filter(fexp, agg @ Aggregate(_, ae, _)) if
!fexp.childrenResolved => {
+ val alias = Alias(fexp, makeTmp())()
+ val aggExprs = Seq(alias) ++ ae
+
+ val newCond = EqualTo(Cast(alias.toAttribute, BooleanType),
Literal(true, BooleanType))
+
+ val newFilter = ResolveReferences(pl.copy(condition = newCond,
+ child = agg.copy(aggregateExpressions = aggExprs)))
+
+ Project(pl.output, newFilter)
+ }
+ }
+
+ private val curId = new java.util.concurrent.atomic.AtomicLong()
+
+ private def makeTmp() = {
+ val id = curId.getAndIncrement()
+ s"tmp_cond_$id"
--- End diff --
All `NamedExpression`s have a globally unique id (i.e. the `#number` in the
`toString`), so we can just create an Alias with a fixed `name`. I'd consider
naming it something like `havingCondition`.
---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [email protected] or file a JIRA ticket
with INFRA.
---