Skip to content

Commit

Permalink
test(nexmark): explain q16 q17 (risingwavelabs#3715)
Browse files Browse the repository at this point in the history
Signed-off-by: TennyZhuang <[email protected]>

Co-authored-by: mergify[bot] <37929162+mergify[bot]@users.noreply.github.com>
  • Loading branch information
2 people authored and nasnoisaac committed Aug 9, 2022
1 parent ef59c80 commit 1a581bc
Showing 1 changed file with 48 additions and 8 deletions.
56 changes: 48 additions & 8 deletions src/frontend/test_runner/tests/testdata/nexmark.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -388,11 +388,10 @@
B.bidder,
B.price,
B.dateTime,
S.`value`
S.value
FROM (SELECT *, PROCTIME() as p_time FROM bid) B
JOIN side_input FOR SYSTEM_TIME AS OF B.p_time AS S
ON mod(B.auction, 10000) = S.key; */
/* parser error */
select 1;
- id: nexmark_q14
before:
Expand Down Expand Up @@ -465,15 +464,44 @@
- id: nexmark_q16
before:
- create_tables
sql: "/* \nSELECT\n channel,\n DATE_FORMAT(dateTime, 'yyyy-MM-dd') as `day`,\n max(DATE_FORMAT(dateTime, 'HH:mm')) as `minute`,\n count(*) AS total_bids,\n count(*) filter (where price < 10000) AS rank1_bids,\n count(*) filter (where price >= 10000 and price < 1000000) AS rank2_bids,\n count(*) filter (where price >= 1000000) AS rank3_bids,\n count(distinct bidder) AS total_bidders,\n count(distinct bidder) filter (where price < 10000) AS rank1_bidders,\n count(distinct bidder) filter (where price >= 10000 and price < 1000000) AS rank2_bidders,\n count(distinct bidder) filter (where price >= 1000000) AS rank3_bidders,\n count(distinct auction) AS total_auctions,\n count(distinct auction) filter (where price < 10000) AS rank1_auctions,\n count(distinct auction) filter (where price >= 10000 and price < 1000000) AS rank2_auctions,\n count(distinct auction) filter (where price >= 1000000) AS rank3_auctions\nFROM bid\nGROUP BY channel, DATE_FORMAT(dateTime, 'yyyy-MM-dd'); */\n/* parser error */\nselect 1\n"
sql: |
SELECT
channel,
to_char(dateTime, 'yyyy-MM-dd') AS day,
max(to_char(dateTime, 'HH:mm')) AS minute,
count(*) AS total_bids,
count(*) filter (where price < 10000) AS rank1_bids,
count(*) filter (where price >= 10000 and price < 1000000) AS rank2_bids,
count(*) filter (where price >= 1000000) AS rank3_bids,
count(distinct bidder) AS total_bidders,
count(distinct bidder) filter (where price < 10000) AS rank1_bidders,
count(distinct bidder) filter (where price >= 10000 and price < 1000000) AS rank2_bidders,
count(distinct bidder) filter (where price >= 1000000) AS rank3_bidders,
count(distinct auction) AS total_auctions,
count(distinct auction) filter (where price < 10000) AS rank1_auctions,
count(distinct auction) filter (where price >= 10000 and price < 1000000) AS rank2_auctions,
count(distinct auction) filter (where price >= 1000000) AS rank3_auctions
FROM bid
GROUP BY channel, to_char(dateTime, 'yyyy-MM-dd');
batch_plan: |
BatchExchange { order: [], dist: Single }
BatchHashAgg { group_key: [$0, $1], aggs: [max($2), count, count filter(($3 < 10000:Int32)), count filter((($3 >= 10000:Int32) AND ($3 < 1000000:Int32))), count filter(($3 >= 1000000:Int32)), count(distinct $4), count(distinct $4) filter(($3 < 10000:Int32)), count(distinct $4) filter((($3 >= 10000:Int32) AND ($3 < 1000000:Int32))), count(distinct $4) filter(($3 >= 1000000:Int32)), count(distinct $5), count(distinct $5) filter(($3 < 10000:Int32)), count(distinct $5) filter((($3 >= 10000:Int32) AND ($3 < 1000000:Int32))), count(distinct $5) filter(($3 >= 1000000:Int32))] }
BatchExchange { order: [], dist: HashShard([0, 1]) }
BatchProject { exprs: [$3, ToChar($4, 'yyyy-MM-dd':Varchar), ToChar($4, 'HH:mm':Varchar), $2, $1, $0] }
BatchScan { table: bid, columns: [auction, bidder, price, channel, dateTime] }
stream_plan: |
StreamMaterialize { columns: [channel, day, agg#0(hidden), 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], pk_columns: [channel, day] }
StreamHashAgg { group_key: [$0, $1], aggs: [count, max($2), count, count filter(($3 < 10000:Int32)), count filter((($3 >= 10000:Int32) AND ($3 < 1000000:Int32))), count filter(($3 >= 1000000:Int32)), count(distinct $4), count(distinct $4) filter(($3 < 10000:Int32)), count(distinct $4) filter((($3 >= 10000:Int32) AND ($3 < 1000000:Int32))), count(distinct $4) filter(($3 >= 1000000:Int32)), count(distinct $5), count(distinct $5) filter(($3 < 10000:Int32)), count(distinct $5) filter((($3 >= 10000:Int32) AND ($3 < 1000000:Int32))), count(distinct $5) filter(($3 >= 1000000:Int32))] }
StreamExchange { dist: HashShard([0, 1]) }
StreamProject { exprs: [$3, ToChar($4, 'yyyy-MM-dd':Varchar), ToChar($4, 'HH:mm':Varchar), $2, $1, $0, $5] }
StreamTableScan { table: bid, columns: [auction, bidder, price, channel, dateTime, _row_id], pk_indices: [5] }
- id: nexmark_q17
before:
- create_tables
sql: |
/*
SELECT
auction,
DATE_FORMAT(dateTime, 'yyyy-MM-dd') as `day`,
to_char(dateTime, 'yyyy-MM-dd') AS day,
count(*) AS total_bids,
count(*) filter (where price < 10000) AS rank1_bids,
count(*) filter (where price >= 10000 and price < 1000000) AS rank2_bids,
Expand All @@ -483,9 +511,21 @@
avg(price) AS avg_price,
sum(price) AS sum_price
FROM bid
GROUP BY auction, DATE_FORMAT(dateTime, 'yyyy-MM-dd'); */
/* parser error */
select 1
GROUP BY auction, to_char(dateTime, 'yyyy-MM-dd');
batch_plan: |
BatchExchange { order: [], dist: Single }
BatchProject { exprs: [$0, $1, $2, $3, $4, $5, $6, $7, ($8 / $9), $10] }
BatchHashAgg { group_key: [$0, $1], aggs: [count, count filter(($2 < 10000:Int32)), count filter((($2 >= 10000:Int32) AND ($2 < 1000000:Int32))), count filter(($2 >= 1000000:Int32)), min($2), max($2), sum($2), count($2), sum($2)] }
BatchExchange { order: [], dist: HashShard([0, 1]) }
BatchProject { exprs: [$0, ToChar($2, 'yyyy-MM-dd':Varchar), $1] }
BatchScan { table: bid, columns: [auction, price, dateTime] }
stream_plan: |
StreamMaterialize { columns: [auction, day, total_bids, rank1_bids, rank2_bids, rank3_bids, min_price, max_price, avg_price, sum_price], pk_columns: [auction, day] }
StreamProject { exprs: [$0, $1, $3, $4, $5, $6, $7, $8, ($9 / $10), $11] }
StreamHashAgg { group_key: [$0, $1], aggs: [count, count, count filter(($2 < 10000:Int32)), count filter((($2 >= 10000:Int32) AND ($2 < 1000000:Int32))), count filter(($2 >= 1000000:Int32)), min($2), max($2), sum($2), count($2), sum($2)] }
StreamExchange { dist: HashShard([0, 1]) }
StreamProject { exprs: [$0, ToChar($2, 'yyyy-MM-dd':Varchar), $1, $3] }
StreamTableScan { table: bid, columns: [auction, price, dateTime, _row_id], pk_indices: [3] }
- id: nexmark_q18
before:
- create_tables
Expand Down

0 comments on commit 1a581bc

Please sign in to comment.