diff --git a/e2e_test/streaming/aggregate/count_star.slt b/e2e_test/streaming/aggregate/count_star.slt index 134a850930eae..2bc6c88c96fd1 100644 --- a/e2e_test/streaming/aggregate/count_star.slt +++ b/e2e_test/streaming/aggregate/count_star.slt @@ -1,3 +1,6 @@ +statement ok +SET RW_IMPLICIT_FLUSH TO true; + statement ok create table t (v int); @@ -7,9 +10,6 @@ insert into t values (114), (514); statement ok create materialized view mv as select * from t; -statement ok -flush; - query I select count(*) from t; ---- diff --git a/src/frontend/planner_test/tests/testdata/output/batch_source.yaml b/src/frontend/planner_test/tests/testdata/output/batch_source.yaml index 8613f28305667..63daa9c43b2db 100644 --- a/src/frontend/planner_test/tests/testdata/output/batch_source.yaml +++ b/src/frontend/planner_test/tests/testdata/output/batch_source.yaml @@ -3,11 +3,11 @@ select * from s logical_plan: |- LogicalProject { exprs: [id, value] } - └─LogicalSource { source: s, columns: [id, value, _rw_kafka_timestamp, _row_id], time_range: (Unbounded, Unbounded) } + └─LogicalSource { source: s, columns: [id, value, _rw_kafka_timestamp, _row_id] } batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchProject { exprs: [id, value] } - └─BatchSource { source: s, columns: [id, value, _rw_kafka_timestamp, _row_id], filter: (None, None) } + └─BatchKafkaScan { source: s, columns: [id, value, _rw_kafka_timestamp, _row_id], filter: (None, None) } create_source: format: plain encode: protobuf diff --git a/src/frontend/planner_test/tests/testdata/output/generated_columns.yaml b/src/frontend/planner_test/tests/testdata/output/generated_columns.yaml index e88df797de0c8..3ed95c1ac1463 100644 --- a/src/frontend/planner_test/tests/testdata/output/generated_columns.yaml +++ b/src/frontend/planner_test/tests/testdata/output/generated_columns.yaml @@ -17,7 +17,7 @@ batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchProject { exprs: [(v2 + 1:Int32) as $expr1] } - └─BatchSource { source: s1, columns: [v2, _row_id], filter: (None, None) } + └─BatchSource { source: s1, columns: [v2, _row_id] } - name: select proctime() sql: | select proctime(); diff --git a/src/frontend/planner_test/tests/testdata/output/nexmark_source.yaml b/src/frontend/planner_test/tests/testdata/output/nexmark_source.yaml index 2e9484a4c1edf..823fa85459df7 100644 --- a/src/frontend/planner_test/tests/testdata/output/nexmark_source.yaml +++ b/src/frontend/planner_test/tests/testdata/output/nexmark_source.yaml @@ -51,7 +51,7 @@ batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchProject { exprs: [auction, bidder, price, date_time] } - └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], filter: (None, None) } + └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } stream_plan: |- StreamMaterialize { columns: [auction, bidder, price, date_time, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: NoCheck } └─StreamProject { exprs: [auction, bidder, price, date_time, _row_id] } @@ -87,7 +87,7 @@ batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchProject { exprs: [auction, bidder, (0.908:Decimal * price::Decimal) as $expr1, date_time] } - └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], filter: (None, None) } + └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } stream_plan: |- StreamMaterialize { columns: [auction, bidder, price, date_time, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: NoCheck } └─StreamProject { exprs: [auction, bidder, (0.908:Decimal * price::Decimal) as $expr1, date_time, _row_id] } @@ -118,7 +118,7 @@ BatchExchange { order: [], dist: Single } └─BatchFilter { predicate: (((((auction = 1007:Int32) OR (auction = 1020:Int32)) OR (auction = 2001:Int32)) OR (auction = 2019:Int32)) OR (auction = 2087:Int32)) } └─BatchProject { exprs: [auction, price] } - └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], filter: (None, None) } + └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } stream_plan: |- StreamMaterialize { columns: [auction, price, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: NoCheck } └─StreamProject { exprs: [auction, price, _row_id] } @@ -154,11 +154,11 @@ ├─BatchExchange { order: [], dist: HashShard(seller) } │ └─BatchFilter { predicate: (category = 10:Int32) } │ └─BatchProject { exprs: [id, seller, category] } - │ └─BatchSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id], filter: (None, None) } + │ └─BatchSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id] } └─BatchExchange { order: [], dist: HashShard(id) } └─BatchFilter { predicate: (((state = 'or':Varchar) OR (state = 'id':Varchar)) OR (state = 'ca':Varchar)) } └─BatchProject { exprs: [id, name, city, state] } - └─BatchSource { source: person, columns: [id, name, email_address, credit_card, city, state, date_time, extra, _row_id], filter: (None, None) } + └─BatchSource { source: person, columns: [id, name, email_address, credit_card, city, state, date_time, extra, _row_id] } stream_plan: |- StreamMaterialize { columns: [name, city, state, id, _row_id(hidden), seller(hidden), _row_id#1(hidden)], stream_key: [_row_id, _row_id#1, seller], pk_columns: [_row_id, _row_id#1, seller], pk_conflict: NoCheck } └─StreamExchange { dist: HashShard(_row_id, seller, _row_id) } @@ -244,9 +244,9 @@ └─BatchHashAgg { group_key: [id, category], aggs: [max(price)] } └─BatchHashJoin { type: Inner, predicate: id = auction AND (date_time >= date_time) AND (date_time <= expires), output: [id, category, price] } ├─BatchExchange { order: [], dist: HashShard(id) } - │ └─BatchSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id], filter: (None, None) } + │ └─BatchSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id] } └─BatchExchange { order: [], dist: HashShard(auction) } - └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], filter: (None, None) } + └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } stream_plan: |- StreamMaterialize { columns: [category, avg], stream_key: [category], pk_columns: [category], pk_conflict: NoCheck } └─StreamProject { exprs: [category, (sum(max(price)) / count(max(price))::Decimal) as $expr1] } @@ -380,7 +380,7 @@ │ └─LogicalProject { exprs: [window_start, auction] } │ └─LogicalHopWindow { time_col: date_time, slide: 00:00:02, size: 00:00:10, output: all } │ └─LogicalFilter { predicate: IsNotNull(date_time) } - │ └─LogicalSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], time_range: (Unbounded, Unbounded) } + │ └─LogicalSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } └─LogicalProject { exprs: [max(count), window_start] } └─LogicalAgg { group_key: [window_start], aggs: [max(count)] } └─LogicalProject { exprs: [window_start, count] } @@ -389,7 +389,7 @@ └─LogicalProject { exprs: [auction, window_start] } └─LogicalHopWindow { time_col: date_time, slide: 00:00:02, size: 00:00:10, output: all } └─LogicalFilter { predicate: IsNotNull(date_time) } - └─LogicalSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], time_range: (Unbounded, Unbounded) } + └─LogicalSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchHashJoin { type: Inner, predicate: window_start = window_start AND (count >= max(count)), output: [auction, count] } @@ -399,7 +399,7 @@ │ └─BatchExchange { order: [], dist: HashShard(auction) } │ └─BatchProject { exprs: [auction, date_time] } │ └─BatchFilter { predicate: IsNotNull(date_time) } - │ └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], filter: (None, None) } + │ └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } └─BatchHashAgg { group_key: [window_start], aggs: [max(count)] } └─BatchExchange { order: [], dist: HashShard(window_start) } └─BatchHashAgg { group_key: [auction, window_start], aggs: [count] } @@ -407,7 +407,7 @@ └─BatchExchange { order: [], dist: HashShard(auction) } └─BatchProject { exprs: [auction, date_time] } └─BatchFilter { predicate: IsNotNull(date_time) } - └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], filter: (None, None) } + └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } stream_plan: |- StreamMaterialize { columns: [auction, num, window_start(hidden), window_start#1(hidden)], stream_key: [auction, window_start], pk_columns: [auction, window_start], pk_conflict: NoCheck } └─StreamProject { exprs: [auction, count, window_start, window_start] } @@ -614,13 +614,13 @@ BatchExchange { order: [], dist: Single } └─BatchHashJoin { type: Inner, predicate: price = max(price) AND (date_time >= $expr2) AND (date_time <= $expr1), output: [auction, price, bidder, date_time] } ├─BatchExchange { order: [], dist: HashShard(price) } - │ └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], filter: (None, None) } + │ └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } └─BatchExchange { order: [], dist: HashShard(max(price)) } └─BatchProject { exprs: [max(price), $expr1, ($expr1 - '00:00:10':Interval) as $expr2] } └─BatchHashAgg { group_key: [$expr1], aggs: [max(price)] } └─BatchExchange { order: [], dist: HashShard($expr1) } └─BatchProject { exprs: [(TumbleStart(date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr1, price] } - └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], filter: (None, None) } + └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } stream_plan: |- StreamMaterialize { columns: [auction, price, bidder, date_time, _row_id(hidden), $expr1(hidden)], stream_key: [_row_id, $expr1, price], pk_columns: [_row_id, $expr1, price], pk_conflict: NoCheck } └─StreamProject { exprs: [auction, price, bidder, date_time, _row_id, $expr1] } @@ -731,12 +731,12 @@ │ └─BatchExchange { order: [], dist: HashShard(id, name, $expr1, $expr2) } │ └─BatchProject { exprs: [id, name, $expr1, ($expr1 + '00:00:10':Interval) as $expr2] } │ └─BatchProject { exprs: [id, name, email_address, credit_card, city, state, date_time, extra, _row_id, TumbleStart(date_time, '00:00:10':Interval) as $expr1] } - │ └─BatchSource { source: person, columns: [id, name, email_address, credit_card, city, state, date_time, extra, _row_id], filter: (None, None) } + │ └─BatchSource { source: person, columns: [id, name, email_address, credit_card, city, state, date_time, extra, _row_id] } └─BatchHashAgg { group_key: [seller, $expr3, $expr4], aggs: [] } └─BatchExchange { order: [], dist: HashShard(seller, $expr3, $expr4) } └─BatchProject { exprs: [seller, $expr3, ($expr3 + '00:00:10':Interval) as $expr4] } └─BatchProject { exprs: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id, TumbleStart(date_time, '00:00:10':Interval) as $expr3] } - └─BatchSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id], filter: (None, None) } + └─BatchSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id] } stream_plan: |- StreamMaterialize { columns: [id, name, starttime, $expr2(hidden), seller(hidden), $expr3(hidden), $expr4(hidden)], stream_key: [id, name, starttime, $expr2], pk_columns: [id, name, starttime, $expr2], pk_conflict: NoCheck } └─StreamExchange { dist: HashShard(id, name, $expr1, $expr2) } @@ -827,21 +827,21 @@ └─LogicalProject { exprs: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id, auction, bidder, price, channel, url, date_time, extra, _row_id] } └─LogicalFilter { predicate: (id = auction) AND (date_time >= date_time) AND (date_time <= expires) } └─LogicalJoin { type: Inner, on: true, output: all } - ├─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id], time_range: (Unbounded, Unbounded) } - └─LogicalSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], time_range: (Unbounded, Unbounded) } + ├─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id] } + └─LogicalSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } optimized_logical_plan_for_batch: |- LogicalTopN { order: [price DESC, date_time ASC], limit: 1, offset: 0, group_key: [id] } └─LogicalJoin { type: Inner, on: (id = auction) AND (date_time >= date_time) AND (date_time <= expires), output: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, auction, bidder, price, date_time] } - ├─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id], time_range: (Unbounded, Unbounded) } - └─LogicalSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], time_range: (Unbounded, Unbounded) } + ├─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id] } + └─LogicalSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchGroupTopN { order: [price DESC, date_time ASC], limit: 1, offset: 0, group_key: [id] } └─BatchHashJoin { type: Inner, predicate: id = auction AND (date_time >= date_time) AND (date_time <= expires), output: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, auction, bidder, price, date_time] } ├─BatchExchange { order: [], dist: HashShard(id) } - │ └─BatchSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id], filter: (None, None) } + │ └─BatchSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id] } └─BatchExchange { order: [], dist: HashShard(auction) } - └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], filter: (None, None) } + └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } stream_plan: |- StreamMaterialize { columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, auction, bidder, price, bid_date_time, _row_id(hidden), _row_id#1(hidden)], stream_key: [id], pk_columns: [id], pk_conflict: NoCheck } └─StreamGroupTopN [append_only] { order: [price DESC, date_time ASC], limit: 1, offset: 0, group_key: [id] } @@ -912,7 +912,7 @@ batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchProject { exprs: [auction, bidder, price, date_time, ToChar(date_time, 'YYYY-MM-DD':Varchar) as $expr1, ToChar(date_time, 'HH:MI':Varchar) as $expr2] } - └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], filter: (None, None) } + └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } stream_plan: |- StreamMaterialize { columns: [auction, bidder, price, date_time, date, time, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: NoCheck } └─StreamProject { exprs: [auction, bidder, price, date_time, ToChar(date_time, 'YYYY-MM-DD':Varchar) as $expr1, ToChar(date_time, 'HH:MI':Varchar) as $expr2, _row_id] } @@ -1012,7 +1012,7 @@ └─BatchProject { exprs: [auction, bidder, (0.908:Decimal * price::Decimal) as $expr1, Case(((Extract('HOUR':Varchar, date_time) >= 8:Decimal) AND (Extract('HOUR':Varchar, date_time) <= 18:Decimal)), 'dayTime':Varchar, ((Extract('HOUR':Varchar, date_time) <= 6:Decimal) OR (Extract('HOUR':Varchar, date_time) >= 20:Decimal)), 'nightTime':Varchar, 'otherTime':Varchar) as $expr2, date_time, extra] } └─BatchFilter { predicate: ((0.908:Decimal * price::Decimal) > 1000000:Decimal) AND ((0.908:Decimal * price::Decimal) < 50000000:Decimal) } └─BatchProject { exprs: [auction, bidder, price, date_time, extra] } - └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], filter: (None, None) } + └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } stream_plan: |- StreamMaterialize { columns: [auction, bidder, price, bidtimetype, date_time, extra, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: NoCheck } └─StreamProject { exprs: [auction, bidder, (0.908:Decimal * price::Decimal) as $expr1, Case(((Extract('HOUR':Varchar, date_time) >= 8:Decimal) AND (Extract('HOUR':Varchar, date_time) <= 18:Decimal)), 'dayTime':Varchar, ((Extract('HOUR':Varchar, date_time) <= 6:Decimal) OR (Extract('HOUR':Varchar, date_time) >= 20:Decimal)), 'nightTime':Varchar, 'otherTime':Varchar) as $expr2, date_time, extra, _row_id] } @@ -1059,7 +1059,7 @@ └─BatchExchange { order: [], dist: HashShard($expr1_expanded, bidder_expanded, auction_expanded, flag) } └─BatchExpand { column_subsets: [[$expr1], [$expr1, bidder], [$expr1, auction]] } └─BatchProject { exprs: [ToChar(date_time, 'yyyy-MM-dd':Varchar) as $expr1, price, bidder, auction] } - └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], filter: (None, None) } + └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } stream_plan: |- StreamMaterialize { columns: [day, total_bids, rank1_bids, rank2_bids, rank3_bids, total_bidders, rank1_bidders, rank2_bidders, rank3_bidders, total_auctions, rank1_auctions, rank2_auctions, rank3_auctions], stream_key: [day], pk_columns: [day], pk_conflict: NoCheck } └─StreamHashAgg [append_only] { group_key: [$expr1], aggs: [count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count(distinct bidder), count(distinct bidder) filter((price < 10000:Int32)), count(distinct bidder) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct bidder) filter((price >= 1000000:Int32)), count(distinct auction), count(distinct auction) filter((price < 10000:Int32)), count(distinct auction) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct auction) filter((price >= 1000000:Int32))] } @@ -1265,7 +1265,7 @@ └─BatchExchange { order: [], dist: HashShard(channel_expanded, $expr1_expanded, bidder_expanded, auction_expanded, flag) } └─BatchExpand { column_subsets: [[channel, $expr1, $expr2], [channel, $expr1, bidder], [channel, $expr1, auction]] } └─BatchProject { exprs: [channel, ToChar(date_time, 'yyyy-MM-dd':Varchar) as $expr1, ToChar(date_time, 'HH:mm':Varchar) as $expr2, price, bidder, auction] } - └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], filter: (None, None) } + └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } stream_plan: |- StreamMaterialize { columns: [channel, day, minute, total_bids, rank1_bids, rank2_bids, rank3_bids, total_bidders, rank1_bidders, rank2_bidders, rank3_bidders, total_auctions, rank1_auctions, rank2_auctions, rank3_auctions], stream_key: [channel, day], pk_columns: [channel, day], pk_conflict: NoCheck } └─StreamHashAgg [append_only] { group_key: [channel, $expr1], aggs: [max($expr2), count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count(distinct bidder), count(distinct bidder) filter((price < 10000:Int32)), count(distinct bidder) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct bidder) filter((price >= 1000000:Int32)), count(distinct auction), count(distinct auction) filter((price < 10000:Int32)), count(distinct auction) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct auction) filter((price >= 1000000:Int32))] } @@ -1391,7 +1391,7 @@ └─BatchHashAgg { group_key: [auction, $expr1], aggs: [count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), min(price), max(price), sum(price), count(price)] } └─BatchExchange { order: [], dist: HashShard(auction, $expr1) } └─BatchProject { exprs: [auction, ToChar(date_time, 'YYYY-MM-DD':Varchar) as $expr1, price] } - └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], filter: (None, None) } + └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } stream_plan: |- StreamMaterialize { columns: [auction, day, total_bids, rank1_bids, rank2_bids, rank3_bids, min_price, max_price, avg_price, sum_price], stream_key: [auction, day], pk_columns: [auction, day], pk_conflict: NoCheck } └─StreamProject { exprs: [auction, $expr1, count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), min(price), max(price), (sum(price) / count(price)::Decimal) as $expr2, sum(price)] } @@ -1438,13 +1438,13 @@ └─LogicalProject { exprs: [auction, bidder, price, channel, url, date_time, extra, row_number] } └─LogicalOverWindow { window_functions: [row_number() OVER(PARTITION BY bidder, auction ORDER BY date_time DESC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } └─LogicalProject { exprs: [auction, bidder, price, channel, url, date_time, extra, _row_id] } - └─LogicalSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], time_range: (Unbounded, Unbounded) } + └─LogicalSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchGroupTopN { order: [date_time DESC], limit: 1, offset: 0, group_key: [bidder, auction] } └─BatchExchange { order: [], dist: HashShard(bidder, auction) } └─BatchProject { exprs: [auction, bidder, price, channel, url, date_time, extra] } - └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], filter: (None, None) } + └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } stream_plan: |- StreamMaterialize { columns: [auction, bidder, price, channel, url, date_time, extra, _row_id(hidden)], stream_key: [bidder, auction], pk_columns: [bidder, auction], pk_conflict: NoCheck } └─StreamGroupTopN [append_only] { order: [date_time DESC], limit: 1, offset: 0, group_key: [bidder, auction] } @@ -1494,13 +1494,13 @@ └─LogicalProject { exprs: [auction, bidder, price, channel, url, date_time, extra, rank] } └─LogicalOverWindow { window_functions: [rank() OVER(PARTITION BY bidder, auction ORDER BY date_time DESC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } └─LogicalProject { exprs: [auction, bidder, price, channel, url, date_time, extra, _row_id] } - └─LogicalSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], time_range: (Unbounded, Unbounded) } + └─LogicalSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchGroupTopN { order: [date_time DESC], limit: 1, offset: 0, with_ties: true, group_key: [bidder, auction] } └─BatchExchange { order: [], dist: HashShard(bidder, auction) } └─BatchProject { exprs: [auction, bidder, price, channel, url, date_time, extra] } - └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], filter: (None, None) } + └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } stream_plan: |- StreamMaterialize { columns: [auction, bidder, price, channel, url, date_time, extra, _row_id(hidden)], stream_key: [bidder, auction, _row_id], pk_columns: [bidder, auction, _row_id], pk_conflict: NoCheck } └─StreamGroupTopN [append_only] { order: [date_time DESC], limit: 1, offset: 0, with_ties: true, group_key: [bidder, auction] } @@ -1549,7 +1549,7 @@ └─LogicalProject { exprs: [auction, bidder, price, channel, url, date_time, extra, row_number] } └─LogicalOverWindow { window_functions: [row_number() OVER(PARTITION BY auction ORDER BY price DESC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } └─LogicalProject { exprs: [auction, bidder, price, channel, url, date_time, extra, _row_id] } - └─LogicalSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], time_range: (Unbounded, Unbounded) } + └─LogicalSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchOverWindow { window_functions: [row_number() OVER(PARTITION BY auction ORDER BY price DESC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } @@ -1557,7 +1557,7 @@ └─BatchGroupTopN { order: [price DESC], limit: 10, offset: 0, group_key: [auction] } └─BatchExchange { order: [], dist: HashShard(auction) } └─BatchProject { exprs: [auction, bidder, price, channel, url, date_time, extra] } - └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], filter: (None, None) } + └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } stream_plan: |- StreamMaterialize { columns: [auction, bidder, price, channel, url, date_time, extra, _row_id(hidden), rank_number], stream_key: [auction, _row_id], pk_columns: [auction, _row_id], pk_conflict: NoCheck } └─StreamOverWindow { window_functions: [row_number() OVER(PARTITION BY auction ORDER BY price DESC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } @@ -1616,11 +1616,11 @@ BatchExchange { order: [], dist: Single } └─BatchHashJoin { type: Inner, predicate: auction = id, output: [auction, bidder, price, channel, url, date_time, item_name, description, initial_bid, reserve, date_time, expires, seller, category] } ├─BatchExchange { order: [], dist: HashShard(auction) } - │ └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], filter: (None, None) } + │ └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } └─BatchExchange { order: [], dist: HashShard(id) } └─BatchFilter { predicate: (category = 10:Int32) } └─BatchProject { exprs: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category] } - └─BatchSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id], filter: (None, None) } + └─BatchSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id] } stream_plan: |- StreamMaterialize { columns: [auction, bidder, price, channel, url, date_timeb, item_name, description, initial_bid, reserve, date_timea, expires, seller, category, _row_id(hidden), _row_id#1(hidden)], stream_key: [_row_id, _row_id#1, auction], pk_columns: [_row_id, _row_id#1, auction], pk_conflict: NoCheck } └─StreamExchange { dist: HashShard(auction, _row_id, _row_id) } @@ -1693,7 +1693,7 @@ └─BatchProject { exprs: [auction, bidder, price, channel, Case((Lower(channel) = 'apple':Varchar), '0':Varchar, (Lower(channel) = 'google':Varchar), '1':Varchar, (Lower(channel) = 'facebook':Varchar), '2':Varchar, (Lower(channel) = 'baidu':Varchar), '3':Varchar, ArrayAccess(RegexpMatch(url, '(&|^)channel_id=([^&]*)':Varchar), 2:Int32)) as $expr1] } └─BatchFilter { predicate: (IsNotNull(ArrayAccess(RegexpMatch(url, '(&|^)channel_id=([^&]*)':Varchar), 2:Int32)) OR In(Lower(channel), 'apple':Varchar, 'google':Varchar, 'facebook':Varchar, 'baidu':Varchar)) } └─BatchProject { exprs: [auction, bidder, price, channel, url] } - └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], filter: (None, None) } + └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } stream_plan: |- StreamMaterialize { columns: [auction, bidder, price, channel, channel_id, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: NoCheck } └─StreamProject { exprs: [auction, bidder, price, channel, Case((Lower(channel) = 'apple':Varchar), '0':Varchar, (Lower(channel) = 'google':Varchar), '1':Varchar, (Lower(channel) = 'facebook':Varchar), '2':Varchar, (Lower(channel) = 'baidu':Varchar), '3':Varchar, ArrayAccess(RegexpMatch(url, '(&|^)channel_id=([^&]*)':Varchar), 2:Int32)) as $expr1, _row_id] } @@ -1724,7 +1724,7 @@ batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchProject { exprs: [auction, bidder, price, channel, SplitPart(url, '/':Varchar, 4:Int32) as $expr1, SplitPart(url, '/':Varchar, 5:Int32) as $expr2, SplitPart(url, '/':Varchar, 6:Int32) as $expr3] } - └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], filter: (None, None) } + └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } stream_plan: |- StreamMaterialize { columns: [auction, bidder, price, channel, dir1, dir2, dir3, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: NoCheck } └─StreamProject { exprs: [auction, bidder, price, channel, SplitPart(url, '/':Varchar, 4:Int32) as $expr1, SplitPart(url, '/':Varchar, 5:Int32) as $expr2, SplitPart(url, '/':Varchar, 6:Int32) as $expr3, _row_id] } @@ -1765,10 +1765,10 @@ BatchExchange { order: [], dist: Single } └─BatchHashJoin { type: LeftOuter, predicate: id = auction, output: [id, item_name, max(price)] } ├─BatchExchange { order: [], dist: HashShard(id) } - │ └─BatchSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id], filter: (None, None) } + │ └─BatchSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id] } └─BatchHashAgg { group_key: [auction], aggs: [max(price)] } └─BatchExchange { order: [], dist: HashShard(auction) } - └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], filter: (None, None) } + └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } stream_plan: |- StreamMaterialize { columns: [auction_id, auction_item_name, current_highest_bid, _row_id(hidden), auction(hidden)], stream_key: [_row_id, auction_id], pk_columns: [_row_id, auction_id], pk_conflict: NoCheck } └─StreamExchange { dist: HashShard(id, _row_id) } @@ -1853,16 +1853,16 @@ │ └─BatchHashAgg { group_key: [id, item_name], aggs: [count(auction)] } │ └─BatchHashJoin { type: Inner, predicate: id = auction, output: [id, item_name, auction] } │ ├─BatchExchange { order: [], dist: HashShard(id) } - │ │ └─BatchSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id], filter: (None, None) } + │ │ └─BatchSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id] } │ └─BatchExchange { order: [], dist: HashShard(auction) } - │ └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], filter: (None, None) } + │ └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } └─BatchProject { exprs: [(sum0(sum0(count)) / sum0(count(auction))) as $expr1] } └─BatchSimpleAgg { aggs: [sum0(sum0(count)), sum0(count(auction))] } └─BatchExchange { order: [], dist: Single } └─BatchSimpleAgg { aggs: [sum0(count), count(auction)] } └─BatchHashAgg { group_key: [auction], aggs: [count] } └─BatchExchange { order: [], dist: HashShard(auction) } - └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], filter: (None, None) } + └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } stream_plan: |- StreamMaterialize { columns: [auction_id, auction_item_name, bid_count], stream_key: [auction_id, auction_item_name], pk_columns: [auction_id, auction_item_name], pk_conflict: NoCheck } └─StreamDynamicFilter { predicate: (count(auction) >= $expr1), output: [id, item_name, count(auction)] } @@ -1982,12 +1982,12 @@ BatchExchange { order: [], dist: Single } └─BatchHashJoin { type: LeftSemi, predicate: id = auction, output: [id, item_name] } ├─BatchExchange { order: [], dist: HashShard(id) } - │ └─BatchSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id], filter: (None, None) } + │ └─BatchSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id] } └─BatchProject { exprs: [auction] } └─BatchFilter { predicate: (count >= 20:Int32) } └─BatchHashAgg { group_key: [auction], aggs: [count] } └─BatchExchange { order: [], dist: HashShard(auction) } - └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], filter: (None, None) } + └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } stream_plan: |- StreamMaterialize { columns: [auction_id, auction_item_name, _row_id(hidden)], stream_key: [_row_id, auction_id], pk_columns: [_row_id, auction_id], pk_conflict: NoCheck } └─StreamExchange { dist: HashShard(id, _row_id) } @@ -2071,12 +2071,12 @@ BatchExchange { order: [], dist: Single } └─BatchHashJoin { type: LeftAnti, predicate: id = auction, output: [id, item_name] } ├─BatchExchange { order: [], dist: HashShard(id) } - │ └─BatchSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id], filter: (None, None) } + │ └─BatchSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id] } └─BatchProject { exprs: [auction] } └─BatchFilter { predicate: (count < 20:Int32) } └─BatchHashAgg { group_key: [auction], aggs: [count] } └─BatchExchange { order: [], dist: HashShard(auction) } - └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], filter: (None, None) } + └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } stream_plan: |- StreamMaterialize { columns: [auction_id, auction_item_name, _row_id(hidden)], stream_key: [_row_id, auction_id], pk_columns: [_row_id, auction_id], pk_conflict: NoCheck } └─StreamExchange { dist: HashShard(id, _row_id) } @@ -2163,9 +2163,9 @@ └─BatchHashAgg { group_key: [id, item_name], aggs: [count(auction)] } └─BatchHashJoin { type: Inner, predicate: id = auction, output: [id, item_name, auction] } ├─BatchExchange { order: [], dist: HashShard(id) } - │ └─BatchSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id], filter: (None, None) } + │ └─BatchSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id] } └─BatchExchange { order: [], dist: HashShard(auction) } - └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], filter: (None, None) } + └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } stream_plan: |- StreamMaterialize { columns: [auction_id, auction_item_name, bid_count], stream_key: [auction_id, auction_item_name], pk_columns: [bid_count, auction_id, auction_item_name], pk_conflict: NoCheck } └─StreamProject { exprs: [id, item_name, count(auction)] } @@ -2272,9 +2272,9 @@ └─BatchHashAgg { group_key: [id], aggs: [max(price)] } └─BatchHashJoin { type: Inner, predicate: id = auction AND (date_time >= date_time) AND (date_time <= expires), output: [id, price] } ├─BatchExchange { order: [], dist: HashShard(id) } - │ └─BatchSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id], filter: (None, None) } + │ └─BatchSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id] } └─BatchExchange { order: [], dist: HashShard(auction) } - └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], filter: (None, None) } + └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } stream_plan: |- StreamMaterialize { columns: [min_final], stream_key: [], pk_columns: [], pk_conflict: NoCheck } └─StreamProject { exprs: [min(min(max(price)))] } diff --git a/src/frontend/planner_test/tests/testdata/output/nexmark_watermark.yaml b/src/frontend/planner_test/tests/testdata/output/nexmark_watermark.yaml index 12593221419d6..6dd731cffffb0 100644 --- a/src/frontend/planner_test/tests/testdata/output/nexmark_watermark.yaml +++ b/src/frontend/planner_test/tests/testdata/output/nexmark_watermark.yaml @@ -11,7 +11,7 @@ └─BatchProject { exprs: [Field(bid, 0:Int32) as $expr2, Field(bid, 1:Int32) as $expr3, Field(bid, 2:Int32) as $expr4, $expr1] } └─BatchFilter { predicate: (event_type = 2:Int32) } └─BatchProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } - └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } + └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } stream_plan: |- StreamMaterialize { columns: [auction, bidder, price, date_time, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: NoCheck, watermark_columns: [date_time] } └─StreamProject { exprs: [Field(bid, 0:Int32) as $expr2, Field(bid, 1:Int32) as $expr3, Field(bid, 2:Int32) as $expr4, $expr1, _row_id], output_watermarks: [$expr1] } @@ -35,7 +35,7 @@ └─BatchProject { exprs: [Field(bid, 0:Int32) as $expr2, Field(bid, 1:Int32) as $expr3, (0.908:Decimal * Field(bid, 2:Int32)::Decimal) as $expr4, $expr1] } └─BatchFilter { predicate: (event_type = 2:Int32) } └─BatchProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } - └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } + └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } stream_plan: |- StreamMaterialize { columns: [auction, bidder, price, date_time, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: NoCheck, watermark_columns: [date_time] } └─StreamProject { exprs: [Field(bid, 0:Int32) as $expr2, Field(bid, 1:Int32) as $expr3, (0.908:Decimal * Field(bid, 2:Int32)::Decimal) as $expr4, $expr1, _row_id], output_watermarks: [$expr1] } @@ -79,7 +79,7 @@ └─BatchProject { exprs: [Field(bid, 0:Int32) as $expr2, Field(bid, 2:Int32) as $expr3] } └─BatchFilter { predicate: (((((Field(bid, 0:Int32) = 1007:Int32) OR (Field(bid, 0:Int32) = 1020:Int32)) OR (Field(bid, 0:Int32) = 2001:Int32)) OR (Field(bid, 0:Int32) = 2019:Int32)) OR (Field(bid, 0:Int32) = 2087:Int32)) AND (event_type = 2:Int32) } └─BatchProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } - └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } + └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } stream_plan: |- StreamMaterialize { columns: [auction, price, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: NoCheck } └─StreamProject { exprs: [Field(bid, 0:Int32) as $expr2, Field(bid, 2:Int32) as $expr3, _row_id] } @@ -124,12 +124,12 @@ │ └─BatchProject { exprs: [Field(auction, 0:Int32) as $expr1, Field(auction, 7:Int32) as $expr2] } │ └─BatchFilter { predicate: (Field(auction, 8:Int32) = 10:Int32) AND (event_type = 1:Int32) } │ └─BatchProject { exprs: [event_type, auction] } - │ └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } + │ └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } └─BatchExchange { order: [], dist: HashShard($expr3) } └─BatchProject { exprs: [Field(person, 0:Int32) as $expr3, Field(person, 1:Int32) as $expr4, Field(person, 4:Int32) as $expr5, Field(person, 5:Int32) as $expr6] } └─BatchFilter { predicate: (((Field(person, 5:Int32) = 'or':Varchar) OR (Field(person, 5:Int32) = 'id':Varchar)) OR (Field(person, 5:Int32) = 'ca':Varchar)) AND (event_type = 0:Int32) } └─BatchProject { exprs: [event_type, person] } - └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } + └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } stream_plan: |- StreamMaterialize { columns: [name, city, state, id, _row_id(hidden), $expr3(hidden), _row_id#1(hidden)], stream_key: [_row_id, _row_id#1, $expr3], pk_columns: [_row_id, _row_id#1, $expr3], pk_conflict: NoCheck } └─StreamExchange { dist: HashShard(_row_id, $expr3, _row_id) } @@ -226,12 +226,12 @@ │ └─BatchProject { exprs: [Field(auction, 0:Int32) as $expr2, $expr1, Field(auction, 6:Int32) as $expr3, Field(auction, 8:Int32) as $expr4] } │ └─BatchFilter { predicate: (event_type = 1:Int32) } │ └─BatchProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } - │ └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } + │ └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } └─BatchExchange { order: [], dist: HashShard($expr6) } └─BatchProject { exprs: [Field(bid, 0:Int32) as $expr6, Field(bid, 2:Int32) as $expr7, $expr5] } └─BatchFilter { predicate: (event_type = 2:Int32) } └─BatchProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr5, _row_id] } - └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } + └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } stream_plan: |- StreamMaterialize { columns: [category, avg], stream_key: [category], pk_columns: [category], pk_conflict: NoCheck } └─StreamProject { exprs: [$expr4, (sum(max($expr6)) / count(max($expr6))::Decimal) as $expr7] } @@ -359,7 +359,7 @@ │ └─LogicalShare { id: 3 } │ └─LogicalProject { exprs: [Field(bid, 0:Int32) as $expr1, Field(bid, 1:Int32) as $expr2, Field(bid, 2:Int32) as $expr3, Field(bid, 3:Int32) as $expr4, Field(bid, 4:Int32) as $expr5, date_time, Field(bid, 6:Int32) as $expr6] } │ └─LogicalFilter { predicate: (event_type = 2:Int32) } - │ └─LogicalSource { source: nexmark, columns: [event_type, person, auction, bid, date_time, _row_id], time_range: (Unbounded, Unbounded) } + │ └─LogicalSource { source: nexmark, columns: [event_type, person, auction, bid, date_time, _row_id] } └─LogicalProject { exprs: [max(count), window_start] } └─LogicalAgg { group_key: [window_start], aggs: [max(count)] } └─LogicalProject { exprs: [window_start, count] } @@ -371,7 +371,7 @@ └─LogicalShare { id: 3 } └─LogicalProject { exprs: [Field(bid, 0:Int32) as $expr1, Field(bid, 1:Int32) as $expr2, Field(bid, 2:Int32) as $expr3, Field(bid, 3:Int32) as $expr4, Field(bid, 4:Int32) as $expr5, date_time, Field(bid, 6:Int32) as $expr6] } └─LogicalFilter { predicate: (event_type = 2:Int32) } - └─LogicalSource { source: nexmark, columns: [event_type, person, auction, bid, date_time, _row_id], time_range: (Unbounded, Unbounded) } + └─LogicalSource { source: nexmark, columns: [event_type, person, auction, bid, date_time, _row_id] } batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchHashJoin { type: Inner, predicate: window_start = window_start AND (count >= max(count)), output: [$expr2, count] } @@ -382,7 +382,7 @@ │ └─BatchProject { exprs: [Field(bid, 0:Int32) as $expr2, $expr1] } │ └─BatchFilter { predicate: IsNotNull($expr1) AND (event_type = 2:Int32) } │ └─BatchProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } - │ └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } + │ └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } └─BatchHashAgg { group_key: [window_start], aggs: [max(count)] } └─BatchExchange { order: [], dist: HashShard(window_start) } └─BatchHashAgg { group_key: [$expr4, window_start], aggs: [count] } @@ -391,7 +391,7 @@ └─BatchProject { exprs: [Field(bid, 0:Int32) as $expr4, $expr3] } └─BatchFilter { predicate: IsNotNull($expr3) AND (event_type = 2:Int32) } └─BatchProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr3, _row_id] } - └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } + └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } stream_plan: |- StreamMaterialize { columns: [auction, num, window_start(hidden), window_start#1(hidden)], stream_key: [auction, window_start], pk_columns: [auction, window_start], pk_conflict: NoCheck, watermark_columns: [window_start(hidden), window_start#1(hidden)] } └─StreamProject { exprs: [$expr2, count, window_start, window_start], output_watermarks: [window_start, window_start] } @@ -679,7 +679,7 @@ │ └─BatchProject { exprs: [Field(bid, 0:Int32) as $expr2, Field(bid, 1:Int32) as $expr3, Field(bid, 2:Int32) as $expr4, $expr1] } │ └─BatchFilter { predicate: (event_type = 2:Int32) } │ └─BatchProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } - │ └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } + │ └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } └─BatchExchange { order: [], dist: HashShard(max($expr7)) } └─BatchProject { exprs: [max($expr7), $expr6, ($expr6 - '00:00:10':Interval) as $expr8] } └─BatchHashAgg { group_key: [$expr6], aggs: [max($expr7)] } @@ -687,7 +687,7 @@ └─BatchProject { exprs: [(TumbleStart($expr5, '00:00:10':Interval) + '00:00:10':Interval) as $expr6, Field(bid, 2:Int32) as $expr7] } └─BatchFilter { predicate: (event_type = 2:Int32) } └─BatchProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr5, _row_id] } - └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } + └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } stream_plan: |- StreamMaterialize { columns: [auction, price, bidder, date_time, _row_id(hidden), $expr5(hidden)], stream_key: [_row_id, $expr5, price], pk_columns: [_row_id, $expr5, price], pk_conflict: NoCheck, watermark_columns: [date_time, $expr5(hidden)] } └─StreamExchange { dist: HashShard($expr4, _row_id, $expr5) } @@ -831,14 +831,14 @@ │ └─BatchProject { exprs: [event_type, person, auction, bid, $expr1, _row_id, TumbleStart($expr1, '00:00:10':Interval) as $expr2] } │ └─BatchFilter { predicate: (event_type = 0:Int32) } │ └─BatchProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } - │ └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } + │ └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } └─BatchHashAgg { group_key: [$expr8, $expr7, $expr9], aggs: [] } └─BatchExchange { order: [], dist: HashShard($expr8, $expr7, $expr9) } └─BatchProject { exprs: [Field(auction, 7:Int32) as $expr8, $expr7, ($expr7 + '00:00:10':Interval) as $expr9] } └─BatchProject { exprs: [event_type, person, auction, bid, $expr6, _row_id, TumbleStart($expr6, '00:00:10':Interval) as $expr7] } └─BatchFilter { predicate: (event_type = 1:Int32) } └─BatchProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr6, _row_id] } - └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } + └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } stream_plan: |- StreamMaterialize { columns: [id, name, starttime, $expr5(hidden), $expr7(hidden), $expr6(hidden), $expr8(hidden)], stream_key: [id, name, starttime, $expr5], pk_columns: [id, name, starttime, $expr5], pk_conflict: NoCheck, watermark_columns: [starttime, $expr5(hidden), $expr6(hidden), $expr8(hidden)] } └─StreamExchange { dist: HashShard($expr3, $expr4, $expr2, $expr5) } @@ -976,20 +976,20 @@ ├─LogicalShare { id: 3 } │ └─LogicalProject { exprs: [Field(auction, 0:Int32) as $expr1, Field(auction, 1:Int32) as $expr2, Field(auction, 2:Int32) as $expr3, Field(auction, 3:Int32) as $expr4, Field(auction, 4:Int32) as $expr5, date_time, Field(auction, 6:Int32) as $expr6, Field(auction, 7:Int32) as $expr7, Field(auction, 8:Int32) as $expr8, Field(auction, 9:Int32) as $expr9] } │ └─LogicalFilter { predicate: (event_type = 1:Int32) } - │ └─LogicalSource { source: nexmark, columns: [event_type, person, auction, bid, date_time, _row_id], time_range: (Unbounded, Unbounded) } + │ └─LogicalSource { source: nexmark, columns: [event_type, person, auction, bid, date_time, _row_id] } └─LogicalShare { id: 6 } └─LogicalProject { exprs: [Field(bid, 0:Int32) as $expr10, Field(bid, 1:Int32) as $expr11, Field(bid, 2:Int32) as $expr12, Field(bid, 3:Int32) as $expr13, Field(bid, 4:Int32) as $expr14, date_time, Field(bid, 6:Int32) as $expr15] } └─LogicalFilter { predicate: (event_type = 2:Int32) } - └─LogicalSource { source: nexmark, columns: [event_type, person, auction, bid, date_time, _row_id], time_range: (Unbounded, Unbounded) } + └─LogicalSource { source: nexmark, columns: [event_type, person, auction, bid, date_time, _row_id] } optimized_logical_plan_for_batch: |- LogicalTopN { order: [$expr11 DESC, date_time ASC], limit: 1, offset: 0, group_key: [$expr1] } └─LogicalJoin { type: Inner, on: ($expr1 = $expr9) AND (date_time >= date_time) AND (date_time <= $expr6), output: all } ├─LogicalProject { exprs: [Field(auction, 0:Int32) as $expr1, Field(auction, 1:Int32) as $expr2, Field(auction, 2:Int32) as $expr3, Field(auction, 3:Int32) as $expr4, Field(auction, 4:Int32) as $expr5, date_time, Field(auction, 6:Int32) as $expr6, Field(auction, 7:Int32) as $expr7, Field(auction, 8:Int32) as $expr8] } │ └─LogicalFilter { predicate: (event_type = 1:Int32) } - │ └─LogicalSource { source: nexmark, columns: [event_type, person, auction, bid, date_time, _row_id], time_range: (Unbounded, Unbounded) } + │ └─LogicalSource { source: nexmark, columns: [event_type, person, auction, bid, date_time, _row_id] } └─LogicalProject { exprs: [Field(bid, 0:Int32) as $expr9, Field(bid, 1:Int32) as $expr10, Field(bid, 2:Int32) as $expr11, date_time] } └─LogicalFilter { predicate: (event_type = 2:Int32) } - └─LogicalSource { source: nexmark, columns: [event_type, person, auction, bid, date_time, _row_id], time_range: (Unbounded, Unbounded) } + └─LogicalSource { source: nexmark, columns: [event_type, person, auction, bid, date_time, _row_id] } batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchGroupTopN { order: [$expr13 DESC, $expr10 ASC], limit: 1, offset: 0, group_key: [$expr2] } @@ -998,12 +998,12 @@ │ └─BatchProject { exprs: [Field(auction, 0:Int32) as $expr2, Field(auction, 1:Int32) as $expr3, Field(auction, 2:Int32) as $expr4, Field(auction, 3:Int32) as $expr5, Field(auction, 4:Int32) as $expr6, $expr1, Field(auction, 6:Int32) as $expr7, Field(auction, 7:Int32) as $expr8, Field(auction, 8:Int32) as $expr9] } │ └─BatchFilter { predicate: (event_type = 1:Int32) } │ └─BatchProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } - │ └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } + │ └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } └─BatchExchange { order: [], dist: HashShard($expr11) } └─BatchProject { exprs: [Field(bid, 0:Int32) as $expr11, Field(bid, 1:Int32) as $expr12, Field(bid, 2:Int32) as $expr13, $expr10] } └─BatchFilter { predicate: (event_type = 2:Int32) } └─BatchProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr10, _row_id] } - └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } + └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } stream_plan: |- StreamMaterialize { columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, auction, bidder, price, bid_date_time, _row_id(hidden), _row_id#1(hidden)], stream_key: [id], pk_columns: [id], pk_conflict: NoCheck, watermark_columns: [date_time, bid_date_time] } └─StreamGroupTopN [append_only] { order: [$expr12 DESC, $expr1 ASC], limit: 1, offset: 0, group_key: [$expr2], output_watermarks: [$expr1, $expr1] } @@ -1106,7 +1106,7 @@ └─BatchProject { exprs: [Field(bid, 0:Int32) as $expr2, Field(bid, 1:Int32) as $expr3, Field(bid, 2:Int32) as $expr4, $expr1, ToChar($expr1, 'YYYY-MM-DD':Varchar) as $expr5, ToChar($expr1, 'HH:MI':Varchar) as $expr6] } └─BatchFilter { predicate: (event_type = 2:Int32) } └─BatchProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } - └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } + └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } stream_plan: |- StreamMaterialize { columns: [auction, bidder, price, date_time, date, time, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: NoCheck, watermark_columns: [date_time] } └─StreamProject { exprs: [Field(bid, 0:Int32) as $expr2, Field(bid, 1:Int32) as $expr3, Field(bid, 2:Int32) as $expr4, $expr1, ToChar($expr1, 'YYYY-MM-DD':Varchar) as $expr5, ToChar($expr1, 'HH:MI':Varchar) as $expr6, _row_id], output_watermarks: [$expr1] } @@ -1217,7 +1217,7 @@ └─BatchProject { exprs: [Field(bid, 0:Int32) as $expr2, Field(bid, 1:Int32) as $expr3, (0.908:Decimal * Field(bid, 2:Int32)::Decimal) as $expr4, Case(((Extract('HOUR':Varchar, $expr1) >= 8:Decimal) AND (Extract('HOUR':Varchar, $expr1) <= 18:Decimal)), 'dayTime':Varchar, ((Extract('HOUR':Varchar, $expr1) <= 6:Decimal) OR (Extract('HOUR':Varchar, $expr1) >= 20:Decimal)), 'nightTime':Varchar, 'otherTime':Varchar) as $expr5, $expr1, Field(bid, 6:Int32) as $expr6] } └─BatchFilter { predicate: ((0.908:Decimal * Field(bid, 2:Int32)::Decimal) > 1000000:Decimal) AND ((0.908:Decimal * Field(bid, 2:Int32)::Decimal) < 50000000:Decimal) AND (event_type = 2:Int32) } └─BatchProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } - └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } + └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } stream_plan: |- StreamMaterialize { columns: [auction, bidder, price, bidtimetype, date_time, extra, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: NoCheck, watermark_columns: [date_time] } └─StreamProject { exprs: [Field(bid, 0:Int32) as $expr2, Field(bid, 1:Int32) as $expr3, (0.908:Decimal * Field(bid, 2:Int32)::Decimal) as $expr4, Case(((Extract('HOUR':Varchar, $expr1) >= 8:Decimal) AND (Extract('HOUR':Varchar, $expr1) <= 18:Decimal)), 'dayTime':Varchar, ((Extract('HOUR':Varchar, $expr1) <= 6:Decimal) OR (Extract('HOUR':Varchar, $expr1) >= 20:Decimal)), 'nightTime':Varchar, 'otherTime':Varchar) as $expr5, $expr1, Field(bid, 6:Int32) as $expr6, _row_id], output_watermarks: [$expr1] } @@ -1281,7 +1281,7 @@ └─BatchProject { exprs: [ToChar($expr1, 'yyyy-MM-dd':Varchar) as $expr2, Field(bid, 2:Int32) as $expr3, Field(bid, 1:Int32) as $expr4, Field(bid, 0:Int32) as $expr5] } └─BatchFilter { predicate: (event_type = 2:Int32) } └─BatchProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } - └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } + └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } stream_plan: |- StreamMaterialize { columns: [day, total_bids, rank1_bids, rank2_bids, rank3_bids, total_bidders, rank1_bidders, rank2_bidders, rank3_bidders, total_auctions, rank1_auctions, rank2_auctions, rank3_auctions], stream_key: [day], pk_columns: [day], pk_conflict: NoCheck } └─StreamHashAgg [append_only] { group_key: [$expr2], aggs: [count, count filter(($expr3 < 10000:Int32)), count filter(($expr3 >= 10000:Int32) AND ($expr3 < 1000000:Int32)), count filter(($expr3 >= 1000000:Int32)), count(distinct $expr4), count(distinct $expr4) filter(($expr3 < 10000:Int32)), count(distinct $expr4) filter(($expr3 >= 10000:Int32) AND ($expr3 < 1000000:Int32)), count(distinct $expr4) filter(($expr3 >= 1000000:Int32)), count(distinct $expr5), count(distinct $expr5) filter(($expr3 < 10000:Int32)), count(distinct $expr5) filter(($expr3 >= 10000:Int32) AND ($expr3 < 1000000:Int32)), count(distinct $expr5) filter(($expr3 >= 1000000:Int32))] } @@ -1359,7 +1359,7 @@ └─BatchProject { exprs: [Field(bid, 3:Int32) as $expr2, ToChar($expr1, 'yyyy-MM-dd':Varchar) as $expr3, ToChar($expr1, 'HH:mm':Varchar) as $expr4, Field(bid, 2:Int32) as $expr5, Field(bid, 1:Int32) as $expr6, Field(bid, 0:Int32) as $expr7] } └─BatchFilter { predicate: (event_type = 2:Int32) } └─BatchProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } - └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } + └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } stream_plan: |- StreamMaterialize { columns: [channel, day, minute, total_bids, rank1_bids, rank2_bids, rank3_bids, total_bidders, rank1_bidders, rank2_bidders, rank3_bidders, total_auctions, rank1_auctions, rank2_auctions, rank3_auctions], stream_key: [channel, day], pk_columns: [channel, day], pk_conflict: NoCheck } └─StreamHashAgg [append_only] { group_key: [$expr2, $expr3], aggs: [max($expr4), count, count filter(($expr5 < 10000:Int32)), count filter(($expr5 >= 10000:Int32) AND ($expr5 < 1000000:Int32)), count filter(($expr5 >= 1000000:Int32)), count(distinct $expr6), count(distinct $expr6) filter(($expr5 < 10000:Int32)), count(distinct $expr6) filter(($expr5 >= 10000:Int32) AND ($expr5 < 1000000:Int32)), count(distinct $expr6) filter(($expr5 >= 1000000:Int32)), count(distinct $expr7), count(distinct $expr7) filter(($expr5 < 10000:Int32)), count(distinct $expr7) filter(($expr5 >= 10000:Int32) AND ($expr5 < 1000000:Int32)), count(distinct $expr7) filter(($expr5 >= 1000000:Int32))] } @@ -1430,7 +1430,7 @@ └─BatchProject { exprs: [Field(bid, 0:Int32) as $expr2, ToChar($expr1, 'YYYY-MM-DD':Varchar) as $expr3, Field(bid, 2:Int32) as $expr4] } └─BatchFilter { predicate: (event_type = 2:Int32) } └─BatchProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } - └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } + └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } stream_plan: |- StreamMaterialize { columns: [auction, day, total_bids, rank1_bids, rank2_bids, rank3_bids, min_price, max_price, avg_price, sum_price], stream_key: [auction, day], pk_columns: [auction, day], pk_conflict: NoCheck } └─StreamProject { exprs: [$expr2, $expr3, count, count filter(($expr4 < 10000:Int32)), count filter(($expr4 >= 10000:Int32) AND ($expr4 < 1000000:Int32)), count filter(($expr4 >= 1000000:Int32)), min($expr4), max($expr4), (sum($expr4) / count($expr4)::Decimal) as $expr5, sum($expr4)] } @@ -1491,7 +1491,7 @@ └─LogicalShare { id: 3 } └─LogicalProject { exprs: [Field(bid, 0:Int32) as $expr1, Field(bid, 1:Int32) as $expr2, Field(bid, 2:Int32) as $expr3, Field(bid, 3:Int32) as $expr4, Field(bid, 4:Int32) as $expr5, date_time, Field(bid, 6:Int32) as $expr6] } └─LogicalFilter { predicate: (event_type = 2:Int32) } - └─LogicalSource { source: nexmark, columns: [event_type, person, auction, bid, date_time, _row_id], time_range: (Unbounded, Unbounded) } + └─LogicalSource { source: nexmark, columns: [event_type, person, auction, bid, date_time, _row_id] } batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchGroupTopN { order: [$expr1 DESC], limit: 1, offset: 0, group_key: [$expr3, $expr2] } @@ -1499,7 +1499,7 @@ └─BatchProject { exprs: [Field(bid, 0:Int32) as $expr2, Field(bid, 1:Int32) as $expr3, Field(bid, 2:Int32) as $expr4, Field(bid, 3:Int32) as $expr5, Field(bid, 4:Int32) as $expr6, $expr1, Field(bid, 6:Int32) as $expr7] } └─BatchFilter { predicate: (event_type = 2:Int32) } └─BatchProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } - └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } + └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } stream_plan: |- StreamMaterialize { columns: [auction, bidder, price, channel, url, date_time, extra, _row_id(hidden)], stream_key: [bidder, auction], pk_columns: [bidder, auction], pk_conflict: NoCheck, watermark_columns: [date_time] } └─StreamGroupTopN [append_only] { order: [$expr1 DESC], limit: 1, offset: 0, group_key: [$expr3, $expr2], output_watermarks: [$expr1] } @@ -1561,7 +1561,7 @@ └─LogicalShare { id: 3 } └─LogicalProject { exprs: [Field(bid, 0:Int32) as $expr1, Field(bid, 1:Int32) as $expr2, Field(bid, 2:Int32) as $expr3, Field(bid, 3:Int32) as $expr4, Field(bid, 4:Int32) as $expr5, date_time, Field(bid, 6:Int32) as $expr6] } └─LogicalFilter { predicate: (event_type = 2:Int32) } - └─LogicalSource { source: nexmark, columns: [event_type, person, auction, bid, date_time, _row_id], time_range: (Unbounded, Unbounded) } + └─LogicalSource { source: nexmark, columns: [event_type, person, auction, bid, date_time, _row_id] } batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchGroupTopN { order: [$expr1 DESC], limit: 1, offset: 0, with_ties: true, group_key: [$expr3, $expr2] } @@ -1569,7 +1569,7 @@ └─BatchProject { exprs: [Field(bid, 0:Int32) as $expr2, Field(bid, 1:Int32) as $expr3, Field(bid, 2:Int32) as $expr4, Field(bid, 3:Int32) as $expr5, Field(bid, 4:Int32) as $expr6, $expr1, Field(bid, 6:Int32) as $expr7] } └─BatchFilter { predicate: (event_type = 2:Int32) } └─BatchProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } - └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } + └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } stream_plan: |- StreamMaterialize { columns: [auction, bidder, price, channel, url, date_time, extra, _row_id(hidden)], stream_key: [bidder, auction, _row_id], pk_columns: [bidder, auction, _row_id], pk_conflict: NoCheck, watermark_columns: [date_time] } └─StreamGroupTopN [append_only] { order: [$expr1 DESC], limit: 1, offset: 0, with_ties: true, group_key: [$expr3, $expr2], output_watermarks: [$expr1] } @@ -1630,7 +1630,7 @@ └─LogicalShare { id: 3 } └─LogicalProject { exprs: [Field(bid, 0:Int32) as $expr1, Field(bid, 1:Int32) as $expr2, Field(bid, 2:Int32) as $expr3, Field(bid, 3:Int32) as $expr4, Field(bid, 4:Int32) as $expr5, date_time, Field(bid, 6:Int32) as $expr6] } └─LogicalFilter { predicate: (event_type = 2:Int32) } - └─LogicalSource { source: nexmark, columns: [event_type, person, auction, bid, date_time, _row_id], time_range: (Unbounded, Unbounded) } + └─LogicalSource { source: nexmark, columns: [event_type, person, auction, bid, date_time, _row_id] } batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchOverWindow { window_functions: [row_number() OVER(PARTITION BY $expr2 ORDER BY $expr4 DESC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } @@ -1640,7 +1640,7 @@ └─BatchProject { exprs: [Field(bid, 0:Int32) as $expr2, Field(bid, 1:Int32) as $expr3, Field(bid, 2:Int32) as $expr4, Field(bid, 3:Int32) as $expr5, Field(bid, 4:Int32) as $expr6, $expr1, Field(bid, 6:Int32) as $expr7] } └─BatchFilter { predicate: (event_type = 2:Int32) } └─BatchProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } - └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } + └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } stream_plan: |- StreamMaterialize { columns: [auction, bidder, price, channel, url, date_time, extra, _row_id(hidden), rank_number], stream_key: [auction, _row_id], pk_columns: [auction, _row_id], pk_conflict: NoCheck } └─StreamOverWindow { window_functions: [row_number() OVER(PARTITION BY $expr2 ORDER BY $expr4 DESC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } @@ -1696,12 +1696,12 @@ │ └─BatchProject { exprs: [Field(bid, 0:Int32) as $expr2, Field(bid, 1:Int32) as $expr3, Field(bid, 2:Int32) as $expr4, Field(bid, 3:Int32) as $expr5, Field(bid, 4:Int32) as $expr6, $expr1] } │ └─BatchFilter { predicate: (event_type = 2:Int32) } │ └─BatchProject { exprs: [event_type, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1] } - │ └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } + │ └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } └─BatchExchange { order: [], dist: HashShard($expr8) } └─BatchProject { exprs: [Field(auction, 0:Int32) as $expr8, Field(auction, 1:Int32) as $expr9, Field(auction, 2:Int32) as $expr10, Field(auction, 3:Int32) as $expr11, Field(auction, 4:Int32) as $expr12, $expr7, Field(auction, 6:Int32) as $expr13, Field(auction, 7:Int32) as $expr14, Field(auction, 8:Int32) as $expr15] } └─BatchFilter { predicate: (Field(auction, 8:Int32) = 10:Int32) AND (event_type = 1:Int32) } └─BatchProject { exprs: [event_type, auction, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr7] } - └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } + └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } stream_plan: |- StreamMaterialize { columns: [auction, bidder, price, channel, url, date_timeb, item_name, description, initial_bid, reserve, date_timea, expires, seller, category, _row_id(hidden), _row_id#1(hidden)], stream_key: [_row_id, _row_id#1, auction], pk_columns: [_row_id, _row_id#1, auction], pk_conflict: NoCheck } └─StreamExchange { dist: HashShard($expr2, _row_id, _row_id) } @@ -1800,7 +1800,7 @@ └─BatchProject { exprs: [Field(bid, 0:Int32) as $expr1, Field(bid, 1:Int32) as $expr2, Field(bid, 2:Int32) as $expr3, Field(bid, 3:Int32) as $expr4, Field(bid, 4:Int32) as $expr5] } └─BatchFilter { predicate: (IsNotNull(ArrayAccess(RegexpMatch(Field(bid, 4:Int32), '(&|^)channel_id=([^&]*)':Varchar), 2:Int32)) OR In(Lower(Field(bid, 3:Int32)), 'apple':Varchar, 'google':Varchar, 'facebook':Varchar, 'baidu':Varchar)) AND (event_type = 2:Int32) } └─BatchProject { exprs: [event_type, bid] } - └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } + └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } stream_plan: |- StreamMaterialize { columns: [auction, bidder, price, channel, channel_id, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: NoCheck } └─StreamProject { exprs: [$expr2, $expr3, $expr4, $expr5, Case((Lower($expr5) = 'apple':Varchar), '0':Varchar, (Lower($expr5) = 'google':Varchar), '1':Varchar, (Lower($expr5) = 'facebook':Varchar), '2':Varchar, (Lower($expr5) = 'baidu':Varchar), '3':Varchar, ArrayAccess(RegexpMatch($expr6, '(&|^)channel_id=([^&]*)':Varchar), 2:Int32)) as $expr7, _row_id] } @@ -1842,7 +1842,7 @@ └─BatchProject { exprs: [Field(bid, 0:Int32) as $expr1, Field(bid, 1:Int32) as $expr2, Field(bid, 2:Int32) as $expr3, Field(bid, 3:Int32) as $expr4, Field(bid, 4:Int32) as $expr5] } └─BatchFilter { predicate: (event_type = 2:Int32) } └─BatchProject { exprs: [event_type, bid] } - └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } + └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } stream_plan: |- StreamMaterialize { columns: [auction, bidder, price, channel, dir1, dir2, dir3, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: NoCheck } └─StreamProject { exprs: [$expr2, $expr3, $expr4, $expr5, SplitPart($expr6, '/':Varchar, 4:Int32) as $expr7, SplitPart($expr6, '/':Varchar, 5:Int32) as $expr8, SplitPart($expr6, '/':Varchar, 6:Int32) as $expr9, _row_id] } @@ -1899,13 +1899,13 @@ │ └─BatchProject { exprs: [Field(auction, 0:Int32) as $expr2, Field(auction, 1:Int32) as $expr3] } │ └─BatchFilter { predicate: (event_type = 1:Int32) } │ └─BatchProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } - │ └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } + │ └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } └─BatchHashAgg { group_key: [$expr5], aggs: [max($expr6)] } └─BatchExchange { order: [], dist: HashShard($expr5) } └─BatchProject { exprs: [Field(bid, 0:Int32) as $expr5, Field(bid, 2:Int32) as $expr6] } └─BatchFilter { predicate: (event_type = 2:Int32) } └─BatchProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr4, _row_id] } - └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } + └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } stream_plan: |- StreamMaterialize { columns: [auction_id, auction_item_name, current_highest_bid, _row_id(hidden), $expr4(hidden)], stream_key: [_row_id, auction_id], pk_columns: [_row_id, auction_id], pk_conflict: NoCheck } └─StreamExchange { dist: HashShard($expr2, _row_id) } @@ -2015,12 +2015,12 @@ │ │ └─BatchProject { exprs: [Field(auction, 0:Int32) as $expr2, Field(auction, 1:Int32) as $expr3] } │ │ └─BatchFilter { predicate: (event_type = 1:Int32) } │ │ └─BatchProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } - │ │ └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } + │ │ └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } │ └─BatchExchange { order: [], dist: HashShard($expr5) } │ └─BatchProject { exprs: [Field(bid, 0:Int32) as $expr5] } │ └─BatchFilter { predicate: (event_type = 2:Int32) } │ └─BatchProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr4, _row_id] } - │ └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } + │ └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } └─BatchProject { exprs: [(sum0(sum0(count)) / sum0(count($expr7))) as $expr8] } └─BatchSimpleAgg { aggs: [sum0(sum0(count)), sum0(count($expr7))] } └─BatchExchange { order: [], dist: Single } @@ -2030,7 +2030,7 @@ └─BatchProject { exprs: [Field(bid, 0:Int32) as $expr7] } └─BatchFilter { predicate: (event_type = 2:Int32) } └─BatchProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr6, _row_id] } - └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } + └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } stream_plan: |- StreamMaterialize { columns: [auction_id, auction_item_name, bid_count], stream_key: [auction_id, auction_item_name], pk_columns: [auction_id, auction_item_name], pk_conflict: NoCheck } └─StreamDynamicFilter { predicate: (count($expr4) >= $expr5), output: [$expr2, $expr3, count($expr4)] } @@ -2174,7 +2174,7 @@ │ └─BatchProject { exprs: [Field(auction, 0:Int32) as $expr2, Field(auction, 1:Int32) as $expr3] } │ └─BatchFilter { predicate: (event_type = 1:Int32) } │ └─BatchProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } - │ └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } + │ └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } └─BatchProject { exprs: [$expr5] } └─BatchFilter { predicate: (count >= 20:Int32) } └─BatchHashAgg { group_key: [$expr5], aggs: [count] } @@ -2182,7 +2182,7 @@ └─BatchProject { exprs: [Field(bid, 0:Int32) as $expr5] } └─BatchFilter { predicate: (event_type = 2:Int32) } └─BatchProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr4, _row_id] } - └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } + └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } stream_plan: |- StreamMaterialize { columns: [auction_id, auction_item_name, _row_id(hidden)], stream_key: [_row_id, auction_id], pk_columns: [_row_id, auction_id], pk_conflict: NoCheck } └─StreamExchange { dist: HashShard($expr2, _row_id) } @@ -2285,7 +2285,7 @@ │ └─BatchProject { exprs: [Field(auction, 0:Int32) as $expr2, Field(auction, 1:Int32) as $expr3] } │ └─BatchFilter { predicate: (event_type = 1:Int32) } │ └─BatchProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } - │ └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } + │ └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } └─BatchProject { exprs: [$expr5] } └─BatchFilter { predicate: (count < 20:Int32) } └─BatchHashAgg { group_key: [$expr5], aggs: [count] } @@ -2293,7 +2293,7 @@ └─BatchProject { exprs: [Field(bid, 0:Int32) as $expr5] } └─BatchFilter { predicate: (event_type = 2:Int32) } └─BatchProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr4, _row_id] } - └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } + └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } stream_plan: |- StreamMaterialize { columns: [auction_id, auction_item_name, _row_id(hidden)], stream_key: [_row_id, auction_id], pk_columns: [_row_id, auction_id], pk_conflict: NoCheck } └─StreamExchange { dist: HashShard($expr2, _row_id) } @@ -2399,12 +2399,12 @@ │ └─BatchProject { exprs: [Field(auction, 0:Int32) as $expr2, Field(auction, 1:Int32) as $expr3] } │ └─BatchFilter { predicate: (event_type = 1:Int32) } │ └─BatchProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } - │ └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } + │ └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } └─BatchExchange { order: [], dist: HashShard($expr5) } └─BatchProject { exprs: [Field(bid, 0:Int32) as $expr5] } └─BatchFilter { predicate: (event_type = 2:Int32) } └─BatchProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr4, _row_id] } - └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } + └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } stream_plan: |- StreamMaterialize { columns: [auction_id, auction_item_name, bid_count], stream_key: [auction_id, auction_item_name], pk_columns: [bid_count, auction_id, auction_item_name], pk_conflict: NoCheck } └─StreamProject { exprs: [$expr2, $expr3, count($expr4)] } @@ -2525,12 +2525,12 @@ │ └─BatchProject { exprs: [Field(auction, 0:Int32) as $expr2, $expr1, Field(auction, 6:Int32) as $expr3] } │ └─BatchFilter { predicate: (event_type = 1:Int32) } │ └─BatchProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } - │ └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } + │ └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } └─BatchExchange { order: [], dist: HashShard($expr5) } └─BatchProject { exprs: [Field(bid, 0:Int32) as $expr5, Field(bid, 2:Int32) as $expr6, $expr4] } └─BatchFilter { predicate: (event_type = 2:Int32) } └─BatchProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr4, _row_id] } - └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } + └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } stream_plan: |- StreamMaterialize { columns: [min_final], stream_key: [], pk_columns: [], pk_conflict: NoCheck } └─StreamProject { exprs: [min(min(max($expr5)))] } diff --git a/src/frontend/planner_test/tests/testdata/output/share.yaml b/src/frontend/planner_test/tests/testdata/output/share.yaml index 5df21816d1fec..cfe5d841cb2c1 100644 --- a/src/frontend/planner_test/tests/testdata/output/share.yaml +++ b/src/frontend/planner_test/tests/testdata/output/share.yaml @@ -27,10 +27,10 @@ └─BatchHashJoin { type: Inner, predicate: id = id, output: [] } ├─BatchExchange { order: [], dist: HashShard(id) } │ └─BatchFilter { predicate: (initial_bid = 1:Int32) } - │ └─BatchSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id], filter: (None, None) } + │ └─BatchSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id] } └─BatchExchange { order: [], dist: HashShard(id) } └─BatchFilter { predicate: (initial_bid = 2:Int32) } - └─BatchSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id], filter: (None, None) } + └─BatchSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id] } stream_plan: |- StreamMaterialize { columns: [cnt], stream_key: [], pk_columns: [], pk_conflict: NoCheck } └─StreamProject { exprs: [sum0(count)] } @@ -92,7 +92,7 @@ │ └─LogicalProject { exprs: [window_start, auction] } │ └─LogicalHopWindow { time_col: date_time, slide: 00:00:02, size: 00:00:10, output: all } │ └─LogicalFilter { predicate: IsNotNull(date_time) } - │ └─LogicalSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], time_range: (Unbounded, Unbounded) } + │ └─LogicalSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } └─LogicalProject { exprs: [max(count), window_start] } └─LogicalAgg { group_key: [window_start], aggs: [max(count)] } └─LogicalProject { exprs: [window_start, count] } @@ -101,7 +101,7 @@ └─LogicalProject { exprs: [auction, window_start] } └─LogicalHopWindow { time_col: date_time, slide: 00:00:02, size: 00:00:10, output: all } └─LogicalFilter { predicate: IsNotNull(date_time) } - └─LogicalSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], time_range: (Unbounded, Unbounded) } + └─LogicalSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchHashJoin { type: Inner, predicate: window_start = window_start AND (count >= max(count)), output: [auction, count] } @@ -111,7 +111,7 @@ │ └─BatchExchange { order: [], dist: HashShard(auction) } │ └─BatchProject { exprs: [auction, date_time] } │ └─BatchFilter { predicate: IsNotNull(date_time) } - │ └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], filter: (None, None) } + │ └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } └─BatchHashAgg { group_key: [window_start], aggs: [max(count)] } └─BatchExchange { order: [], dist: HashShard(window_start) } └─BatchHashAgg { group_key: [auction, window_start], aggs: [count] } @@ -119,7 +119,7 @@ └─BatchExchange { order: [], dist: HashShard(auction) } └─BatchProject { exprs: [auction, date_time] } └─BatchFilter { predicate: IsNotNull(date_time) } - └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], filter: (None, None) } + └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } stream_plan: |- StreamMaterialize { columns: [auction, num, window_start(hidden), window_start#1(hidden)], stream_key: [auction, window_start], pk_columns: [auction, window_start], pk_conflict: NoCheck } └─StreamProject { exprs: [auction, count, window_start, window_start] } @@ -298,11 +298,11 @@ ├─BatchExchange { order: [], dist: HashShard(id) } │ └─BatchFilter { predicate: (initial_bid = 1:Int32) AND (AtTimeZone(date_time, 'UTC':Varchar) > '2021-03-31 23:59:59+00:00':Timestamptz) } │ └─BatchProject { exprs: [id, initial_bid, date_time] } - │ └─BatchSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id], filter: (None, None) } + │ └─BatchSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id] } └─BatchExchange { order: [], dist: HashShard(id) } └─BatchFilter { predicate: (initial_bid = 2:Int32) } └─BatchProject { exprs: [id, initial_bid, date_time] } - └─BatchSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id], filter: (None, None) } + └─BatchSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id] } stream_plan: |- StreamMaterialize { columns: [a_id, b_id, a_ts, b_ts, _row_id(hidden), _row_id#1(hidden)], stream_key: [_row_id, _row_id#1, a_id], pk_columns: [_row_id, _row_id#1, a_id], pk_conflict: NoCheck } └─StreamExchange { dist: HashShard(id, _row_id, _row_id) } diff --git a/src/frontend/planner_test/tests/testdata/output/shared_source.yml b/src/frontend/planner_test/tests/testdata/output/shared_source.yml index 54f50fc2c687d..5bf3739f28411 100644 --- a/src/frontend/planner_test/tests/testdata/output/shared_source.yml +++ b/src/frontend/planner_test/tests/testdata/output/shared_source.yml @@ -36,7 +36,7 @@ batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchProject { exprs: [x, y] } - └─BatchSource { source: s, columns: [x, y, _rw_kafka_timestamp, _row_id], filter: (None, None) } + └─BatchKafkaScan { source: s, columns: [x, y, _rw_kafka_timestamp, _row_id], filter: (None, None) } stream_plan: |- StreamMaterialize { columns: [x, y, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: NoCheck } └─StreamProject { exprs: [x, y, _row_id] } @@ -52,7 +52,7 @@ batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchProject { exprs: [x, y] } - └─BatchSource { source: s, columns: [x, y, _rw_kafka_timestamp, _row_id], filter: (None, None) } + └─BatchKafkaScan { source: s, columns: [x, y, _rw_kafka_timestamp, _row_id], filter: (None, None) } stream_plan: |- StreamMaterialize { columns: [x, y, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: NoCheck } └─StreamProject { exprs: [x, y, _row_id] } @@ -68,7 +68,7 @@ batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchProject { exprs: [x, y] } - └─BatchSource { source: s, columns: [x, y, _rw_kafka_timestamp, _row_id], filter: (None, None) } + └─BatchKafkaScan { source: s, columns: [x, y, _rw_kafka_timestamp, _row_id], filter: (None, None) } stream_plan: |- StreamMaterialize { columns: [x, y, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: NoCheck } └─StreamProject { exprs: [x, y, _row_id] } @@ -84,7 +84,7 @@ batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchProject { exprs: [x, y] } - └─BatchSource { source: s, columns: [x, y, _rw_kafka_timestamp, _row_id], filter: (None, None) } + └─BatchKafkaScan { source: s, columns: [x, y, _rw_kafka_timestamp, _row_id], filter: (None, None) } stream_plan: |- StreamMaterialize { columns: [x, y, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: NoCheck } └─StreamProject { exprs: [x, y, _row_id] } diff --git a/src/frontend/planner_test/tests/testdata/output/subquery.yaml b/src/frontend/planner_test/tests/testdata/output/subquery.yaml index 451ce0b70a7b8..e17f2f3cf699a 100644 --- a/src/frontend/planner_test/tests/testdata/output/subquery.yaml +++ b/src/frontend/planner_test/tests/testdata/output/subquery.yaml @@ -486,21 +486,21 @@ └─LogicalJoin { type: LeftSemi, on: IsNotDistinctFrom(a1, a1), output: [] } ├─LogicalShare { id: 2 } │ └─LogicalProject { exprs: [a1] } - │ └─LogicalSource { source: a, columns: [a1, a2, _row_id], time_range: (Unbounded, Unbounded) } + │ └─LogicalSource { source: a, columns: [a1, a2, _row_id] } └─LogicalJoin { type: LeftSemi, on: IsNotDistinctFrom(a1, a1), output: all } ├─LogicalJoin { type: Inner, on: true, output: all } │ ├─LogicalAgg { group_key: [a1], aggs: [] } │ │ └─LogicalShare { id: 2 } │ │ └─LogicalProject { exprs: [a1] } - │ │ └─LogicalSource { source: a, columns: [a1, a2, _row_id], time_range: (Unbounded, Unbounded) } + │ │ └─LogicalSource { source: a, columns: [a1, a2, _row_id] } │ └─LogicalAgg { aggs: [] } - │ └─LogicalSource { source: b, columns: [b1, b2, _row_id], time_range: (Unbounded, Unbounded) } + │ └─LogicalSource { source: b, columns: [b1, b2, _row_id] } └─LogicalJoin { type: Inner, on: true, output: [a1] } ├─LogicalAgg { group_key: [a1], aggs: [] } │ └─LogicalShare { id: 2 } │ └─LogicalProject { exprs: [a1] } - │ └─LogicalSource { source: a, columns: [a1, a2, _row_id], time_range: (Unbounded, Unbounded) } - └─LogicalSource { source: c, columns: [c1, c2, _row_id], time_range: (Unbounded, Unbounded) } + │ └─LogicalSource { source: a, columns: [a1, a2, _row_id] } + └─LogicalSource { source: c, columns: [c1, c2, _row_id] } - name: test subquery in table function sql: | create table t(x int[], y int[], k int primary key); diff --git a/src/frontend/planner_test/tests/testdata/output/tpch_variant.yaml b/src/frontend/planner_test/tests/testdata/output/tpch_variant.yaml index 89e884f962235..7b192ccceed49 100644 --- a/src/frontend/planner_test/tests/testdata/output/tpch_variant.yaml +++ b/src/frontend/planner_test/tests/testdata/output/tpch_variant.yaml @@ -136,11 +136,11 @@ │ ├─LogicalJoin { type: Inner, on: true, output: all } │ │ ├─LogicalJoin { type: Inner, on: true, output: all } │ │ │ ├─LogicalJoin { type: Inner, on: true, output: all } - │ │ │ │ ├─LogicalSource { source: part, columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ │ │ │ └─LogicalSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ │ │ └─LogicalSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ │ └─LogicalSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ └─LogicalSource { source: region, columns: [r_regionkey, r_name, r_comment, _row_id], time_range: (Unbounded, Unbounded) } + │ │ │ │ ├─LogicalSource { source: part, columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _row_id] } + │ │ │ │ └─LogicalSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } + │ │ │ └─LogicalSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id] } + │ │ └─LogicalSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] } + │ └─LogicalSource { source: region, columns: [r_regionkey, r_name, r_comment, _row_id] } └─LogicalProject { exprs: [min(ps_supplycost)] } └─LogicalAgg { aggs: [min(ps_supplycost)] } └─LogicalProject { exprs: [ps_supplycost] } @@ -148,29 +148,29 @@ └─LogicalJoin { type: Inner, on: true, output: all } ├─LogicalJoin { type: Inner, on: true, output: all } │ ├─LogicalJoin { type: Inner, on: true, output: all } - │ │ ├─LogicalSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ │ └─LogicalSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ └─LogicalSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id], time_range: (Unbounded, Unbounded) } - └─LogicalSource { source: region, columns: [r_regionkey, r_name, r_comment, _row_id], time_range: (Unbounded, Unbounded) } + │ │ ├─LogicalSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id] } + │ │ └─LogicalSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } + │ └─LogicalSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] } + └─LogicalSource { source: region, columns: [r_regionkey, r_name, r_comment, _row_id] } optimized_logical_plan_for_batch: |- LogicalJoin { type: Inner, on: (n_regionkey = r_regionkey), output: [s_acctbal, s_name, n_name, p_partkey, p_mfgr, s_address, s_phone, s_comment] } ├─LogicalJoin { type: Inner, on: (s_nationkey = n_nationkey), output: [p_partkey, p_mfgr, s_name, s_address, s_phone, s_acctbal, s_comment, n_name, n_regionkey] } │ ├─LogicalJoin { type: Inner, on: (s_suppkey = ps_suppkey), output: [p_partkey, p_mfgr, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment] } │ │ ├─LogicalJoin { type: Inner, on: (p_partkey = ps_partkey) AND (ps_supplycost = min(ps_supplycost)), output: [p_partkey, p_mfgr, ps_suppkey] } │ │ │ ├─LogicalJoin { type: Inner, on: (p_partkey = ps_partkey), output: [p_partkey, p_mfgr, ps_suppkey, ps_supplycost] } - │ │ │ │ ├─LogicalSource { source: part, columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ │ │ │ └─LogicalSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id], time_range: (Unbounded, Unbounded) } + │ │ │ │ ├─LogicalSource { source: part, columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _row_id] } + │ │ │ │ └─LogicalSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id] } │ │ │ └─LogicalAgg { group_key: [ps_partkey], aggs: [min(ps_supplycost)] } │ │ │ └─LogicalJoin { type: Inner, on: (n_regionkey = r_regionkey), output: [ps_supplycost, ps_partkey] } │ │ │ ├─LogicalJoin { type: Inner, on: (s_nationkey = n_nationkey), output: [ps_partkey, ps_supplycost, n_regionkey] } │ │ │ │ ├─LogicalJoin { type: Inner, on: (s_suppkey = ps_suppkey), output: [ps_partkey, ps_supplycost, s_nationkey] } - │ │ │ │ │ ├─LogicalSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ │ │ │ │ └─LogicalSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ │ │ │ └─LogicalSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ │ │ └─LogicalSource { source: region, columns: [r_regionkey, r_name, r_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ │ └─LogicalSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ └─LogicalSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id], time_range: (Unbounded, Unbounded) } - └─LogicalSource { source: region, columns: [r_regionkey, r_name, r_comment, _row_id], time_range: (Unbounded, Unbounded) } + │ │ │ │ │ ├─LogicalSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id] } + │ │ │ │ │ └─LogicalSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } + │ │ │ │ └─LogicalSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] } + │ │ │ └─LogicalSource { source: region, columns: [r_regionkey, r_name, r_comment, _row_id] } + │ │ └─LogicalSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } + │ └─LogicalSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] } + └─LogicalSource { source: region, columns: [r_regionkey, r_name, r_comment, _row_id] } batch_plan: |- BatchExchange { order: [s_acctbal DESC, n_name ASC, s_name ASC, p_partkey ASC], dist: Single } └─BatchSort { order: [s_acctbal DESC, n_name ASC, s_name ASC, p_partkey ASC] } @@ -183,9 +183,9 @@ │ │ │ └─BatchHashJoin { type: Inner, predicate: p_partkey = ps_partkey AND ps_supplycost = min(ps_supplycost), output: [p_partkey, p_mfgr, ps_suppkey] } │ │ │ ├─BatchHashJoin { type: Inner, predicate: p_partkey = ps_partkey, output: [p_partkey, p_mfgr, ps_suppkey, ps_supplycost] } │ │ │ │ ├─BatchExchange { order: [], dist: HashShard(p_partkey) } - │ │ │ │ │ └─BatchSource { source: part, columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _row_id], filter: (None, None) } + │ │ │ │ │ └─BatchSource { source: part, columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _row_id] } │ │ │ │ └─BatchExchange { order: [], dist: HashShard(ps_partkey) } - │ │ │ │ └─BatchSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id], filter: (None, None) } + │ │ │ │ └─BatchSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id] } │ │ │ └─BatchHashAgg { group_key: [ps_partkey], aggs: [min(ps_supplycost)] } │ │ │ └─BatchExchange { order: [], dist: HashShard(ps_partkey) } │ │ │ └─BatchHashJoin { type: Inner, predicate: n_regionkey = r_regionkey, output: [ps_supplycost, ps_partkey] } @@ -194,19 +194,19 @@ │ │ │ │ ├─BatchExchange { order: [], dist: HashShard(s_nationkey) } │ │ │ │ │ └─BatchHashJoin { type: Inner, predicate: ps_suppkey = s_suppkey, output: [ps_partkey, ps_supplycost, s_nationkey] } │ │ │ │ │ ├─BatchExchange { order: [], dist: HashShard(ps_suppkey) } - │ │ │ │ │ │ └─BatchSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id], filter: (None, None) } + │ │ │ │ │ │ └─BatchSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id] } │ │ │ │ │ └─BatchExchange { order: [], dist: HashShard(s_suppkey) } - │ │ │ │ │ └─BatchSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id], filter: (None, None) } + │ │ │ │ │ └─BatchSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } │ │ │ │ └─BatchExchange { order: [], dist: HashShard(n_nationkey) } - │ │ │ │ └─BatchSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id], filter: (None, None) } + │ │ │ │ └─BatchSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] } │ │ │ └─BatchExchange { order: [], dist: HashShard(r_regionkey) } - │ │ │ └─BatchSource { source: region, columns: [r_regionkey, r_name, r_comment, _row_id], filter: (None, None) } + │ │ │ └─BatchSource { source: region, columns: [r_regionkey, r_name, r_comment, _row_id] } │ │ └─BatchExchange { order: [], dist: HashShard(s_suppkey) } - │ │ └─BatchSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id], filter: (None, None) } + │ │ └─BatchSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } │ └─BatchExchange { order: [], dist: HashShard(n_nationkey) } - │ └─BatchSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id], filter: (None, None) } + │ └─BatchSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] } └─BatchExchange { order: [], dist: HashShard(r_regionkey) } - └─BatchSource { source: region, columns: [r_regionkey, r_name, r_comment, _row_id], filter: (None, None) } + └─BatchSource { source: region, columns: [r_regionkey, r_name, r_comment, _row_id] } stream_plan: |- StreamMaterialize { columns: [s_acctbal, s_name, n_name, p_partkey, p_mfgr, s_address, s_phone, s_comment, _row_id(hidden), min(ps_supplycost)(hidden), ps_partkey(hidden), _row_id#1(hidden), _row_id#2(hidden), r_regionkey(hidden), _row_id#3(hidden), n_nationkey(hidden), _row_id#4(hidden), s_suppkey(hidden)], stream_key: [_row_id, p_partkey, _row_id#1, _row_id#2, r_regionkey, _row_id#3, n_nationkey, _row_id#4, s_suppkey, min(ps_supplycost), ps_partkey], pk_columns: [s_acctbal, n_name, s_name, p_partkey, _row_id, _row_id#1, _row_id#2, r_regionkey, _row_id#3, n_nationkey, _row_id#4, s_suppkey, min(ps_supplycost), ps_partkey], pk_conflict: NoCheck } └─StreamExchange { dist: HashShard(p_partkey, _row_id, min(ps_supplycost), ps_partkey, _row_id, _row_id, r_regionkey, _row_id, n_nationkey, _row_id, s_suppkey) } @@ -488,12 +488,12 @@ │ ├─LogicalJoin { type: Inner, on: true, output: all } │ │ ├─LogicalJoin { type: Inner, on: true, output: all } │ │ │ ├─LogicalJoin { type: Inner, on: true, output: all } - │ │ │ │ ├─LogicalSource { source: customer, columns: [c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ │ │ │ └─LogicalSource { source: orders, columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ │ │ └─LogicalSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ │ └─LogicalSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ └─LogicalSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id], time_range: (Unbounded, Unbounded) } - └─LogicalSource { source: region, columns: [r_regionkey, r_name, r_comment, _row_id], time_range: (Unbounded, Unbounded) } + │ │ │ │ ├─LogicalSource { source: customer, columns: [c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment, _row_id] } + │ │ │ │ └─LogicalSource { source: orders, columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _row_id] } + │ │ │ └─LogicalSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id] } + │ │ └─LogicalSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } + │ └─LogicalSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] } + └─LogicalSource { source: region, columns: [r_regionkey, r_name, r_comment, _row_id] } optimized_logical_plan_for_batch: |- LogicalAgg { group_key: [n_name], aggs: [sum($expr1)] } └─LogicalProject { exprs: [n_name, (l_extendedprice * (1:Int32::Decimal - l_discount)) as $expr1] } @@ -502,12 +502,12 @@ │ ├─LogicalJoin { type: Inner, on: (l_orderkey = o_orderkey) AND (l_suppkey = s_suppkey), output: [s_nationkey, l_extendedprice, l_discount] } │ │ ├─LogicalJoin { type: Inner, on: (c_nationkey = s_nationkey), output: [o_orderkey, s_suppkey, s_nationkey] } │ │ │ ├─LogicalJoin { type: Inner, on: (c_custkey = o_custkey), output: [c_nationkey, o_orderkey] } - │ │ │ │ ├─LogicalSource { source: customer, columns: [c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ │ │ │ └─LogicalSource { source: orders, columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ │ │ └─LogicalSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ │ └─LogicalSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ └─LogicalSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id], time_range: (Unbounded, Unbounded) } - └─LogicalSource { source: region, columns: [r_regionkey, r_name, r_comment, _row_id], time_range: (Unbounded, Unbounded) } + │ │ │ │ ├─LogicalSource { source: customer, columns: [c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment, _row_id] } + │ │ │ │ └─LogicalSource { source: orders, columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _row_id] } + │ │ │ └─LogicalSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } + │ │ └─LogicalSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id] } + │ └─LogicalSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] } + └─LogicalSource { source: region, columns: [r_regionkey, r_name, r_comment, _row_id] } batch_plan: |- BatchExchange { order: [sum($expr1) DESC], dist: Single } └─BatchSort { order: [sum($expr1) DESC] } @@ -524,17 +524,17 @@ │ │ │ ├─BatchExchange { order: [], dist: HashShard(c_nationkey) } │ │ │ │ └─BatchHashJoin { type: Inner, predicate: c_custkey = o_custkey, output: [c_nationkey, o_orderkey] } │ │ │ │ ├─BatchExchange { order: [], dist: HashShard(c_custkey) } - │ │ │ │ │ └─BatchSource { source: customer, columns: [c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment, _row_id], filter: (None, None) } + │ │ │ │ │ └─BatchSource { source: customer, columns: [c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment, _row_id] } │ │ │ │ └─BatchExchange { order: [], dist: HashShard(o_custkey) } - │ │ │ │ └─BatchSource { source: orders, columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _row_id], filter: (None, None) } + │ │ │ │ └─BatchSource { source: orders, columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _row_id] } │ │ │ └─BatchExchange { order: [], dist: HashShard(s_nationkey) } - │ │ │ └─BatchSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id], filter: (None, None) } + │ │ │ └─BatchSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } │ │ └─BatchExchange { order: [], dist: HashShard(l_orderkey, l_suppkey) } - │ │ └─BatchSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id], filter: (None, None) } + │ │ └─BatchSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id] } │ └─BatchExchange { order: [], dist: HashShard(n_nationkey) } - │ └─BatchSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id], filter: (None, None) } + │ └─BatchSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] } └─BatchExchange { order: [], dist: HashShard(r_regionkey) } - └─BatchSource { source: region, columns: [r_regionkey, r_name, r_comment, _row_id], filter: (None, None) } + └─BatchSource { source: region, columns: [r_regionkey, r_name, r_comment, _row_id] } stream_plan: |- StreamMaterialize { columns: [n_name, revenue], stream_key: [n_name], pk_columns: [revenue, n_name], pk_conflict: NoCheck } └─StreamProject { exprs: [n_name, sum($expr1)] } @@ -750,12 +750,12 @@ │ ├─LogicalJoin { type: Inner, on: true, output: all } │ │ ├─LogicalJoin { type: Inner, on: true, output: all } │ │ │ ├─LogicalJoin { type: Inner, on: true, output: all } - │ │ │ │ ├─LogicalSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ │ │ │ └─LogicalSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ │ │ └─LogicalSource { source: orders, columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ │ └─LogicalSource { source: customer, columns: [c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ └─LogicalSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id], time_range: (Unbounded, Unbounded) } - └─LogicalSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id], time_range: (Unbounded, Unbounded) } + │ │ │ │ ├─LogicalSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } + │ │ │ │ └─LogicalSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id] } + │ │ │ └─LogicalSource { source: orders, columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _row_id] } + │ │ └─LogicalSource { source: customer, columns: [c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment, _row_id] } + │ └─LogicalSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] } + └─LogicalSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] } optimized_logical_plan_for_batch: |- LogicalAgg { group_key: [n_name, n_name, $expr1], aggs: [sum($expr2)] } └─LogicalProject { exprs: [n_name, n_name, Extract('YEAR':Varchar, l_shipdate) as $expr1, (l_extendedprice * (1:Int32::Decimal - l_discount)) as $expr2] } @@ -764,12 +764,12 @@ │ ├─LogicalJoin { type: Inner, on: (o_orderkey = l_orderkey), output: [l_extendedprice, l_discount, l_shipdate, n_name, o_custkey] } │ │ ├─LogicalJoin { type: Inner, on: (s_nationkey = n_nationkey), output: [l_orderkey, l_extendedprice, l_discount, l_shipdate, n_name] } │ │ │ ├─LogicalJoin { type: Inner, on: (s_suppkey = l_suppkey), output: [s_nationkey, l_orderkey, l_extendedprice, l_discount, l_shipdate] } - │ │ │ │ ├─LogicalSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ │ │ │ └─LogicalSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ │ │ └─LogicalSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ │ └─LogicalSource { source: orders, columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ └─LogicalSource { source: customer, columns: [c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment, _row_id], time_range: (Unbounded, Unbounded) } - └─LogicalSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id], time_range: (Unbounded, Unbounded) } + │ │ │ │ ├─LogicalSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } + │ │ │ │ └─LogicalSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id] } + │ │ │ └─LogicalSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] } + │ │ └─LogicalSource { source: orders, columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _row_id] } + │ └─LogicalSource { source: customer, columns: [c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment, _row_id] } + └─LogicalSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] } batch_plan: |- BatchExchange { order: [n_name ASC, n_name ASC, $expr1 ASC], dist: Single } └─BatchSort { order: [n_name ASC, n_name ASC, $expr1 ASC] } @@ -786,17 +786,17 @@ │ │ │ ├─BatchExchange { order: [], dist: HashShard(s_nationkey) } │ │ │ │ └─BatchHashJoin { type: Inner, predicate: s_suppkey = l_suppkey, output: [s_nationkey, l_orderkey, l_extendedprice, l_discount, l_shipdate] } │ │ │ │ ├─BatchExchange { order: [], dist: HashShard(s_suppkey) } - │ │ │ │ │ └─BatchSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id], filter: (None, None) } + │ │ │ │ │ └─BatchSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } │ │ │ │ └─BatchExchange { order: [], dist: HashShard(l_suppkey) } - │ │ │ │ └─BatchSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id], filter: (None, None) } + │ │ │ │ └─BatchSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id] } │ │ │ └─BatchExchange { order: [], dist: HashShard(n_nationkey) } - │ │ │ └─BatchSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id], filter: (None, None) } + │ │ │ └─BatchSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] } │ │ └─BatchExchange { order: [], dist: HashShard(o_orderkey) } - │ │ └─BatchSource { source: orders, columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _row_id], filter: (None, None) } + │ │ └─BatchSource { source: orders, columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _row_id] } │ └─BatchExchange { order: [], dist: HashShard(c_custkey) } - │ └─BatchSource { source: customer, columns: [c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment, _row_id], filter: (None, None) } + │ └─BatchSource { source: customer, columns: [c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment, _row_id] } └─BatchExchange { order: [], dist: HashShard(n_nationkey) } - └─BatchSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id], filter: (None, None) } + └─BatchSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] } stream_plan: |- StreamMaterialize { columns: [supp_nation, cust_nation, l_year, revenue], stream_key: [supp_nation, cust_nation, l_year], pk_columns: [supp_nation, cust_nation, l_year], pk_conflict: NoCheck } └─StreamProject { exprs: [n_name, n_name, $expr1, sum($expr2)] } @@ -1020,14 +1020,14 @@ │ │ │ ├─LogicalJoin { type: Inner, on: true, output: all } │ │ │ │ ├─LogicalJoin { type: Inner, on: true, output: all } │ │ │ │ │ ├─LogicalJoin { type: Inner, on: true, output: all } - │ │ │ │ │ │ ├─LogicalSource { source: part, columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ │ │ │ │ │ └─LogicalSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ │ │ │ │ └─LogicalSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ │ │ │ └─LogicalSource { source: orders, columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ │ │ └─LogicalSource { source: customer, columns: [c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ │ └─LogicalSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ └─LogicalSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id], time_range: (Unbounded, Unbounded) } - └─LogicalSource { source: region, columns: [r_regionkey, r_name, r_comment, _row_id], time_range: (Unbounded, Unbounded) } + │ │ │ │ │ │ ├─LogicalSource { source: part, columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _row_id] } + │ │ │ │ │ │ └─LogicalSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } + │ │ │ │ │ └─LogicalSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id] } + │ │ │ │ └─LogicalSource { source: orders, columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _row_id] } + │ │ │ └─LogicalSource { source: customer, columns: [c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment, _row_id] } + │ │ └─LogicalSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] } + │ └─LogicalSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] } + └─LogicalSource { source: region, columns: [r_regionkey, r_name, r_comment, _row_id] } optimized_logical_plan_for_batch: |- LogicalProject { exprs: [$expr1, RoundDigit((sum($expr3) / sum($expr2)), 6:Int32) as $expr4] } └─LogicalAgg { group_key: [$expr1], aggs: [sum($expr3), sum($expr2)] } @@ -1040,14 +1040,14 @@ │ │ │ ├─LogicalJoin { type: Inner, on: (s_nationkey = n_nationkey), output: [l_orderkey, l_extendedprice, l_discount, n_name] } │ │ │ │ ├─LogicalJoin { type: Inner, on: (s_suppkey = l_suppkey), output: [l_orderkey, l_extendedprice, l_discount, s_nationkey] } │ │ │ │ │ ├─LogicalJoin { type: Inner, on: (p_partkey = l_partkey), output: [l_orderkey, l_suppkey, l_extendedprice, l_discount] } - │ │ │ │ │ │ ├─LogicalSource { source: part, columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ │ │ │ │ │ └─LogicalSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ │ │ │ │ └─LogicalSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ │ │ │ └─LogicalSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ │ │ └─LogicalSource { source: orders, columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ │ └─LogicalSource { source: customer, columns: [c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ └─LogicalSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id], time_range: (Unbounded, Unbounded) } - └─LogicalSource { source: region, columns: [r_regionkey, r_name, r_comment, _row_id], time_range: (Unbounded, Unbounded) } + │ │ │ │ │ │ ├─LogicalSource { source: part, columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _row_id] } + │ │ │ │ │ │ └─LogicalSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id] } + │ │ │ │ │ └─LogicalSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } + │ │ │ │ └─LogicalSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] } + │ │ │ └─LogicalSource { source: orders, columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _row_id] } + │ │ └─LogicalSource { source: customer, columns: [c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment, _row_id] } + │ └─LogicalSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] } + └─LogicalSource { source: region, columns: [r_regionkey, r_name, r_comment, _row_id] } batch_plan: |- BatchExchange { order: [$expr1 ASC], dist: Single } └─BatchProject { exprs: [$expr1, RoundDigit((sum($expr3) / sum($expr2)), 6:Int32) as $expr4] } @@ -1070,21 +1070,21 @@ │ │ │ │ │ ├─BatchExchange { order: [], dist: HashShard(l_suppkey) } │ │ │ │ │ │ └─BatchHashJoin { type: Inner, predicate: p_partkey = l_partkey, output: [l_orderkey, l_suppkey, l_extendedprice, l_discount] } │ │ │ │ │ │ ├─BatchExchange { order: [], dist: HashShard(p_partkey) } - │ │ │ │ │ │ │ └─BatchSource { source: part, columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _row_id], filter: (None, None) } + │ │ │ │ │ │ │ └─BatchSource { source: part, columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _row_id] } │ │ │ │ │ │ └─BatchExchange { order: [], dist: HashShard(l_partkey) } - │ │ │ │ │ │ └─BatchSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id], filter: (None, None) } + │ │ │ │ │ │ └─BatchSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id] } │ │ │ │ │ └─BatchExchange { order: [], dist: HashShard(s_suppkey) } - │ │ │ │ │ └─BatchSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id], filter: (None, None) } + │ │ │ │ │ └─BatchSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } │ │ │ │ └─BatchExchange { order: [], dist: HashShard(n_nationkey) } - │ │ │ │ └─BatchSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id], filter: (None, None) } + │ │ │ │ └─BatchSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] } │ │ │ └─BatchExchange { order: [], dist: HashShard(o_orderkey) } - │ │ │ └─BatchSource { source: orders, columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _row_id], filter: (None, None) } + │ │ │ └─BatchSource { source: orders, columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _row_id] } │ │ └─BatchExchange { order: [], dist: HashShard(c_custkey) } - │ │ └─BatchSource { source: customer, columns: [c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment, _row_id], filter: (None, None) } + │ │ └─BatchSource { source: customer, columns: [c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment, _row_id] } │ └─BatchExchange { order: [], dist: HashShard(n_nationkey) } - │ └─BatchSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id], filter: (None, None) } + │ └─BatchSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] } └─BatchExchange { order: [], dist: HashShard(r_regionkey) } - └─BatchSource { source: region, columns: [r_regionkey, r_name, r_comment, _row_id], filter: (None, None) } + └─BatchSource { source: region, columns: [r_regionkey, r_name, r_comment, _row_id] } stream_plan: |- StreamMaterialize { columns: [o_year, mkt_share], stream_key: [o_year], pk_columns: [o_year], pk_conflict: NoCheck } └─StreamProject { exprs: [$expr1, RoundDigit((sum($expr3) / sum($expr2)), 6:Int32) as $expr4] } @@ -1359,12 +1359,12 @@ │ ├─LogicalJoin { type: Inner, on: true, output: all } │ │ ├─LogicalJoin { type: Inner, on: true, output: all } │ │ │ ├─LogicalJoin { type: Inner, on: true, output: all } - │ │ │ │ ├─LogicalSource { source: part, columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ │ │ │ └─LogicalSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ │ │ └─LogicalSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ │ └─LogicalSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ └─LogicalSource { source: orders, columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _row_id], time_range: (Unbounded, Unbounded) } - └─LogicalSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id], time_range: (Unbounded, Unbounded) } + │ │ │ │ ├─LogicalSource { source: part, columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _row_id] } + │ │ │ │ └─LogicalSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } + │ │ │ └─LogicalSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id] } + │ │ └─LogicalSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id] } + │ └─LogicalSource { source: orders, columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _row_id] } + └─LogicalSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] } optimized_logical_plan_for_batch: |- LogicalProject { exprs: [n_name, $expr1, RoundDigit(sum($expr2), 2:Int32) as $expr3] } └─LogicalAgg { group_key: [n_name, $expr1], aggs: [sum($expr2)] } @@ -1374,12 +1374,12 @@ │ ├─LogicalJoin { type: Inner, on: (s_nationkey = n_nationkey), output: [l_orderkey, l_partkey, l_suppkey, l_quantity, l_extendedprice, l_discount, n_name] } │ │ ├─LogicalJoin { type: Inner, on: (s_suppkey = l_suppkey), output: [l_orderkey, l_partkey, l_suppkey, l_quantity, l_extendedprice, l_discount, s_nationkey] } │ │ │ ├─LogicalJoin { type: Inner, on: (p_partkey = l_partkey), output: [l_orderkey, l_partkey, l_suppkey, l_quantity, l_extendedprice, l_discount] } - │ │ │ │ ├─LogicalSource { source: part, columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ │ │ │ └─LogicalSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ │ │ └─LogicalSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ │ └─LogicalSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ └─LogicalSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id], time_range: (Unbounded, Unbounded) } - └─LogicalSource { source: orders, columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _row_id], time_range: (Unbounded, Unbounded) } + │ │ │ │ ├─LogicalSource { source: part, columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _row_id] } + │ │ │ │ └─LogicalSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id] } + │ │ │ └─LogicalSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } + │ │ └─LogicalSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] } + │ └─LogicalSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id] } + └─LogicalSource { source: orders, columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _row_id] } batch_plan: |- BatchExchange { order: [n_name ASC, $expr1 DESC], dist: Single } └─BatchProject { exprs: [n_name, $expr1, RoundDigit(sum($expr2), 2:Int32) as $expr3] } @@ -1397,17 +1397,17 @@ │ │ │ ├─BatchExchange { order: [], dist: HashShard(l_suppkey) } │ │ │ │ └─BatchHashJoin { type: Inner, predicate: p_partkey = l_partkey, output: [l_orderkey, l_partkey, l_suppkey, l_quantity, l_extendedprice, l_discount] } │ │ │ │ ├─BatchExchange { order: [], dist: HashShard(p_partkey) } - │ │ │ │ │ └─BatchSource { source: part, columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _row_id], filter: (None, None) } + │ │ │ │ │ └─BatchSource { source: part, columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _row_id] } │ │ │ │ └─BatchExchange { order: [], dist: HashShard(l_partkey) } - │ │ │ │ └─BatchSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id], filter: (None, None) } + │ │ │ │ └─BatchSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id] } │ │ │ └─BatchExchange { order: [], dist: HashShard(s_suppkey) } - │ │ │ └─BatchSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id], filter: (None, None) } + │ │ │ └─BatchSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } │ │ └─BatchExchange { order: [], dist: HashShard(n_nationkey) } - │ │ └─BatchSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id], filter: (None, None) } + │ │ └─BatchSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] } │ └─BatchExchange { order: [], dist: HashShard(ps_partkey, ps_suppkey) } - │ └─BatchSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id], filter: (None, None) } + │ └─BatchSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id] } └─BatchExchange { order: [], dist: HashShard(o_orderkey) } - └─BatchSource { source: orders, columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _row_id], filter: (None, None) } + └─BatchSource { source: orders, columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _row_id] } stream_plan: |- StreamMaterialize { columns: [nation, o_year, sum_profit], stream_key: [nation, o_year], pk_columns: [nation, o_year], pk_conflict: NoCheck } └─StreamProject { exprs: [n_name, $expr1, RoundDigit(sum($expr2), 2:Int32) as $expr3] } @@ -1607,34 +1607,34 @@ └─LogicalFilter { predicate: (s_nationkey = n_nationkey) } └─LogicalApply { type: LeftSemi, on: (s_suppkey = ps_suppkey), correlated_id: 1 } ├─LogicalJoin { type: Inner, on: true, output: all } - │ ├─LogicalSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ └─LogicalSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id], time_range: (Unbounded, Unbounded) } + │ ├─LogicalSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } + │ └─LogicalSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] } └─LogicalProject { exprs: [ps_suppkey] } └─LogicalFilter { predicate: (ps_availqty::Decimal > $expr1) } └─LogicalApply { type: LeftOuter, on: true, correlated_id: 3, max_one_row: true } ├─LogicalApply { type: LeftSemi, on: (ps_partkey = p_partkey), correlated_id: 2 } - │ ├─LogicalSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id], time_range: (Unbounded, Unbounded) } + │ ├─LogicalSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id] } │ └─LogicalProject { exprs: [p_partkey] } - │ └─LogicalSource { source: part, columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _row_id], time_range: (Unbounded, Unbounded) } + │ └─LogicalSource { source: part, columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _row_id] } └─LogicalProject { exprs: [(0.5:Decimal * sum(l_quantity)) as $expr1] } └─LogicalAgg { aggs: [sum(l_quantity)] } └─LogicalProject { exprs: [l_quantity] } └─LogicalFilter { predicate: (l_partkey = CorrelatedInputRef { index: 0, correlated_id: 3 }) AND (l_suppkey = CorrelatedInputRef { index: 1, correlated_id: 3 }) } - └─LogicalSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id], time_range: (Unbounded, Unbounded) } + └─LogicalSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id] } optimized_logical_plan_for_batch: |- LogicalJoin { type: LeftSemi, on: (s_suppkey = ps_suppkey), output: [s_name, s_address] } ├─LogicalJoin { type: Inner, on: (s_nationkey = n_nationkey), output: [s_suppkey, s_name, s_address] } - │ ├─LogicalSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ └─LogicalSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id], time_range: (Unbounded, Unbounded) } + │ ├─LogicalSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } + │ └─LogicalSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] } └─LogicalJoin { type: Inner, on: (ps_partkey = l_partkey) AND (ps_suppkey = l_suppkey) AND ($expr1 > $expr2), output: [ps_suppkey] } ├─LogicalProject { exprs: [ps_partkey, ps_suppkey, ps_availqty::Decimal as $expr1] } │ └─LogicalJoin { type: LeftSemi, on: (ps_partkey = p_partkey), output: [ps_partkey, ps_suppkey, ps_availqty] } - │ ├─LogicalSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id], time_range: (Unbounded, Unbounded) } + │ ├─LogicalSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id] } │ └─LogicalProject { exprs: [p_partkey] } - │ └─LogicalSource { source: part, columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _row_id], time_range: (Unbounded, Unbounded) } + │ └─LogicalSource { source: part, columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _row_id] } └─LogicalProject { exprs: [(0.5:Decimal * sum(l_quantity)) as $expr2, l_partkey, l_suppkey] } └─LogicalAgg { group_key: [l_partkey, l_suppkey], aggs: [sum(l_quantity)] } - └─LogicalSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id], time_range: (Unbounded, Unbounded) } + └─LogicalSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id] } batch_plan: |- BatchExchange { order: [s_name ASC], dist: Single } └─BatchSort { order: [s_name ASC] } @@ -1642,23 +1642,23 @@ ├─BatchExchange { order: [], dist: HashShard(s_suppkey) } │ └─BatchHashJoin { type: Inner, predicate: s_nationkey = n_nationkey, output: [s_suppkey, s_name, s_address] } │ ├─BatchExchange { order: [], dist: HashShard(s_nationkey) } - │ │ └─BatchSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id], filter: (None, None) } + │ │ └─BatchSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } │ └─BatchExchange { order: [], dist: HashShard(n_nationkey) } - │ └─BatchSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id], filter: (None, None) } + │ └─BatchSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] } └─BatchExchange { order: [], dist: HashShard(ps_suppkey) } └─BatchHashJoin { type: Inner, predicate: ps_partkey = l_partkey AND ps_suppkey = l_suppkey AND ($expr1 > $expr2), output: [ps_suppkey] } ├─BatchExchange { order: [], dist: HashShard(ps_partkey, ps_suppkey) } │ └─BatchProject { exprs: [ps_partkey, ps_suppkey, ps_availqty::Decimal as $expr1] } │ └─BatchHashJoin { type: LeftSemi, predicate: ps_partkey = p_partkey, output: [ps_partkey, ps_suppkey, ps_availqty] } │ ├─BatchExchange { order: [], dist: HashShard(ps_partkey) } - │ │ └─BatchSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id], filter: (None, None) } + │ │ └─BatchSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id] } │ └─BatchExchange { order: [], dist: HashShard(p_partkey) } │ └─BatchProject { exprs: [p_partkey] } - │ └─BatchSource { source: part, columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _row_id], filter: (None, None) } + │ └─BatchSource { source: part, columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _row_id] } └─BatchProject { exprs: [(0.5:Decimal * sum(l_quantity)) as $expr2, l_partkey, l_suppkey] } └─BatchHashAgg { group_key: [l_partkey, l_suppkey], aggs: [sum(l_quantity)] } └─BatchExchange { order: [], dist: HashShard(l_partkey, l_suppkey) } - └─BatchSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id], filter: (None, None) } + └─BatchSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id] } stream_plan: |- StreamMaterialize { columns: [s_name, s_address, _row_id(hidden), _row_id#1(hidden), s_nationkey(hidden), s_suppkey(hidden)], stream_key: [_row_id, _row_id#1, s_nationkey, s_suppkey], pk_columns: [s_name, _row_id, _row_id#1, s_nationkey, s_suppkey], pk_conflict: NoCheck } └─StreamExchange { dist: HashShard(_row_id, _row_id, s_nationkey, s_suppkey) } @@ -1840,16 +1840,16 @@ │ ├─LogicalJoin { type: Inner, on: true, output: all } │ │ ├─LogicalJoin { type: Inner, on: true, output: all } │ │ │ ├─LogicalJoin { type: Inner, on: true, output: all } - │ │ │ │ ├─LogicalSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ │ │ │ └─LogicalSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ │ │ └─LogicalSource { source: orders, columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ │ └─LogicalSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id], time_range: (Unbounded, Unbounded) } + │ │ │ │ ├─LogicalSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } + │ │ │ │ └─LogicalSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id] } + │ │ │ └─LogicalSource { source: orders, columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _row_id] } + │ │ └─LogicalSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] } │ └─LogicalProject { exprs: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment] } │ └─LogicalFilter { predicate: (l_orderkey = CorrelatedInputRef { index: 8, correlated_id: 1 }) AND (l_suppkey <> CorrelatedInputRef { index: 10, correlated_id: 1 }) } - │ └─LogicalSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id], time_range: (Unbounded, Unbounded) } + │ └─LogicalSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id] } └─LogicalProject { exprs: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment] } └─LogicalFilter { predicate: (l_orderkey = CorrelatedInputRef { index: 8, correlated_id: 2 }) AND (l_suppkey <> CorrelatedInputRef { index: 10, correlated_id: 2 }) AND (l_receiptdate > l_commitdate) } - └─LogicalSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id], time_range: (Unbounded, Unbounded) } + └─LogicalSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id] } optimized_logical_plan_for_batch: |- LogicalAgg { group_key: [s_name], aggs: [count] } └─LogicalJoin { type: LeftAnti, on: (l_orderkey = l_orderkey) AND (l_suppkey <> l_suppkey), output: [s_name] } @@ -1857,16 +1857,16 @@ │ ├─LogicalJoin { type: Inner, on: (o_orderkey = l_orderkey), output: [s_name, l_orderkey, l_suppkey] } │ │ ├─LogicalJoin { type: Inner, on: (s_nationkey = n_nationkey), output: [s_name, l_orderkey, l_suppkey] } │ │ │ ├─LogicalJoin { type: Inner, on: (s_suppkey = l_suppkey), output: [s_name, s_nationkey, l_orderkey, l_suppkey] } - │ │ │ │ ├─LogicalSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id], time_range: (Unbounded, Unbounded) } + │ │ │ │ ├─LogicalSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } │ │ │ │ └─LogicalFilter { predicate: (l_receiptdate > l_commitdate) } - │ │ │ │ └─LogicalSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ │ │ └─LogicalSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ │ └─LogicalSource { source: orders, columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _row_id], time_range: (Unbounded, Unbounded) } + │ │ │ │ └─LogicalSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id] } + │ │ │ └─LogicalSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] } + │ │ └─LogicalSource { source: orders, columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _row_id] } │ └─LogicalProject { exprs: [l_orderkey, l_suppkey] } - │ └─LogicalSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id], time_range: (Unbounded, Unbounded) } + │ └─LogicalSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id] } └─LogicalProject { exprs: [l_orderkey, l_suppkey] } └─LogicalFilter { predicate: (l_receiptdate > l_commitdate) } - └─LogicalSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id], time_range: (Unbounded, Unbounded) } + └─LogicalSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id] } batch_plan: |- BatchExchange { order: [count DESC, s_name ASC], dist: Single } └─BatchSort { order: [count DESC, s_name ASC] } @@ -1880,21 +1880,21 @@ │ │ │ ├─BatchExchange { order: [], dist: HashShard(s_nationkey) } │ │ │ │ └─BatchHashJoin { type: Inner, predicate: s_suppkey = l_suppkey, output: [s_name, s_nationkey, l_orderkey, l_suppkey] } │ │ │ │ ├─BatchExchange { order: [], dist: HashShard(s_suppkey) } - │ │ │ │ │ └─BatchSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id], filter: (None, None) } + │ │ │ │ │ └─BatchSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } │ │ │ │ └─BatchExchange { order: [], dist: HashShard(l_suppkey) } │ │ │ │ └─BatchFilter { predicate: (l_receiptdate > l_commitdate) } - │ │ │ │ └─BatchSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id], filter: (None, None) } + │ │ │ │ └─BatchSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id] } │ │ │ └─BatchExchange { order: [], dist: HashShard(n_nationkey) } - │ │ │ └─BatchSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id], filter: (None, None) } + │ │ │ └─BatchSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] } │ │ └─BatchExchange { order: [], dist: HashShard(o_orderkey) } - │ │ └─BatchSource { source: orders, columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _row_id], filter: (None, None) } + │ │ └─BatchSource { source: orders, columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _row_id] } │ └─BatchExchange { order: [], dist: HashShard(l_orderkey) } │ └─BatchProject { exprs: [l_orderkey, l_suppkey] } - │ └─BatchSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id], filter: (None, None) } + │ └─BatchSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id] } └─BatchExchange { order: [], dist: HashShard(l_orderkey) } └─BatchProject { exprs: [l_orderkey, l_suppkey] } └─BatchFilter { predicate: (l_receiptdate > l_commitdate) } - └─BatchSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id], filter: (None, None) } + └─BatchSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id] } stream_plan: |- StreamMaterialize { columns: [s_name, numwait], stream_key: [s_name], pk_columns: [numwait, s_name], pk_conflict: NoCheck } └─StreamHashAgg { group_key: [s_name], aggs: [count] } diff --git a/src/frontend/planner_test/tests/testdata/output/watermark.yaml b/src/frontend/planner_test/tests/testdata/output/watermark.yaml index 954734459861c..67e098154ece6 100644 --- a/src/frontend/planner_test/tests/testdata/output/watermark.yaml +++ b/src/frontend/planner_test/tests/testdata/output/watermark.yaml @@ -5,7 +5,7 @@ select t.v1 - INTERVAL '2' SECOND as v1 from t; logical_plan: |- LogicalProject { exprs: [(v1 - '00:00:02':Interval) as $expr1] } - └─LogicalSource { source: t, columns: [v1, _row_id], time_range: (Unbounded, Unbounded) } + └─LogicalSource { source: t, columns: [v1, _row_id] } stream_plan: |- StreamMaterialize { columns: [v1, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: NoCheck, watermark_columns: [v1] } └─StreamProject { exprs: [SubtractWithTimeZone(v1, '00:00:02':Interval, 'UTC':Varchar) as $expr1, _row_id], output_watermarks: [$expr1] } diff --git a/src/frontend/src/optimizer/logical_optimization.rs b/src/frontend/src/optimizer/logical_optimization.rs index 6448bd76d8c78..834c293f212b6 100644 --- a/src/frontend/src/optimizer/logical_optimization.rs +++ b/src/frontend/src/optimizer/logical_optimization.rs @@ -424,6 +424,14 @@ static LOGICAL_FILTER_EXPRESSION_SIMPLIFY: LazyLock = LazyLoc ) }); +static REWRITE_SOURCE_FOR_BATCH: LazyLock = LazyLock::new(|| { + OptimizationStage::new( + "Rewrite Source For Batch", + vec![SourceToKafkaScanRule::create()], + ApplyOrder::TopDown, + ) +}); + impl LogicalOptimizer { pub fn predicate_pushdown( plan: PlanRef, @@ -661,6 +669,7 @@ impl LogicalOptimizer { // Convert the dag back to the tree, because we don't support DAG plan for batch. plan = plan.optimize_by_rules(&DAG_TO_TREE); + plan = plan.optimize_by_rules(&REWRITE_SOURCE_FOR_BATCH); plan = plan.optimize_by_rules(&GROUPING_SETS); plan = plan.optimize_by_rules(&REWRITE_LIKE_EXPR); plan = plan.optimize_by_rules(&SET_OPERATION_MERGE); diff --git a/src/frontend/src/optimizer/mod.rs b/src/frontend/src/optimizer/mod.rs index c498f43b95208..f68f2f3bc9e0c 100644 --- a/src/frontend/src/optimizer/mod.rs +++ b/src/frontend/src/optimizer/mod.rs @@ -974,6 +974,7 @@ fn require_additional_exchange_on_root_in_distributed_mode(plan: PlanRef) -> boo fn is_source(plan: &PlanRef) -> bool { plan.node_type() == PlanNodeType::BatchSource + || plan.node_type() == PlanNodeType::BatchKafkaScan } fn is_insert(plan: &PlanRef) -> bool { @@ -1005,6 +1006,7 @@ fn require_additional_exchange_on_root_in_local_mode(plan: PlanRef) -> bool { fn is_source(plan: &PlanRef) -> bool { plan.node_type() == PlanNodeType::BatchSource + || plan.node_type() == PlanNodeType::BatchKafkaScan } fn is_insert(plan: &PlanRef) -> bool { diff --git a/src/frontend/src/optimizer/plan_node/batch_kafka_scan.rs b/src/frontend/src/optimizer/plan_node/batch_kafka_scan.rs new file mode 100644 index 0000000000000..423fe2a6771ee --- /dev/null +++ b/src/frontend/src/optimizer/plan_node/batch_kafka_scan.rs @@ -0,0 +1,140 @@ +// Copyright 2024 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::ops::Bound; +use std::ops::Bound::{Excluded, Included, Unbounded}; +use std::rc::Rc; + +use pretty_xmlish::{Pretty, XmlNode}; +use risingwave_pb::batch_plan::plan_node::NodeBody; +use risingwave_pb::batch_plan::SourceNode; + +use super::batch::prelude::*; +use super::utils::{childless_record, column_names_pretty, Distill}; +use super::{ + generic, ExprRewritable, PlanBase, PlanRef, ToBatchPb, ToDistributedBatch, ToLocalBatch, +}; +use crate::catalog::source_catalog::SourceCatalog; +use crate::error::Result; +use crate::optimizer::plan_node::expr_visitable::ExprVisitable; +use crate::optimizer::property::{Distribution, Order}; + +#[derive(Debug, Clone, PartialEq, Eq, Hash)] +pub struct BatchKafkaScan { + pub base: PlanBase, + pub core: generic::Source, + + /// Kafka timestamp range. + kafka_timestamp_range: (Bound, Bound), +} + +impl BatchKafkaScan { + pub fn new(core: generic::Source, kafka_timestamp_range: (Bound, Bound)) -> Self { + let base = PlanBase::new_batch_with_core( + &core, + // Use `Single` by default, will be updated later with `clone_with_dist`. + Distribution::Single, + Order::any(), + ); + + Self { + base, + core, + kafka_timestamp_range, + } + } + + pub fn column_names(&self) -> Vec<&str> { + self.schema().names_str() + } + + pub fn source_catalog(&self) -> Option> { + self.core.catalog.clone() + } + + pub fn kafka_timestamp_range_value(&self) -> (Option, Option) { + let (lower_bound, upper_bound) = &self.kafka_timestamp_range; + let lower_bound = match lower_bound { + Included(t) => Some(*t), + Excluded(t) => Some(*t - 1), + Unbounded => None, + }; + + let upper_bound = match upper_bound { + Included(t) => Some(*t), + Excluded(t) => Some(*t + 1), + Unbounded => None, + }; + (lower_bound, upper_bound) + } + + pub fn clone_with_dist(&self) -> Self { + let base = self + .base + .clone_with_new_distribution(Distribution::SomeShard); + Self { + base, + core: self.core.clone(), + kafka_timestamp_range: self.kafka_timestamp_range, + } + } +} + +impl_plan_tree_node_for_leaf! { BatchKafkaScan } + +impl Distill for BatchKafkaScan { + fn distill<'a>(&self) -> XmlNode<'a> { + let src = Pretty::from(self.source_catalog().unwrap().name.clone()); + let fields = vec![ + ("source", src), + ("columns", column_names_pretty(self.schema())), + ("filter", Pretty::debug(&self.kafka_timestamp_range_value())), + ]; + childless_record("BatchKafkaScan", fields) + } +} + +impl ToLocalBatch for BatchKafkaScan { + fn to_local(&self) -> Result { + Ok(self.clone_with_dist().into()) + } +} + +impl ToDistributedBatch for BatchKafkaScan { + fn to_distributed(&self) -> Result { + Ok(self.clone_with_dist().into()) + } +} + +impl ToBatchPb for BatchKafkaScan { + fn to_batch_prost_body(&self) -> NodeBody { + let source_catalog = self.source_catalog().unwrap(); + NodeBody::Source(SourceNode { + source_id: source_catalog.id, + info: Some(source_catalog.info.clone()), + columns: self + .core + .column_catalog + .iter() + .map(|c| c.to_protobuf()) + .collect(), + with_properties: source_catalog.with_properties.clone().into_iter().collect(), + split: vec![], + }) + } +} + +impl ExprRewritable for BatchKafkaScan {} + +impl ExprVisitable for BatchKafkaScan {} diff --git a/src/frontend/src/optimizer/plan_node/batch_source.rs b/src/frontend/src/optimizer/plan_node/batch_source.rs index a0ece261dcea0..3836940be745e 100644 --- a/src/frontend/src/optimizer/plan_node/batch_source.rs +++ b/src/frontend/src/optimizer/plan_node/batch_source.rs @@ -56,10 +56,6 @@ impl BatchSource { self.core.catalog.clone() } - pub fn kafka_timestamp_range_value(&self) -> (Option, Option) { - self.core.kafka_timestamp_range_value() - } - pub fn as_of(&self) -> Option { self.core.as_of.clone() } @@ -83,7 +79,6 @@ impl Distill for BatchSource { let mut fields = vec![ ("source", src), ("columns", column_names_pretty(self.schema())), - ("filter", Pretty::debug(&self.kafka_timestamp_range_value())), ]; if let Some(as_of) = &self.core.as_of { fields.push(("as_of", Pretty::debug(as_of))); diff --git a/src/frontend/src/optimizer/plan_node/generic/source.rs b/src/frontend/src/optimizer/plan_node/generic/source.rs index 0c1caa620a4bb..40bd376dd46ca 100644 --- a/src/frontend/src/optimizer/plan_node/generic/source.rs +++ b/src/frontend/src/optimizer/plan_node/generic/source.rs @@ -12,8 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::ops::Bound; -use std::ops::Bound::{Excluded, Included, Unbounded}; use std::rc::Rc; use educe::Educe; @@ -65,9 +63,6 @@ pub struct Source { #[educe(Hash(ignore))] pub ctx: OptimizerContextRef, - /// Kafka timestamp range, currently we only support kafka, so we just leave it like this. - pub(crate) kafka_timestamp_range: (Bound, Bound), - pub as_of: Option, } @@ -113,6 +108,12 @@ impl Source { .is_some_and(|catalog| catalog.with_properties.is_iceberg_connector()) } + pub fn is_kafka_connector(&self) -> bool { + self.catalog + .as_ref() + .is_some_and(|catalog| catalog.with_properties.is_kafka_connector()) + } + /// Currently, only iceberg source supports time travel. pub fn support_time_travel(&self) -> bool { self.is_iceberg_connector() @@ -136,22 +137,6 @@ impl Source { (self, original_row_id_index) } - pub fn kafka_timestamp_range_value(&self) -> (Option, Option) { - let (lower_bound, upper_bound) = &self.kafka_timestamp_range; - let lower_bound = match lower_bound { - Included(t) => Some(*t), - Excluded(t) => Some(*t - 1), - Unbounded => None, - }; - - let upper_bound = match upper_bound { - Included(t) => Some(*t), - Excluded(t) => Some(*t + 1), - Unbounded => None, - }; - (lower_bound, upper_bound) - } - pub fn infer_internal_table_catalog(require_dist_key: bool) -> TableCatalog { // note that source's internal table is to store partition_id -> offset mapping and its // schema is irrelevant to input schema diff --git a/src/frontend/src/optimizer/plan_node/logical_kafka_scan.rs b/src/frontend/src/optimizer/plan_node/logical_kafka_scan.rs new file mode 100644 index 0000000000000..18ae05a135ff9 --- /dev/null +++ b/src/frontend/src/optimizer/plan_node/logical_kafka_scan.rs @@ -0,0 +1,346 @@ +// Copyright 2024 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::cmp::{max, min}; +use std::ops::Bound; +use std::ops::Bound::{Excluded, Included, Unbounded}; +use std::rc::Rc; + +use pretty_xmlish::{Pretty, XmlNode}; +use risingwave_common::catalog::{Schema, KAFKA_TIMESTAMP_COLUMN_NAME}; +use risingwave_connector::source::DataType; + +use super::generic::GenericPlanRef; +use super::utils::{childless_record, Distill}; +use super::{ + generic, BatchProject, ColPrunable, ExprRewritable, Logical, LogicalFilter, LogicalProject, + PlanBase, PlanRef, PredicatePushdown, ToBatch, ToStream, +}; +use crate::catalog::source_catalog::SourceCatalog; +use crate::error::Result; +use crate::expr::{Expr, ExprImpl, ExprRewriter, ExprType, ExprVisitor}; +use crate::optimizer::plan_node::expr_visitable::ExprVisitable; +use crate::optimizer::plan_node::utils::column_names_pretty; +use crate::optimizer::plan_node::{ + BatchKafkaScan, ColumnPruningContext, LogicalSource, PredicatePushdownContext, + RewriteStreamContext, ToStreamContext, +}; +use crate::utils::{ColIndexMapping, Condition}; + +/// `LogicalKafkaScan` is only used by batch queries. At the beginning of the batch query optimization, `LogicalSource` with a kafka property would be converted into a `LogicalKafkaScan`. +#[derive(Debug, Clone, PartialEq, Eq, Hash)] +pub struct LogicalKafkaScan { + pub base: PlanBase, + pub core: generic::Source, + + /// Expressions to output. This field presents and will be turned to a `Project` when + /// converting to a physical plan, only if there are generated columns. + output_exprs: Option>, + + /// Kafka timestamp range. + kafka_timestamp_range: (Bound, Bound), +} + +impl LogicalKafkaScan { + pub fn new(logical_source: &LogicalSource) -> Self { + assert!(logical_source.core.is_kafka_connector()); + + let base = logical_source.base.clone_with_new_plan_id(); + let core = logical_source.core.clone(); + let kafka_timestamp_range = (Bound::Unbounded, Bound::Unbounded); + + LogicalKafkaScan { + base, + core, + output_exprs: logical_source.output_exprs.clone(), + kafka_timestamp_range, + } + } + + pub fn source_catalog(&self) -> Option> { + self.core.catalog.clone() + } + + fn clone_with_kafka_timestamp_range(&self, range: (Bound, Bound)) -> Self { + Self { + base: self.base.clone(), + core: self.core.clone(), + output_exprs: self.output_exprs.clone(), + kafka_timestamp_range: range, + } + } +} + +impl_plan_tree_node_for_leaf! {LogicalKafkaScan} +impl Distill for LogicalKafkaScan { + fn distill<'a>(&self) -> XmlNode<'a> { + let fields = if let Some(catalog) = self.source_catalog() { + let src = Pretty::from(catalog.name.clone()); + let time = Pretty::debug(&self.kafka_timestamp_range); + vec![ + ("source", src), + ("columns", column_names_pretty(self.schema())), + ("time_range", time), + ] + } else { + vec![] + }; + childless_record("LogicalKafkaScan", fields) + } +} + +impl ColPrunable for LogicalKafkaScan { + fn prune_col(&self, required_cols: &[usize], _ctx: &mut ColumnPruningContext) -> PlanRef { + let mapping = ColIndexMapping::with_remaining_columns(required_cols, self.schema().len()); + LogicalProject::with_mapping(self.clone().into(), mapping).into() + } +} + +impl ExprRewritable for LogicalKafkaScan { + fn has_rewritable_expr(&self) -> bool { + self.output_exprs.is_some() + } + + fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { + let mut output_exprs = self.output_exprs.clone(); + + for expr in output_exprs.iter_mut().flatten() { + *expr = r.rewrite_expr(expr.clone()); + } + + Self { + output_exprs, + ..self.clone() + } + .into() + } +} + +impl ExprVisitable for LogicalKafkaScan { + fn visit_exprs(&self, v: &mut dyn ExprVisitor) { + self.output_exprs + .iter() + .flatten() + .for_each(|e| v.visit_expr(e)); + } +} + +/// A util function to extract kafka offset timestamp range. +/// +/// Currently we only support limiting kafka offset timestamp range using literals, e.g. we only +/// support expressions like `_rw_kafka_timestamp <= '2022-10-11 1:00:00+00:00'`. +/// +/// # Parameters +/// +/// * `expr`: Expression to be consumed. +/// * `range`: Original timestamp range, if `expr` can be recognized, we will update `range`. +/// * `schema`: Input schema. +/// +/// # Return Value +/// +/// If `expr` can be recognized and consumed by this function, then we return `None`. +/// Otherwise `expr` is returned. +fn expr_to_kafka_timestamp_range( + expr: ExprImpl, + range: &mut (Bound, Bound), + schema: &Schema, +) -> Option { + let merge_upper_bound = |first, second| -> Bound { + match (first, second) { + (first, Unbounded) => first, + (Unbounded, second) => second, + (Included(f1), Included(f2)) => Included(min(f1, f2)), + (Included(f1), Excluded(f2)) => { + if f1 < f2 { + Included(f1) + } else { + Excluded(f2) + } + } + (Excluded(f1), Included(f2)) => { + if f2 < f1 { + Included(f2) + } else { + Excluded(f1) + } + } + (Excluded(f1), Excluded(f2)) => Excluded(min(f1, f2)), + } + }; + + let merge_lower_bound = |first, second| -> Bound { + match (first, second) { + (first, Unbounded) => first, + (Unbounded, second) => second, + (Included(f1), Included(f2)) => Included(max(f1, f2)), + (Included(f1), Excluded(f2)) => { + if f1 > f2 { + Included(f1) + } else { + Excluded(f2) + } + } + (Excluded(f1), Included(f2)) => { + if f2 > f1 { + Included(f2) + } else { + Excluded(f1) + } + } + (Excluded(f1), Excluded(f2)) => Excluded(max(f1, f2)), + } + }; + + let extract_timestampz_literal = |expr: &ExprImpl| -> Result> { + match expr { + ExprImpl::FunctionCall(function_call) if function_call.inputs().len() == 2 => { + match (&function_call.inputs()[0], &function_call.inputs()[1]) { + (ExprImpl::InputRef(input_ref), literal) + if let Some(datum) = literal.try_fold_const().transpose()? + && schema.fields[input_ref.index].name + == KAFKA_TIMESTAMP_COLUMN_NAME + && literal.return_type() == DataType::Timestamptz => + { + Ok(Some(( + datum.unwrap().into_timestamptz().timestamp_millis(), + false, + ))) + } + (literal, ExprImpl::InputRef(input_ref)) + if let Some(datum) = literal.try_fold_const().transpose()? + && schema.fields[input_ref.index].name + == KAFKA_TIMESTAMP_COLUMN_NAME + && literal.return_type() == DataType::Timestamptz => + { + Ok(Some(( + datum.unwrap().into_timestamptz().timestamp_millis(), + true, + ))) + } + _ => Ok(None), + } + } + _ => Ok(None), + } + }; + + match &expr { + ExprImpl::FunctionCall(function_call) => { + if let Ok(Some((timestampz_literal, reverse))) = extract_timestampz_literal(&expr) { + match function_call.func_type() { + ExprType::GreaterThan => { + if reverse { + range.1 = merge_upper_bound(range.1, Excluded(timestampz_literal)); + } else { + range.0 = merge_lower_bound(range.0, Excluded(timestampz_literal)); + } + + None + } + ExprType::GreaterThanOrEqual => { + if reverse { + range.1 = merge_upper_bound(range.1, Included(timestampz_literal)); + } else { + range.0 = merge_lower_bound(range.0, Included(timestampz_literal)); + } + None + } + ExprType::Equal => { + range.0 = merge_lower_bound(range.0, Included(timestampz_literal)); + range.1 = merge_upper_bound(range.1, Included(timestampz_literal)); + None + } + ExprType::LessThan => { + if reverse { + range.0 = merge_lower_bound(range.0, Excluded(timestampz_literal)); + } else { + range.1 = merge_upper_bound(range.1, Excluded(timestampz_literal)); + } + None + } + ExprType::LessThanOrEqual => { + if reverse { + range.0 = merge_lower_bound(range.0, Included(timestampz_literal)); + } else { + range.1 = merge_upper_bound(range.1, Included(timestampz_literal)); + } + None + } + _ => Some(expr), + } + } else { + Some(expr) + } + } + _ => Some(expr), + } +} + +impl PredicatePushdown for LogicalKafkaScan { + fn predicate_pushdown( + &self, + predicate: Condition, + _ctx: &mut PredicatePushdownContext, + ) -> PlanRef { + let mut range = self.kafka_timestamp_range; + + let mut new_conjunctions = Vec::with_capacity(predicate.conjunctions.len()); + for expr in predicate.conjunctions { + if let Some(e) = expr_to_kafka_timestamp_range(expr, &mut range, self.base.schema()) { + // Not recognized, so push back + new_conjunctions.push(e); + } + } + + let new_source = self.clone_with_kafka_timestamp_range(range).into(); + + if new_conjunctions.is_empty() { + new_source + } else { + LogicalFilter::create( + new_source, + Condition { + conjunctions: new_conjunctions, + }, + ) + } + } +} + +impl ToBatch for LogicalKafkaScan { + fn to_batch(&self) -> Result { + let mut plan: PlanRef = + BatchKafkaScan::new(self.core.clone(), self.kafka_timestamp_range).into(); + + if let Some(exprs) = &self.output_exprs { + let logical_project = generic::Project::new(exprs.to_vec(), plan); + plan = BatchProject::new(logical_project).into(); + } + + Ok(plan) + } +} + +impl ToStream for LogicalKafkaScan { + fn to_stream(&self, _ctx: &mut ToStreamContext) -> Result { + unreachable!() + } + + fn logical_rewrite_for_stream( + &self, + _ctx: &mut RewriteStreamContext, + ) -> Result<(PlanRef, ColIndexMapping)> { + unreachable!() + } +} diff --git a/src/frontend/src/optimizer/plan_node/logical_source.rs b/src/frontend/src/optimizer/plan_node/logical_source.rs index fd4aaba5b7d20..8a8fd37bd66b3 100644 --- a/src/frontend/src/optimizer/plan_node/logical_source.rs +++ b/src/frontend/src/optimizer/plan_node/logical_source.rs @@ -12,17 +12,12 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::cmp::{max, min}; -use std::ops::Bound; -use std::ops::Bound::{Excluded, Included, Unbounded}; use std::rc::Rc; use fixedbitset::FixedBitSet; use pretty_xmlish::{Pretty, XmlNode}; use risingwave_common::bail; -use risingwave_common::catalog::{ - ColumnCatalog, ColumnDesc, Field, Schema, KAFKA_TIMESTAMP_COLUMN_NAME, -}; +use risingwave_common::catalog::{ColumnCatalog, ColumnDesc, Field}; use risingwave_connector::source::iceberg::ICEBERG_CONNECTOR; use risingwave_connector::source::{DataType, UPSTREAM_SOURCE_KEY}; use risingwave_pb::plan_common::column_desc::GeneratedOrDefaultColumn; @@ -39,7 +34,7 @@ use super::{ }; use crate::catalog::source_catalog::SourceCatalog; use crate::error::Result; -use crate::expr::{Expr, ExprImpl, ExprRewriter, ExprType, ExprVisitor, InputRef}; +use crate::expr::{ExprImpl, ExprRewriter, ExprVisitor, InputRef}; use crate::optimizer::optimizer_context::OptimizerContextRef; use crate::optimizer::plan_node::expr_visitable::ExprVisitable; use crate::optimizer::plan_node::stream_fs_fetch::StreamFsFetch; @@ -60,10 +55,10 @@ pub struct LogicalSource { /// Expressions to output. This field presents and will be turned to a `Project` when /// converting to a physical plan, only if there are generated columns. - output_exprs: Option>, + pub(crate) output_exprs: Option>, /// When there are generated columns, the `StreamRowIdGen`'s `row_id_index` is different from /// the one in `core`. So we store the one in `output_exprs` here. - output_row_id_index: Option, + pub(crate) output_row_id_index: Option, } impl LogicalSource { @@ -75,15 +70,12 @@ impl LogicalSource { ctx: OptimizerContextRef, as_of: Option, ) -> Result { - let kafka_timestamp_range = (Bound::Unbounded, Bound::Unbounded); - let core = generic::Source { catalog: source_catalog, column_catalog, row_id_index, kind, ctx, - kafka_timestamp_range, as_of, }; @@ -249,17 +241,6 @@ impl LogicalSource { pub fn source_catalog(&self) -> Option> { self.core.catalog.clone() } - - fn clone_with_kafka_timestamp_range(&self, range: (Bound, Bound)) -> Self { - let mut core = self.core.clone(); - core.kafka_timestamp_range = range; - Self { - base: self.base.clone(), - core, - output_exprs: self.output_exprs.clone(), - output_row_id_index: self.output_row_id_index, - } - } } impl_plan_tree_node_for_leaf! {LogicalSource} @@ -267,11 +248,9 @@ impl Distill for LogicalSource { fn distill<'a>(&self) -> XmlNode<'a> { let fields = if let Some(catalog) = self.source_catalog() { let src = Pretty::from(catalog.name.clone()); - let time = Pretty::debug(&self.core.kafka_timestamp_range); let mut fields = vec![ ("source", src), ("columns", column_names_pretty(self.schema())), - ("time_range", time), ]; if let Some(as_of) = &self.core.as_of { fields.push(("as_of", Pretty::debug(as_of))); @@ -320,190 +299,22 @@ impl ExprVisitable for LogicalSource { } } -/// A util function to extract kafka offset timestamp range. -/// -/// Currently we only support limiting kafka offset timestamp range using literals, e.g. we only -/// support expressions like `_rw_kafka_timestamp <= '2022-10-11 1:00:00+00:00'`. -/// -/// # Parameters -/// -/// * `expr`: Expression to be consumed. -/// * `range`: Original timestamp range, if `expr` can be recognized, we will update `range`. -/// * `schema`: Input schema. -/// -/// # Return Value -/// -/// If `expr` can be recognized and consumed by this function, then we return `None`. -/// Otherwise `expr` is returned. -fn expr_to_kafka_timestamp_range( - expr: ExprImpl, - range: &mut (Bound, Bound), - schema: &Schema, -) -> Option { - let merge_upper_bound = |first, second| -> Bound { - match (first, second) { - (first, Unbounded) => first, - (Unbounded, second) => second, - (Included(f1), Included(f2)) => Included(min(f1, f2)), - (Included(f1), Excluded(f2)) => { - if f1 < f2 { - Included(f1) - } else { - Excluded(f2) - } - } - (Excluded(f1), Included(f2)) => { - if f2 < f1 { - Included(f2) - } else { - Excluded(f1) - } - } - (Excluded(f1), Excluded(f2)) => Excluded(min(f1, f2)), - } - }; - - let merge_lower_bound = |first, second| -> Bound { - match (first, second) { - (first, Unbounded) => first, - (Unbounded, second) => second, - (Included(f1), Included(f2)) => Included(max(f1, f2)), - (Included(f1), Excluded(f2)) => { - if f1 > f2 { - Included(f1) - } else { - Excluded(f2) - } - } - (Excluded(f1), Included(f2)) => { - if f2 > f1 { - Included(f2) - } else { - Excluded(f1) - } - } - (Excluded(f1), Excluded(f2)) => Excluded(max(f1, f2)), - } - }; - - let extract_timestampz_literal = |expr: &ExprImpl| -> Result> { - match expr { - ExprImpl::FunctionCall(function_call) if function_call.inputs().len() == 2 => { - match (&function_call.inputs()[0], &function_call.inputs()[1]) { - (ExprImpl::InputRef(input_ref), literal) - if let Some(datum) = literal.try_fold_const().transpose()? - && schema.fields[input_ref.index].name - == KAFKA_TIMESTAMP_COLUMN_NAME - && literal.return_type() == DataType::Timestamptz => - { - Ok(Some(( - datum.unwrap().into_timestamptz().timestamp_millis(), - false, - ))) - } - (literal, ExprImpl::InputRef(input_ref)) - if let Some(datum) = literal.try_fold_const().transpose()? - && schema.fields[input_ref.index].name - == KAFKA_TIMESTAMP_COLUMN_NAME - && literal.return_type() == DataType::Timestamptz => - { - Ok(Some(( - datum.unwrap().into_timestamptz().timestamp_millis(), - true, - ))) - } - _ => Ok(None), - } - } - _ => Ok(None), - } - }; - - match &expr { - ExprImpl::FunctionCall(function_call) => { - if let Ok(Some((timestampz_literal, reverse))) = extract_timestampz_literal(&expr) { - match function_call.func_type() { - ExprType::GreaterThan => { - if reverse { - range.1 = merge_upper_bound(range.1, Excluded(timestampz_literal)); - } else { - range.0 = merge_lower_bound(range.0, Excluded(timestampz_literal)); - } - - None - } - ExprType::GreaterThanOrEqual => { - if reverse { - range.1 = merge_upper_bound(range.1, Included(timestampz_literal)); - } else { - range.0 = merge_lower_bound(range.0, Included(timestampz_literal)); - } - None - } - ExprType::Equal => { - range.0 = merge_lower_bound(range.0, Included(timestampz_literal)); - range.1 = merge_upper_bound(range.1, Included(timestampz_literal)); - None - } - ExprType::LessThan => { - if reverse { - range.0 = merge_lower_bound(range.0, Excluded(timestampz_literal)); - } else { - range.1 = merge_upper_bound(range.1, Excluded(timestampz_literal)); - } - None - } - ExprType::LessThanOrEqual => { - if reverse { - range.0 = merge_lower_bound(range.0, Included(timestampz_literal)); - } else { - range.1 = merge_upper_bound(range.1, Included(timestampz_literal)); - } - None - } - _ => Some(expr), - } - } else { - Some(expr) - } - } - _ => Some(expr), - } -} - impl PredicatePushdown for LogicalSource { fn predicate_pushdown( &self, predicate: Condition, _ctx: &mut PredicatePushdownContext, ) -> PlanRef { - let mut range = self.core.kafka_timestamp_range; - - let mut new_conjunctions = Vec::with_capacity(predicate.conjunctions.len()); - for expr in predicate.conjunctions { - if let Some(e) = expr_to_kafka_timestamp_range(expr, &mut range, self.base.schema()) { - // Not recognized, so push back - new_conjunctions.push(e); - } - } - - let new_source = self.clone_with_kafka_timestamp_range(range).into(); - - if new_conjunctions.is_empty() { - new_source - } else { - LogicalFilter::create( - new_source, - Condition { - conjunctions: new_conjunctions, - }, - ) - } + LogicalFilter::create(self.clone().into(), predicate) } } impl ToBatch for LogicalSource { fn to_batch(&self) -> Result { + assert!( + !self.core.is_kafka_connector(), + "LogicalSource with a kafka property should be converted to LogicalKafkaScan" + ); let mut plan: PlanRef = BatchSource::new(self.core.clone()).into(); if let Some(exprs) = &self.output_exprs { diff --git a/src/frontend/src/optimizer/plan_node/mod.rs b/src/frontend/src/optimizer/plan_node/mod.rs index ff749781f9265..c2c46e0766efb 100644 --- a/src/frontend/src/optimizer/plan_node/mod.rs +++ b/src/frontend/src/optimizer/plan_node/mod.rs @@ -856,6 +856,7 @@ mod logical_hop_window; mod logical_insert; mod logical_intersect; mod logical_join; +mod logical_kafka_scan; mod logical_limit; mod logical_max_one_row; mod logical_multi_join; @@ -903,6 +904,7 @@ mod stream_topn; mod stream_values; mod stream_watermark_filter; +mod batch_kafka_scan; mod derive; mod stream_cdc_table_scan; mod stream_share; @@ -919,6 +921,7 @@ pub use batch_hash_agg::BatchHashAgg; pub use batch_hash_join::BatchHashJoin; pub use batch_hop_window::BatchHopWindow; pub use batch_insert::BatchInsert; +pub use batch_kafka_scan::BatchKafkaScan; pub use batch_limit::BatchLimit; pub use batch_lookup_join::BatchLookupJoin; pub use batch_max_one_row::BatchMaxOneRow; @@ -949,6 +952,7 @@ pub use logical_hop_window::LogicalHopWindow; pub use logical_insert::LogicalInsert; pub use logical_intersect::LogicalIntersect; pub use logical_join::LogicalJoin; +pub use logical_kafka_scan::LogicalKafkaScan; pub use logical_limit::LogicalLimit; pub use logical_max_one_row::LogicalMaxOneRow; pub use logical_multi_join::{LogicalMultiJoin, LogicalMultiJoinBuilder}; @@ -1053,6 +1057,7 @@ macro_rules! for_all_plan_nodes { , { Logical, Intersect } , { Logical, Except } , { Logical, MaxOneRow } + , { Logical, KafkaScan } , { Batch, SimpleAgg } , { Batch, HashAgg } , { Batch, SortAgg } @@ -1080,6 +1085,7 @@ macro_rules! for_all_plan_nodes { , { Batch, Source } , { Batch, OverWindow } , { Batch, MaxOneRow } + , { Batch, KafkaScan } , { Stream, Project } , { Stream, Filter } , { Stream, TableScan } @@ -1151,6 +1157,7 @@ macro_rules! for_logical_plan_nodes { , { Logical, Intersect } , { Logical, Except } , { Logical, MaxOneRow } + , { Logical, KafkaScan } } }; } @@ -1187,6 +1194,7 @@ macro_rules! for_batch_plan_nodes { , { Batch, Source } , { Batch, OverWindow } , { Batch, MaxOneRow } + , { Batch, KafkaScan } } }; } diff --git a/src/frontend/src/optimizer/rule/mod.rs b/src/frontend/src/optimizer/rule/mod.rs index dff3f986ce22a..e52e0ca7a8b37 100644 --- a/src/frontend/src/optimizer/rule/mod.rs +++ b/src/frontend/src/optimizer/rule/mod.rs @@ -157,9 +157,11 @@ pub use apply_hop_window_transpose_rule::*; mod agg_call_merge_rule; pub use agg_call_merge_rule::*; mod pull_up_correlated_predicate_agg_rule; +mod source_to_kafka_scan_rule; mod values_extract_project_rule; pub use batch::batch_push_limit_to_scan_rule::*; pub use pull_up_correlated_predicate_agg_rule::*; +pub use source_to_kafka_scan_rule::*; pub use values_extract_project_rule::*; #[macro_export] @@ -233,6 +235,7 @@ macro_rules! for_all_rules { , { ValuesExtractProjectRule } , { BatchPushLimitToScanRule } , { PullUpCorrelatedPredicateAggRule } + , { SourceToKafkaScanRule } } }; } diff --git a/src/frontend/src/optimizer/rule/source_to_kafka_scan_rule.rs b/src/frontend/src/optimizer/rule/source_to_kafka_scan_rule.rs new file mode 100644 index 0000000000000..244278bdc33c4 --- /dev/null +++ b/src/frontend/src/optimizer/rule/source_to_kafka_scan_rule.rs @@ -0,0 +1,35 @@ +// Copyright 2024 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use super::{BoxedRule, Rule}; +use crate::optimizer::plan_node::{LogicalKafkaScan, LogicalSource}; +use crate::optimizer::PlanRef; + +pub struct SourceToKafkaScanRule {} +impl Rule for SourceToKafkaScanRule { + fn apply(&self, plan: PlanRef) -> Option { + let source: &LogicalSource = plan.as_logical_source()?; + if source.core.is_kafka_connector() { + Some(LogicalKafkaScan::new(source).into()) + } else { + None + } + } +} + +impl SourceToKafkaScanRule { + pub fn create() -> BoxedRule { + Box::new(SourceToKafkaScanRule {}) + } +} diff --git a/src/frontend/src/scheduler/distributed/stage.rs b/src/frontend/src/scheduler/distributed/stage.rs index 9a5e00fbfd702..32dac7200619a 100644 --- a/src/frontend/src/scheduler/distributed/stage.rs +++ b/src/frontend/src/scheduler/distributed/stage.rs @@ -987,7 +987,7 @@ impl StageRunner { node_body: Some(NodeBody::RowSeqScan(scan_node)), } } - PlanNodeType::BatchSource => { + PlanNodeType::BatchSource | PlanNodeType::BatchKafkaScan => { let node_body = execution_plan_node.node.clone(); let NodeBody::Source(mut source_node) = node_body else { unreachable!(); diff --git a/src/frontend/src/scheduler/local.rs b/src/frontend/src/scheduler/local.rs index ede32d2033353..c6340189743f6 100644 --- a/src/frontend/src/scheduler/local.rs +++ b/src/frontend/src/scheduler/local.rs @@ -472,7 +472,7 @@ impl LocalQueryExecution { node_body: Some(node_body), }) } - PlanNodeType::BatchSource => { + PlanNodeType::BatchSource | PlanNodeType::BatchKafkaScan => { let mut node_body = execution_plan_node.node.clone(); match &mut node_body { NodeBody::Source(ref mut source_node) => { diff --git a/src/frontend/src/scheduler/plan_fragmenter.rs b/src/frontend/src/scheduler/plan_fragmenter.rs index d95f4d627926c..a578fd30d2539 100644 --- a/src/frontend/src/scheduler/plan_fragmenter.rs +++ b/src/frontend/src/scheduler/plan_fragmenter.rs @@ -54,7 +54,7 @@ use crate::catalog::catalog_service::CatalogReader; use crate::catalog::TableId; use crate::error::RwError; use crate::optimizer::plan_node::generic::{GenericPlanRef, PhysicalPlanRef}; -use crate::optimizer::plan_node::{BatchSource, PlanNodeId, PlanNodeType}; +use crate::optimizer::plan_node::{BatchKafkaScan, BatchSource, PlanNodeId, PlanNodeType}; use crate::optimizer::property::Distribution; use crate::optimizer::PlanRef; use crate::scheduler::SchedulerResult; @@ -1003,7 +1003,23 @@ impl BatchPlanFragmenter { return Ok(None); } - if let Some(source_node) = node.as_batch_source() { + if let Some(batch_kafka_node) = node.as_batch_kafka_scan() { + let batch_kafka_scan: &BatchKafkaScan = batch_kafka_node; + let source_catalog = batch_kafka_scan.source_catalog(); + if let Some(source_catalog) = source_catalog { + let property = ConnectorProperties::extract( + source_catalog.with_properties.clone().into_iter().collect(), + false, + )?; + let timestamp_bound = batch_kafka_scan.kafka_timestamp_range_value(); + return Ok(Some(SourceScanInfo::new(SourceFetchInfo { + connector: property, + timebound: timestamp_bound, + as_of: None, + }))); + } + } else if let Some(source_node) = node.as_batch_source() { + // TODO: use specific batch operator instead of batch source. let source_node: &BatchSource = source_node; let source_catalog = source_node.source_catalog(); if let Some(source_catalog) = source_catalog { @@ -1011,11 +1027,10 @@ impl BatchPlanFragmenter { source_catalog.with_properties.clone().into_iter().collect(), false, )?; - let timestamp_bound = source_node.kafka_timestamp_range_value(); let as_of = source_node.as_of(); return Ok(Some(SourceScanInfo::new(SourceFetchInfo { connector: property, - timebound: timestamp_bound, + timebound: (None, None), as_of, }))); }