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

Bug(optimizer): fail to plan dynmic filter when there are more than one predicate on the same subquery #15445

Closed
st1page opened this issue Mar 5, 2024 · 0 comments · Fixed by #15449
Milestone

Comments

@st1page
Copy link
Contributor

st1page commented Mar 5, 2024

create table t1(v1 int);
create table t2(v2 int);

dev=> explain create materialized view mv as 
with subquery as (
  select max(v2) as mx from t2
)
select * from t1
where 
v1 > (select mx from subquery) 
AND v1 < (select mx+5 from subquery);
ERROR:  Failed to run the query

Caused by:
  Scalar subquery might produce more than one row.

dev=> explain (trace) create materialized view mv as 
with subquery as (
  select max(v2) as mx from t2
)
select * from t1
where 
v1 > (select mx from subquery) 
AND v1 < (select mx+5 from subquery);
                                      QUERY PLAN                                       
---------------------------------------------------------------------------------------
 Begin:
 
 LogicalProject { exprs: [t1.v1] }
 └─LogicalFilter { predicate: (t1.v1 > max(t2.v2)) AND (t1.v1 < $expr1) }
   └─LogicalApply { type: LeftOuter, on: true, correlated_id: 2, max_one_row: true }
     ├─LogicalApply { type: LeftOuter, on: true, correlated_id: 1, max_one_row: true }
     │ ├─LogicalScan { table: t1, columns: [v1, _row_id] }
     │ └─LogicalProject { exprs: [max(t2.v2)] }
     │   └─LogicalShare { id: 4 }
     │     └─LogicalProject { exprs: [max(t2.v2)] }
     │       └─LogicalAgg { aggs: [max(t2.v2)] }
     │         └─LogicalProject { exprs: [t2.v2] }
     │           └─LogicalScan { table: t2, columns: [v2, _row_id] }
     └─LogicalProject { exprs: [(max(t2.v2) + 5:Int32) as $expr1] }
       └─LogicalShare { id: 4 }
         └─LogicalProject { exprs: [max(t2.v2)] }
           └─LogicalAgg { aggs: [max(t2.v2)] }
             └─LogicalProject { exprs: [t2.v2] }
               └─LogicalScan { table: t2, columns: [v2, _row_id] }
 
 Project Remove:
 
 apply AggProjectMergeRule 1 time(s)
 apply ProjectEliminateRule 2 time(s)
 
 LogicalProject { exprs: [t1.v1] }
 └─LogicalFilter { predicate: (t1.v1 > max(t2.v2)) AND (t1.v1 < $expr1) }
   └─LogicalApply { type: LeftOuter, on: true, correlated_id: 2, max_one_row: true }
     ├─LogicalApply { type: LeftOuter, on: true, correlated_id: 1, max_one_row: true }
     │ ├─LogicalScan { table: t1, columns: [v1, _row_id] }
     │ └─LogicalShare { id: 2 }
     │   └─LogicalAgg { aggs: [max(t2.v2)] }
     │     └─LogicalScan { table: t2, columns: [v2, _row_id] }
     └─LogicalProject { exprs: [(max(t2.v2) + 5:Int32) as $expr1] }
       └─LogicalShare { id: 2 }
         └─LogicalAgg { aggs: [max(t2.v2)] }
           └─LogicalScan { table: t2, columns: [v2, _row_id] }
 
 Common Sub-plan Sharing:
 
 LogicalProject { exprs: [t1.v1] }
 └─LogicalFilter { predicate: (t1.v1 > max(t2.v2)) AND (t1.v1 < $expr1) }
   └─LogicalApply { type: LeftOuter, on: true, correlated_id: 2, max_one_row: true }
     ├─LogicalApply { type: LeftOuter, on: true, correlated_id: 1, max_one_row: true }
     │ ├─LogicalScan { table: t1, columns: [v1, _row_id] }
     │ └─LogicalShare { id: 2 }
     │   └─LogicalAgg { aggs: [max(t2.v2)] }
     │     └─LogicalScan { table: t2, columns: [v2, _row_id] }
     └─LogicalProject { exprs: [(max(t2.v2) + 5:Int32) as $expr1] }
       └─LogicalShare { id: 2 }
         └─LogicalAgg { aggs: [max(t2.v2)] }
           └─LogicalScan { table: t2, columns: [v2, _row_id] }
 
 Simple Unnesting:
 
 apply MaxOneRowEliminateRule 2 time(s)
 apply ApplyToJoinRule 2 time(s)
 
 LogicalProject { exprs: [t1.v1] }
 └─LogicalFilter { predicate: (t1.v1 > max(t2.v2)) AND (t1.v1 < $expr1) }
   └─LogicalJoin { type: LeftOuter, on: true }
     ├─LogicalJoin { type: LeftOuter, on: true }
     │ ├─LogicalScan { table: t1, columns: [v1, _row_id] }
     │ └─LogicalMaxOneRow
     │   └─LogicalShare { id: 2 }
     │     └─LogicalAgg { aggs: [max(t2.v2)] }
     │       └─LogicalScan { table: t2, columns: [v2, _row_id] }
     └─LogicalMaxOneRow
       └─LogicalProject { exprs: [(max(t2.v2) + 5:Int32) as $expr1] }
         └─LogicalShare { id: 2 }
           └─LogicalAgg { aggs: [max(t2.v2)] }
             └─LogicalScan { table: t2, columns: [v2, _row_id] }
 
 Predicate Push Down:
 
 LogicalProject { exprs: [t1.v1] }
 └─LogicalJoin { type: Inner, on: (t1.v1 < $expr1) }
   ├─LogicalJoin { type: Inner, on: (t1.v1 > max(t2.v2)) }
   │ ├─LogicalScan { table: t1, columns: [v1, _row_id] }
   │ └─LogicalMaxOneRow
   │   └─LogicalShare { id: 2 }
   │     └─LogicalAgg { aggs: [max(t2.v2)] }
   │       └─LogicalScan { table: t2, columns: [v2, _row_id] }
   └─LogicalMaxOneRow
     └─LogicalProject { exprs: [(max(t2.v2) + 5:Int32) as $expr1] }
       └─LogicalShare { id: 2 }
         └─LogicalAgg { aggs: [max(t2.v2)] }
           └─LogicalScan { table: t2, columns: [v2, _row_id] }
 
 ERROR: Scalar subquery might produce more than one row.
(90 rows)

dev=> explain create materialized view mv as 
with subquery as (
  select max(v2) as mx from t2
)
select * from t1
where 
v1 > (select mx from subquery);
                                                            QUERY PLAN                                                             
-----------------------------------------------------------------------------------------------------------------------------------
 StreamMaterialize { columns: [v1, t1._row_id(hidden)], stream_key: [t1._row_id], pk_columns: [t1._row_id], pk_conflict: NoCheck }
 └─StreamDynamicFilter { predicate: (t1.v1 > max(max(t2.v2))), output: [t1.v1, t1._row_id] }
   ├─StreamTableScan { table: t1, columns: [v1, _row_id] }
   └─StreamExchange { dist: Broadcast }
     └─StreamProject { exprs: [max(max(t2.v2))] }
       └─StreamSimpleAgg { aggs: [max(max(t2.v2)), count] }
         └─StreamExchange { dist: Single }
           └─StreamHashAgg { group_key: [$expr1], aggs: [max(t2.v2), count] }
             └─StreamProject { exprs: [t2.v2, t2._row_id, Vnode(t2._row_id) as $expr1] }
               └─StreamTableScan { table: t2, columns: [v2, _row_id] }
(10 rows)
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging a pull request may close this issue.

1 participant