Skip to content

Commit

Permalink
dapt
Browse files Browse the repository at this point in the history
  • Loading branch information
kwannoel committed Apr 9, 2024
1 parent b297bd9 commit e7fa42d
Show file tree
Hide file tree
Showing 5 changed files with 104 additions and 107 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -2170,7 +2170,7 @@
└─BatchFilter { predicate: (orders.o_carrier_id > 8:Int32) }
└─BatchScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_carrier_id], distribution: UpstreamHashShard(orders.o_w_id, orders.o_d_id, orders.o_id) }
stream_plan: |-
StreamMaterialize { columns: [c_count, custdist], stream_key: [c_count], pk_columns: [c_count], pk_conflict: NoCheck }
StreamMaterialize { columns: [c_count, custdist], stream_key: [c_count], pk_columns: [custdist, c_count], pk_conflict: NoCheck }
└─StreamHashAgg { group_key: [count(orders.o_id)], aggs: [count] }
└─StreamExchange { dist: HashShard(count(orders.o_id)) }
└─StreamProject { exprs: [customer.c_id, count(orders.o_id)] }
Expand All @@ -2185,7 +2185,7 @@
└─StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_carrier_id], stream_scan_type: ArrangementBackfill, stream_key: [orders.o_w_id, orders.o_d_id, orders.o_id], pk: [o_w_id, o_d_id, o_id], dist: UpstreamHashShard(orders.o_w_id, orders.o_d_id, orders.o_id) }
stream_dist_plan: |+
Fragment 0
StreamMaterialize { columns: [c_count, custdist], stream_key: [c_count], pk_columns: [c_count], pk_conflict: NoCheck }
StreamMaterialize { columns: [c_count, custdist], stream_key: [c_count], pk_columns: [custdist, c_count], pk_conflict: NoCheck }
├── tables: [ Materialize: 4294967294 ]
└── StreamHashAgg { group_key: [count(orders.o_id)], aggs: [count] } { tables: [ HashAggState: 0 ] }
└── StreamExchange Hash([1]) from 1
Expand Down Expand Up @@ -2231,7 +2231,7 @@
Table 7 { columns: [ vnode, o_w_id, o_d_id, o_id, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 }
Table 4294967294 { columns: [ c_count, custdist ], primary key: [ $0 DESC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 }
Table 4294967294 { columns: [ c_count, custdist ], primary key: [ $1 DESC, $0 DESC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 2 }
- id: ch_q14
before:
Expand Down Expand Up @@ -3177,7 +3177,7 @@
└─BatchProject { exprs: [order_line.ol_o_id, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_delivery_d] }
└─BatchScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_delivery_d], distribution: SomeShard }
stream_plan: |-
StreamMaterialize { columns: [s_name, numwait], stream_key: [s_name], pk_columns: [s_name], pk_conflict: NoCheck }
StreamMaterialize { columns: [s_name, numwait], stream_key: [s_name], pk_columns: [numwait, s_name], pk_conflict: NoCheck }
└─StreamHashAgg { group_key: [supplier.s_name], aggs: [count] }
└─StreamExchange { dist: HashShard(supplier.s_name) }
└─StreamHashJoin { type: LeftAnti, predicate: order_line.ol_o_id = order_line.ol_o_id AND order_line.ol_w_id = order_line.ol_w_id AND order_line.ol_d_id = order_line.ol_d_id AND (order_line.ol_delivery_d > order_line.ol_delivery_d), output: [supplier.s_name, stock.s_w_id, stock.s_i_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, supplier.s_nationkey, $expr1, order_line.ol_w_id] }
Expand Down Expand Up @@ -3210,7 +3210,7 @@
└─StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_delivery_d, order_line.ol_number], stream_scan_type: ArrangementBackfill, stream_key: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], pk: [ol_w_id, ol_d_id, ol_o_id, ol_number], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number) }
stream_dist_plan: |+
Fragment 0
StreamMaterialize { columns: [s_name, numwait], stream_key: [s_name], pk_columns: [s_name], pk_conflict: NoCheck }
StreamMaterialize { columns: [s_name, numwait], stream_key: [s_name], pk_columns: [numwait, s_name], pk_conflict: NoCheck }
├── tables: [ Materialize: 4294967294 ]
└── StreamHashAgg { group_key: [supplier.s_name], aggs: [count] } { tables: [ HashAggState: 0 ] }
└── StreamExchange Hash([0]) from 1
Expand Down Expand Up @@ -3334,7 +3334,7 @@
Table 26 { columns: [ vnode, ol_w_id, ol_d_id, ol_o_id, ol_number, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4, 5, 6 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 }
Table 4294967294 { columns: [ s_name, numwait ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 }
Table 4294967294 { columns: [ s_name, numwait ], primary key: [ $1 DESC, $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 2 }
- id: ch_q22
before:
Expand Down
17 changes: 8 additions & 9 deletions src/frontend/planner_test/tests/testdata/output/limit.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -130,7 +130,7 @@
└─BatchScan { table: t, columns: [], distribution: SomeShard }
stream_plan: |-
StreamMaterialize { columns: [count], stream_key: [], pk_columns: [], pk_conflict: NoCheck }
└─StreamTopN { order: [], limit: 1, offset: 0 }
└─StreamTopN { order: [sum0(count) ASC], limit: 1, offset: 0 }
└─StreamProject { exprs: [sum0(count)] }
└─StreamSimpleAgg { aggs: [sum0(count), count] }
└─StreamExchange { dist: Single }
Expand All @@ -140,15 +140,14 @@
create table t (a int);
select count(*) from t order by 1 limit 1;
batch_plan: |-
BatchSort { order: [sum0(count) ASC] }
└─BatchLimit { limit: 1, offset: 0 }
└─BatchSimpleAgg { aggs: [sum0(count)] }
└─BatchExchange { order: [], dist: Single }
└─BatchSimpleAgg { aggs: [count] }
└─BatchScan { table: t, columns: [], distribution: SomeShard }
BatchTopN { order: [sum0(count) ASC], limit: 1, offset: 0 }
└─BatchSimpleAgg { aggs: [sum0(count)] }
└─BatchExchange { order: [], dist: Single }
└─BatchSimpleAgg { aggs: [count] }
└─BatchScan { table: t, columns: [], distribution: SomeShard }
stream_plan: |-
StreamMaterialize { columns: [count], stream_key: [], pk_columns: [], pk_conflict: NoCheck }
└─StreamTopN { order: [], limit: 1, offset: 0 }
StreamMaterialize { columns: [count], stream_key: [], pk_columns: [count], pk_conflict: NoCheck }
└─StreamTopN { order: [sum0(count) ASC], limit: 1, offset: 0 }
└─StreamProject { exprs: [sum0(count)] }
└─StreamSimpleAgg { aggs: [sum0(count), count] }
└─StreamExchange { dist: Single }
Expand Down
7 changes: 3 additions & 4 deletions src/frontend/planner_test/tests/testdata/output/topn.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -5,10 +5,9 @@
SELECT sa, count(*) as cnt2 FROM t1_mv GROUP BY sa ORDER BY cnt2 DESC LIMIT 50 OFFSET 50;
batch_plan: |-
BatchExchange { order: [count DESC], dist: Single }
└─BatchSort { order: [count DESC] }
└─BatchLimit { limit: 50, offset: 50 }
└─BatchSimpleAgg { aggs: [internal_last_seen_value(t1_mv.sa), count] }
└─BatchScan { table: t1_mv, columns: [t1_mv.sa], distribution: Single }
└─BatchTopN { order: [count DESC], limit: 50, offset: 50 }
└─BatchSimpleAgg { aggs: [internal_last_seen_value(t1_mv.sa), count] }
└─BatchScan { table: t1_mv, columns: [t1_mv.sa], distribution: Single }
- sql: |
CREATE TABLE t1 (pk int, a int, b int, c bigint, d int);
CREATE MATERIALIZED VIEW t1_mv AS SELECT * from t1;
Expand Down
Loading

0 comments on commit e7fa42d

Please sign in to comment.