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

feat: eliminate adjacent no-op updates for agg with empty agg call #17048

Merged
merged 6 commits into from
Jun 3, 2024
Merged
Show file tree
Hide file tree
Changes from 4 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
3 changes: 3 additions & 0 deletions proto/stream_plan.proto
Original file line number Diff line number Diff line change
Expand Up @@ -278,6 +278,9 @@ message ProjectNode {
repeated uint32 watermark_input_cols = 2;
repeated uint32 watermark_output_cols = 3;
repeated uint32 nondecreasing_exprs = 4;
// Whether there are likely no-op updates in the output chunks, so that eliminating them with
// `StreamChunk::eliminate_adjacent_noop_update` could be beneficial.
bool noop_update_hint = 5;
}

message FilterNode {
Expand Down
2 changes: 2 additions & 0 deletions src/frontend/planner_test/tests/testdata/input/agg.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -281,12 +281,14 @@
select distinct v1 from t;
expected_outputs:
- logical_plan
- stream_plan
- name: distinct with agg
sql: |
create table t (v1 int, v2 int);
select distinct sum(v1) from t group by v2;
expected_outputs:
- logical_plan
- stream_plan
- name: distinct on
sql: |
create table t (v1 int, v2 int, v3 int);
Expand Down
19 changes: 17 additions & 2 deletions src/frontend/planner_test/tests/testdata/output/agg.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -451,6 +451,12 @@
LogicalAgg { group_key: [t.v1], aggs: [] }
└─LogicalProject { exprs: [t.v1] }
└─LogicalScan { table: t, columns: [t.v1, t.v2, t._row_id] }
stream_plan: |-
StreamMaterialize { columns: [v1], stream_key: [v1], pk_columns: [v1], pk_conflict: NoCheck }
└─StreamProject { exprs: [t.v1], noop_update_hint: true }
└─StreamHashAgg { group_key: [t.v1], aggs: [count] }
└─StreamExchange { dist: HashShard(t.v1) }
└─StreamTableScan { table: t, columns: [t.v1, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) }
- name: distinct with agg
sql: |
create table t (v1 int, v2 int);
Expand All @@ -461,6 +467,15 @@
└─LogicalAgg { group_key: [t.v2], aggs: [sum(t.v1)] }
└─LogicalProject { exprs: [t.v2, t.v1] }
└─LogicalScan { table: t, columns: [t.v1, t.v2, t._row_id] }
stream_plan: |-
StreamMaterialize { columns: [sum], stream_key: [sum], pk_columns: [sum], pk_conflict: NoCheck }
└─StreamProject { exprs: [sum(t.v1)], noop_update_hint: true }
└─StreamHashAgg { group_key: [sum(t.v1)], aggs: [count] }
└─StreamExchange { dist: HashShard(sum(t.v1)) }
└─StreamProject { exprs: [t.v2, sum(t.v1)] }
└─StreamHashAgg { group_key: [t.v2], aggs: [sum(t.v1), count] }
└─StreamExchange { dist: HashShard(t.v2) }
└─StreamTableScan { table: t, columns: [t.v1, t.v2, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) }
- name: distinct on
sql: |
create table t (v1 int, v2 int, v3 int);
Expand Down Expand Up @@ -576,7 +591,7 @@
└─LogicalScan { table: t, columns: [t.v1] }
stream_plan: |-
StreamMaterialize { columns: [v1], stream_key: [v1], pk_columns: [v1], pk_conflict: NoCheck }
└─StreamProject { exprs: [t.v1] }
└─StreamProject { exprs: [t.v1], noop_update_hint: true }
└─StreamHashAgg { group_key: [t.v1], aggs: [count] }
└─StreamExchange { dist: HashShard(t.v1) }
└─StreamTableScan { table: t, columns: [t.v1, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) }
Expand Down Expand Up @@ -1797,7 +1812,7 @@
stream_plan: |-
StreamMaterialize { columns: [a, row_number], stream_key: [a], pk_columns: [a], pk_conflict: NoCheck }
└─StreamOverWindow { window_functions: [row_number() OVER(PARTITION BY t.a ORDER BY t.a DESC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] }
└─StreamProject { exprs: [t.a] }
└─StreamProject { exprs: [t.a], noop_update_hint: true }
└─StreamHashAgg { group_key: [t.a], aggs: [count] }
└─StreamExchange { dist: HashShard(t.a) }
└─StreamTableScan { table: t, columns: [t.a, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) }
Expand Down
4 changes: 2 additions & 2 deletions src/frontend/planner_test/tests/testdata/output/except.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,7 @@
└─BatchScan { table: t2, columns: [t2.a, t2.b, t2.c], distribution: SomeShard }
stream_plan: |-
StreamMaterialize { columns: [a, b, c], stream_key: [a, b, c], pk_columns: [a, b, c], pk_conflict: NoCheck }
└─StreamProject { exprs: [t1.a, t1.b, t1.c] }
└─StreamProject { exprs: [t1.a, t1.b, t1.c], noop_update_hint: true }
└─StreamHashAgg { group_key: [t1.a, t1.b, t1.c], aggs: [count] }
└─StreamHashJoin { type: LeftAnti, predicate: t1.a IS NOT DISTINCT FROM t2.a AND t1.b IS NOT DISTINCT FROM t2.b AND t1.c IS NOT DISTINCT FROM t2.c, output: all }
├─StreamExchange { dist: HashShard(t1.a, t1.b, t1.c) }
Expand All @@ -35,7 +35,7 @@
stream_dist_plan: |+
Fragment 0
StreamMaterialize { columns: [a, b, c], stream_key: [a, b, c], pk_columns: [a, b, c], pk_conflict: NoCheck } { tables: [ Materialize: 4294967294 ] }
└── StreamProject { exprs: [t1.a, t1.b, t1.c] }
└── StreamProject { exprs: [t1.a, t1.b, t1.c], noop_update_hint: true }
└── StreamHashAgg { group_key: [t1.a, t1.b, t1.c], aggs: [count] } { tables: [ HashAggState: 0 ] }
└── StreamHashJoin { type: LeftAnti, predicate: t1.a IS NOT DISTINCT FROM t2.a AND t1.b IS NOT DISTINCT FROM t2.b AND t1.c IS NOT DISTINCT FROM t2.c, output: all }
├── tables: [ HashJoinLeft: 1, HashJoinDegreeLeft: 2, HashJoinRight: 3, HashJoinDegreeRight: 4 ]
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,7 @@
└─BatchScan { table: t2, columns: [t2.a, t2.b, t2.c], distribution: SomeShard }
stream_plan: |-
StreamMaterialize { columns: [a, b, c], stream_key: [a, b, c], pk_columns: [a, b, c], pk_conflict: NoCheck }
└─StreamProject { exprs: [t1.a, t1.b, t1.c] }
└─StreamProject { exprs: [t1.a, t1.b, t1.c], noop_update_hint: true }
└─StreamHashAgg { group_key: [t1.a, t1.b, t1.c], aggs: [count] }
└─StreamHashJoin { type: LeftSemi, predicate: t1.a IS NOT DISTINCT FROM t2.a AND t1.b IS NOT DISTINCT FROM t2.b AND t1.c IS NOT DISTINCT FROM t2.c, output: all }
├─StreamExchange { dist: HashShard(t1.a, t1.b, t1.c) }
Expand All @@ -35,7 +35,7 @@
stream_dist_plan: |+
Fragment 0
StreamMaterialize { columns: [a, b, c], stream_key: [a, b, c], pk_columns: [a, b, c], pk_conflict: NoCheck } { tables: [ Materialize: 4294967294 ] }
└── StreamProject { exprs: [t1.a, t1.b, t1.c] }
└── StreamProject { exprs: [t1.a, t1.b, t1.c], noop_update_hint: true }
└── StreamHashAgg { group_key: [t1.a, t1.b, t1.c], aggs: [count] } { tables: [ HashAggState: 0 ] }
└── StreamHashJoin { type: LeftSemi, predicate: t1.a IS NOT DISTINCT FROM t2.a AND t1.b IS NOT DISTINCT FROM t2.b AND t1.c IS NOT DISTINCT FROM t2.c, output: all }
├── tables: [ HashJoinLeft: 1, HashJoinDegreeLeft: 2, HashJoinRight: 3, HashJoinDegreeRight: 4 ]
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -164,7 +164,7 @@
├─StreamExchange { dist: HashShard(t.arr) }
│ └─StreamTableScan { table: t, columns: [t.x, t.arr, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) }
└─StreamProjectSet { select_list: [$0, Unnest($0)] }
└─StreamProject { exprs: [t.arr] }
└─StreamProject { exprs: [t.arr], noop_update_hint: true }
└─StreamHashAgg { group_key: [t.arr], aggs: [count] }
└─StreamExchange { dist: HashShard(t.arr) }
└─StreamTableScan { table: t, columns: [t.arr, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) }
Expand Down Expand Up @@ -193,7 +193,7 @@
│ │ └─StreamTableScan { table: t1, columns: [t1.n, t1.id, t1._row_id, t1.c], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) }
│ └─StreamExchange { dist: HashShard(Unnest(Case(($1 <> '':Varchar), ArrayAppend(StringToArray(Trim($1, ',':Varchar), ',':Varchar), $3), Array($3)))) }
│ └─StreamProjectSet { select_list: [$0, $1, $2, $3, Unnest(Case(($1 <> '':Varchar), ArrayAppend(StringToArray(Trim($1, ',':Varchar), ',':Varchar), $3), Array($3)))] }
│ └─StreamProject { exprs: [t2.p, t2.p, t2.d, t2.d] }
│ └─StreamProject { exprs: [t2.p, t2.p, t2.d, t2.d], noop_update_hint: true }
│ └─StreamHashAgg { group_key: [t2.p, t2.p, t2.d, t2.d], aggs: [count] }
│ └─StreamExchange { dist: HashShard(t2.p, t2.p, t2.d, t2.d) }
│ └─StreamProject { exprs: [t2.p, t2.p, t2.d, t2.d, t2._row_id] }
Expand Down Expand Up @@ -226,7 +226,7 @@
└─StreamHashAgg { group_key: [t2.c7, t2.c7, t2.c8, t2.c7, t2.c7, t2.c8, t2.c1, t2.c5, t2.c5], aggs: [sum($expr4) filter(((t2.c7 * t1.c3) <= (t2.c7 * t2.c8))), sum(t1.c2) filter(((t2.c7 * t1.c3) <= (t2.c7 * t2.c8))), count] }
└─StreamHashJoin { type: LeftOuter, predicate: t2.c7 IS NOT DISTINCT FROM t2.c7 AND t2.c7 IS NOT DISTINCT FROM t2.c7 AND t2.c8 IS NOT DISTINCT FROM t2.c8 AND t2.c7 IS NOT DISTINCT FROM t2.c7 AND t2.c7 IS NOT DISTINCT FROM t2.c7 AND t2.c8 IS NOT DISTINCT FROM t2.c8 AND t2.c1 IS NOT DISTINCT FROM t2.c1 AND t2.c5 IS NOT DISTINCT FROM t2.c5 AND t2.c5 IS NOT DISTINCT FROM t2.c5, output: [t2.c7, t2.c7, t2.c8, t2.c7, t2.c7, t2.c8, t2.c1, t2.c5, t2.c5, t1.c3, $expr4, t1.c2, t2.c7, t2.c7, t2.c8, t2.c7, t2.c7, t2.c8, t2.c1, t2.c5, t2.c5, t1._row_id, $expr1] }
├─StreamExchange { dist: HashShard(t2.c7, t2.c7, t2.c8, t2.c7, t2.c7, t2.c8, t2.c1, t2.c5, t2.c5) }
│ └─StreamProject { exprs: [t2.c7, t2.c7, t2.c8, t2.c7, t2.c7, t2.c8, t2.c1, t2.c5, t2.c5] }
│ └─StreamProject { exprs: [t2.c7, t2.c7, t2.c8, t2.c7, t2.c7, t2.c8, t2.c1, t2.c5, t2.c5], noop_update_hint: true }
│ └─StreamHashAgg { group_key: [t2.c7, t2.c7, t2.c8, t2.c7, t2.c7, t2.c8, t2.c1, t2.c5, t2.c5], aggs: [count] }
│ └─StreamExchange { dist: HashShard(t2.c7, t2.c7, t2.c8, t2.c7, t2.c7, t2.c8, t2.c1, t2.c5, t2.c5) }
│ └─StreamTableScan { table: t2, columns: [t2.c7, t2.c7, t2.c8, t2.c7, t2.c7, t2.c8, t2.c1, t2.c5, t2.c5, t2._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t2._row_id], pk: [_row_id], dist: UpstreamHashShard(t2._row_id) }
Expand All @@ -235,7 +235,7 @@
└─StreamFilter { predicate: (t2.c5 <= $expr2) }
└─StreamHashJoin { type: Inner, predicate: t2.c1 = t1.c1 AND $expr1 = $expr3, output: all }
├─StreamExchange { dist: HashShard(t2.c1, $expr1) }
│ └─StreamProject { exprs: [t2.c7, t2.c7, t2.c8, t2.c7, t2.c7, t2.c8, t2.c1, t2.c5, t2.c5, AtTimeZone(t2.c5, 'UTC':Varchar)::Date as $expr1] }
│ └─StreamProject { exprs: [t2.c7, t2.c7, t2.c8, t2.c7, t2.c7, t2.c8, t2.c1, t2.c5, t2.c5, AtTimeZone(t2.c5, 'UTC':Varchar)::Date as $expr1], noop_update_hint: true }
│ └─StreamHashAgg { group_key: [t2.c7, t2.c7, t2.c8, t2.c7, t2.c7, t2.c8, t2.c1, t2.c5, t2.c5], aggs: [count] }
│ └─StreamExchange { dist: HashShard(t2.c7, t2.c7, t2.c8, t2.c7, t2.c7, t2.c8, t2.c1, t2.c5, t2.c5) }
│ └─StreamTableScan { table: t2, columns: [t2.c7, t2.c7, t2.c8, t2.c7, t2.c7, t2.c8, t2.c1, t2.c5, t2.c5, t2._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t2._row_id], pk: [_row_id], dist: UpstreamHashShard(t2._row_id) }
Expand Down
4 changes: 2 additions & 2 deletions src/frontend/planner_test/tests/testdata/output/nexmark.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -748,7 +748,7 @@
│ └─StreamProject { exprs: [person.id, person.name, $expr1, ($expr1 + '00:00:10':Interval) as $expr2] }
│ └─StreamProject { exprs: [person.id, person.name, person.date_time, TumbleStart(person.date_time, '00:00:10':Interval) as $expr1] }
│ └─StreamTableScan { table: person, columns: [person.id, person.name, person.date_time], stream_scan_type: ArrangementBackfill, stream_key: [person.id], pk: [id], dist: UpstreamHashShard(person.id) }
└─StreamProject { exprs: [auction.seller, $expr3, $expr4] }
└─StreamProject { exprs: [auction.seller, $expr3, $expr4], noop_update_hint: true }
└─StreamHashAgg { group_key: [auction.seller, $expr3, $expr4], aggs: [count] }
└─StreamExchange { dist: HashShard(auction.seller, $expr3, $expr4) }
└─StreamProject { exprs: [auction.seller, $expr3, ($expr3 + '00:00:10':Interval) as $expr4, auction.id] }
Expand All @@ -764,7 +764,7 @@
StreamHashJoin { type: Inner, predicate: person.id = auction.seller AND $expr1 = $expr3 AND $expr2 = $expr4, output: [person.id, internal_last_seen_value(person.name), $expr1, $expr2, auction.seller, $expr3, $expr4] }
├── tables: [ HashJoinLeft: 0, HashJoinDegreeLeft: 1, HashJoinRight: 2, HashJoinDegreeRight: 3 ]
├── StreamExchange Hash([0, 1, 2]) from 2
└── StreamProject { exprs: [auction.seller, $expr3, $expr4] }
└── StreamProject { exprs: [auction.seller, $expr3, $expr4], noop_update_hint: true }
└── StreamHashAgg { group_key: [auction.seller, $expr3, $expr4], aggs: [count] } { tables: [ HashAggState: 6 ] }
└── StreamExchange Hash([0, 1, 2]) from 3

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -77,7 +77,7 @@
└─LogicalScan { table: t, columns: [t.v1, t.v2, t.v3] }
stream_plan: |-
StreamMaterialize { columns: [v1, v2, v3], stream_key: [v1, v2, v3], pk_columns: [v1, v2, v3], pk_conflict: NoCheck }
└─StreamProject { exprs: [t.v1, t.v2, t.v3] }
└─StreamProject { exprs: [t.v1, t.v2, t.v3], noop_update_hint: true }
└─StreamHashAgg { group_key: [t.v1, t.v2, t.v3], aggs: [count] }
└─StreamExchange { dist: HashShard(t.v1, t.v2, t.v3) }
└─StreamTableScan { table: t, columns: [t.v1, t.v2, t.v3, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) }
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -118,7 +118,7 @@
└─BatchScan { table: t, columns: [t.x], distribution: SomeShard }
stream_plan: |-
StreamMaterialize { columns: [unnest], stream_key: [unnest], pk_columns: [unnest], pk_conflict: NoCheck }
└─StreamProject { exprs: [Unnest($0)] }
└─StreamProject { exprs: [Unnest($0)], noop_update_hint: true }
└─StreamHashAgg { group_key: [Unnest($0)], aggs: [count] }
└─StreamExchange { dist: HashShard(Unnest($0)) }
└─StreamProjectSet { select_list: [Unnest($0), $1] }
Expand Down
6 changes: 3 additions & 3 deletions src/frontend/planner_test/tests/testdata/output/share.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -222,12 +222,12 @@
└─StreamStatelessSimpleAgg { aggs: [count] }
└─StreamHashJoin { type: Inner, predicate: t.a = t.a, output: all }
├─StreamShare { id: 4 }
│ └─StreamProject { exprs: [t.a] }
│ └─StreamProject { exprs: [t.a], noop_update_hint: true }
│ └─StreamHashAgg { group_key: [t.a], aggs: [count] }
│ └─StreamExchange { dist: HashShard(t.a) }
│ └─StreamTableScan { table: t, columns: [t.a, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) }
└─StreamShare { id: 4 }
└─StreamProject { exprs: [t.a] }
└─StreamProject { exprs: [t.a], noop_update_hint: true }
└─StreamHashAgg { group_key: [t.a], aggs: [count] }
└─StreamExchange { dist: HashShard(t.a) }
└─StreamTableScan { table: t, columns: [t.a, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) }
Expand All @@ -251,7 +251,7 @@
└── StreamExchange NoShuffle from 4

Fragment 2
StreamProject { exprs: [t.a] }
StreamProject { exprs: [t.a], noop_update_hint: true }
└── StreamHashAgg { group_key: [t.a], aggs: [count] } { tables: [ HashAggState: 5 ] }
└── StreamExchange Hash([0]) from 3

Expand Down
10 changes: 5 additions & 5 deletions src/frontend/planner_test/tests/testdata/output/subquery.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -331,9 +331,9 @@
│ └─StreamHopWindow { time_col: auction.date_time, slide: 00:00:01, size: 01:00:00, output: [auction.date_time, window_start, window_end, auction._row_id] }
│ └─StreamFilter { predicate: IsNotNull(auction.date_time) }
│ └─StreamTableScan { table: auction, columns: [auction.date_time, auction._row_id], stream_scan_type: ArrangementBackfill, stream_key: [auction._row_id], pk: [_row_id], dist: UpstreamHashShard(auction._row_id) }
└─StreamProject { exprs: [auction.date_time] }
└─StreamProject { exprs: [auction.date_time], noop_update_hint: true }
└─StreamHashAgg { group_key: [auction.date_time], aggs: [count] }
└─StreamProject { exprs: [auction.date_time] }
└─StreamProject { exprs: [auction.date_time], noop_update_hint: true }
└─StreamHashAgg { group_key: [auction.date_time], aggs: [count] }
└─StreamExchange { dist: HashShard(auction.date_time) }
└─StreamShare { id: 3 }
Expand Down Expand Up @@ -529,13 +529,13 @@
└─StreamProject { exprs: [t.x, sum(Unnest($0))] }
└─StreamHashAgg { group_key: [t.x], aggs: [sum(Unnest($0)), count] }
└─StreamHashJoin { type: LeftOuter, predicate: t.x IS NOT DISTINCT FROM t.x, output: [t.x, Unnest($0), t.x, projected_row_id] }
├─StreamProject { exprs: [t.x] }
├─StreamProject { exprs: [t.x], noop_update_hint: true }
│ └─StreamHashAgg { group_key: [t.x], aggs: [count] }
│ └─StreamExchange { dist: HashShard(t.x) }
│ └─StreamTableScan { table: t, columns: [t.x, t.k], stream_scan_type: ArrangementBackfill, stream_key: [t.k], pk: [k], dist: UpstreamHashShard(t.k) }
└─StreamProject { exprs: [t.x, Unnest($0), projected_row_id] }
└─StreamProjectSet { select_list: [$0, Unnest($0)] }
└─StreamProject { exprs: [t.x] }
└─StreamProject { exprs: [t.x], noop_update_hint: true }
└─StreamHashAgg { group_key: [t.x], aggs: [count] }
└─StreamExchange { dist: HashShard(t.x) }
└─StreamTableScan { table: t, columns: [t.x, t.k], stream_scan_type: ArrangementBackfill, stream_key: [t.k], pk: [k], dist: UpstreamHashShard(t.k) }
Expand Down Expand Up @@ -817,7 +817,7 @@
└─StreamProject { exprs: [integers.correlated_col, (count(distinct rows.k) + count(distinct rows.v)) as $expr1] }
└─StreamHashAgg { group_key: [integers.correlated_col], aggs: [count(distinct rows.k), count(distinct rows.v), count] }
└─StreamHashJoin { type: LeftOuter, predicate: integers.correlated_col IS NOT DISTINCT FROM rows.correlated_col, output: [integers.correlated_col, rows.k, rows.v, rows._row_id] }
├─StreamProject { exprs: [integers.correlated_col] }
├─StreamProject { exprs: [integers.correlated_col], noop_update_hint: true }
│ └─StreamHashAgg { group_key: [integers.correlated_col], aggs: [count] }
│ └─StreamExchange { dist: HashShard(integers.correlated_col) }
│ └─StreamTableScan { table: integers, columns: [integers.correlated_col, integers._row_id], stream_scan_type: ArrangementBackfill, stream_key: [integers._row_id], pk: [_row_id], dist: UpstreamHashShard(integers._row_id) }
Expand Down
Loading
Loading