Hisoka-X commented on code in PR #40865:
URL: https://github.com/apache/spark/pull/40865#discussion_r1177458137
##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala:
##########
@@ -459,7 +459,7 @@ object RewriteCorrelatedScalarSubquery extends
Rule[LogicalPlan] with AliasHelpe
case alias @ Alias(_: AttributeReference, _) =>
(alias.exprId, Literal.create(null, alias.dataType))
case alias @ Alias(l: Literal, _) =>
- (alias.exprId, l.copy(value = null))
+ (alias.exprId, l)
Review Comment:
> I think this one-line change is all we need. Count bug does not apply to
the count function only, but all expressions in Aggregate that return non-null
value for empty input. This includes all constant values.
>
> Assume there are two table `t1(c1 int)`, `t2(c2 int)`. `t1` has two rows
(1, 2), `t2` has two rows (1,3)
>
> ```
> scala> sql("select (select f from (select false as f, max(c1) from t1
where c1 = c2)) from t2").show
> +------------------+
> |scalarsubquery(c2)|
> +------------------+
> | false|
> | NULL|
> +------------------+
> ```
>
> The result is wrong and it should be false, false
After this PR, the result will be false, false
>
> `select f from (select false as f, max(c1) from t1 where c1 = c2)` is to
build a global aggregate that has no aggregate function. If SQL syntax supports
group by no column, then we can simply write `select false as f from t1 group
by () where c1 = c2`
After this PR, already is.
```
== Parsed Logical Plan ==
'Project [unresolvedalias(scalar-subquery#245 [], None)]
: +- 'Project ['f]
: +- 'SubqueryAlias __auto_generated_subquery_name
: +- 'Project [false AS f#244, unresolvedalias('max('c1), None)]
: +- 'Filter ('c1 = 'c2)
: +- 'UnresolvedRelation [t1], [], false
+- 'UnresolvedRelation [t2], [], false
== Analyzed Logical Plan ==
scalarsubquery(c2): boolean
Project [scalar-subquery#245 [c2#246] AS scalarsubquery(c2)#252]
: +- Project [f#244]
: +- SubqueryAlias __auto_generated_subquery_name
: +- Aggregate [false AS f#244, max(c1#248) AS max(c1)#251]
: +- Filter (c1#248 = outer(c2#246))
: +- SubqueryAlias spark_catalog.default.t1
: +- View (`spark_catalog`.`default`.`t1`, [c1#248])
: +- Project [cast(col1#249 as int) AS c1#248]
: +- LocalRelation [col1#249]
+- SubqueryAlias spark_catalog.default.t2
+- View (`spark_catalog`.`default`.`t2`, [c2#246])
+- Project [cast(col1#247 as int) AS c2#246]
+- LocalRelation [col1#247]
== Optimized Logical Plan ==
Project [false AS scalarsubquery(c2)#252]
+- LocalRelation [col1#247]
== Physical Plan ==
*(1) Project [false AS scalarsubquery(c2)#252]
+- *(1) LocalTableScan [col1#247]
```
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]