[ 
https://issues.apache.org/jira/browse/CALCITE-7272?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=18039278#comment-18039278
 ] 

weihua zhang commented on CALCITE-7272:
---------------------------------------

[~mbudiu] 

Here is the rationale for why this transformation is correct for {{UNION ALL}} 
(and other SetOps), even if the outer relation is not distinct:

*1. Treating Outer Rows as Groups* The core idea is that the correlated 
subquery is essentially a function applied to each row of the outer relation. 
If multiple outer rows have the same values for the correlation variables 
(e.g., same {{deptno}} and {{{}job{}}}), the subquery should produce the same 
result for them.

*2. Push Down and Projection* In my proposed transformation (as shown in the 
"to" plan), I push the correlation variables (e.g., {{{}emp.job{}}}, 
{{{}emp.deptno{}}}) down into *each branch* of the {{{}UNION ALL{}}}.
 * Instead of just selecting the subquery result {{{}c{}}}, each branch now 
projects {{{}(correlation_keys, c){}}}.

 * For the {{UNION ALL}} operator, this means we are now unioning these wider 
tuples.

*3. Aggregation by Correlation Keys* After the {{{}UNION ALL{}}}, I introduce 
an {{Aggregate}} on the correlation keys ({{{}group=[\{0, 1}]{}}} in the 
example, corresponding to {{JOB}} and {{{}DEPTNO{}}}). This step calculates the 
subquery result (e.g., {{{}SUM(c){}}}) {*}per unique combination of correlation 
variables{*}.
 * This effectively computes the subquery result for the "distinct" set of 
correlation bindings present in the outer table.

*4. Joining Back* Finally, the result of this aggregation is joined back to the 
original outer table ({{{}emp{}}}) using a {{LEFT JOIN}} on the correlation 
keys ({{{}IS NOT DISTINCT FROM{}}}).
 * This correctly distributes the computed aggregate value to _every_ row in 
the outer table that matches those keys, regardless of how many times that row 
(or key combination) appears.

*In summary:* The transformation avoids the "duplicate counting" problem not by 
requiring the LHS to be distinct initially, but by:
 # generating the distinct set of correlation keys (via {{LogicalAggregate}} on 
the LHS in the sub-branches),

 # computing the answer for that distinct set, and

 # joining the answer back to the original LHS.

This pattern ensures that the semantics of the original correlated subquery are 
preserved for any collection, distinct or not.

> Subqueries cannot be decorrelated if have set op
> ------------------------------------------------
>
>                 Key: CALCITE-7272
>                 URL: https://issues.apache.org/jira/browse/CALCITE-7272
>             Project: Calcite
>          Issue Type: Improvement
>            Reporter: weihua zhang
>            Priority: Major
>              Labels: pull-request-available
>         Attachments: image-2025-11-19-10-10-16-318.png
>
>
> case 1:
> {code:sql}
> SELECT ename,
>   (
>     SELECT sum(c)
>     FROM (
>       SELECT deptno AS c
>       FROM   dept
>       WHERE  dept.deptno = emp.deptno
>       UNION ALL
>       SELECT 2 AS c
>       FROM   bonus
>       WHERE  bonus.job = emp.job
>     ) AS union_subquery
>   ) AS correlated_sum FROM
>   emp;
> LogicalProject(ENAME=[$1], CORRELATED_SUM=[$8]), id = 2619
>   LogicalProject(empno=[$0], ename=[$1], job=[$2], mgr=[$3], hiredate=[$4], 
> sal=[$5], comm=[$6], deptno=[$7], EXPR$0=[$8]), id = 2628
>     LogicalCorrelate(correlation=[$cor0], joinType=[left], 
> requiredColumns=[{2, 7}]), id = 2626
>       LogicalTableScan(table=[[testdb, emp]]), id = 2542
>       LogicalAggregate(group=[{}], EXPR$0=[SUM($0)]), id = 2615
>         LogicalUnion(all=[true]), id = 2613
>           LogicalProject(C=[$0]), id = 2606
>             LogicalFilter(condition=[=($0, $cor0.deptno)]), id = 2604
>               LogicalTableScan(table=[[testdb, dept]]), id = 2544
>           LogicalProject(C=[2]), id = 2611
>             LogicalFilter(condition=[=($1, $cor0.job)]), id = 2609
>               LogicalTableScan(table=[[testdb, bonus]]), id = 2548
> {code}
> case 2:
> {code:sql}
> SELECT *,
>   (SELECT COUNT(*)
>    FROM (
>      SELECT * FROM emp WHERE emp.deptno = dept.deptno
>      UNION ALL
>      SELECT * FROM emp
>    ) AS sub
>    GROUP BY deptno
>   ) AS num_dept_groups
> FROM dept;
> {code}



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to