From 17ccb4f682e5f193ac52d1e1b6fda0dd1c7de6d3 Mon Sep 17 00:00:00 2001 From: xxchan Date: Wed, 27 Nov 2024 14:05:25 +0800 Subject: [PATCH] test: add tpch/nexmark tests for shared kafka source --- .../nexmark/create_sources_kafka.slt.part | 48 + e2e_test/nexmark/drop_sources_kafka.slt.part | 8 + e2e_test/nexmark/produce_kafka.slt.part | 58 + e2e_test/source_inline/kafka/nexmark.slt | 9 + e2e_test/source_inline/kafka/tpch.slt | 30 + e2e_test/tpch/create_sources_kafka.slt.part | 118 + e2e_test/tpch/drop_sources_kafka.slt.part | 23 + e2e_test/tpch/produce_kafka.slt.part | 131 + .../testdata/input/nexmark_source_kafka.yaml | 718 +++++ .../tests/testdata/input/tpch_kafka.yaml | 902 ++++++ .../tests/testdata/output/batch_source.yaml | 4 +- .../testdata/output/nexmark_source_kafka.yaml | 2470 +++++++++++++++++ .../tests/testdata/output/shared_source.yml | 12 +- .../tests/testdata/output/tpch_kafka.yaml | 1513 ++++++++++ 14 files changed, 6036 insertions(+), 8 deletions(-) create mode 100644 e2e_test/nexmark/create_sources_kafka.slt.part create mode 100644 e2e_test/nexmark/drop_sources_kafka.slt.part create mode 100644 e2e_test/nexmark/produce_kafka.slt.part create mode 100644 e2e_test/source_inline/kafka/nexmark.slt create mode 100644 e2e_test/source_inline/kafka/tpch.slt create mode 100644 e2e_test/tpch/create_sources_kafka.slt.part create mode 100644 e2e_test/tpch/drop_sources_kafka.slt.part create mode 100644 e2e_test/tpch/produce_kafka.slt.part create mode 100644 src/frontend/planner_test/tests/testdata/input/nexmark_source_kafka.yaml create mode 100644 src/frontend/planner_test/tests/testdata/input/tpch_kafka.yaml create mode 100644 src/frontend/planner_test/tests/testdata/output/nexmark_source_kafka.yaml create mode 100644 src/frontend/planner_test/tests/testdata/output/tpch_kafka.yaml diff --git a/e2e_test/nexmark/create_sources_kafka.slt.part b/e2e_test/nexmark/create_sources_kafka.slt.part new file mode 100644 index 0000000000000..71e05c9134b80 --- /dev/null +++ b/e2e_test/nexmark/create_sources_kafka.slt.part @@ -0,0 +1,48 @@ +control substitution on + +statement ok +CREATE SOURCE person ( + "id" BIGINT, + "name" VARCHAR, + "email_address" VARCHAR, + "credit_card" VARCHAR, + "city" VARCHAR, + "state" VARCHAR, + "date_time" TIMESTAMP, + "extra" VARCHAR, +) WITH ( + ${RISEDEV_KAFKA_WITH_OPTIONS_COMMON}, + topic = 'nexmark-person' +) FORMAT PLAIN ENCODE JSON; + + +statement ok +CREATE SOURCE auction ( + "id" BIGINT, + "item_name" VARCHAR, + "description" VARCHAR, + "initial_bid" BIGINT, + "reserve" BIGINT, + "date_time" TIMESTAMP, + "expires" TIMESTAMP, + "seller" BIGINT, + "category" BIGINT, + "extra" VARCHAR, +) WITH ( + ${RISEDEV_KAFKA_WITH_OPTIONS_COMMON}, + topic = 'nexmark-auction' +) FORMAT PLAIN ENCODE JSON; + +statement ok +CREATE SOURCE bid ( + "auction" BIGINT, + "bidder" BIGINT, + "price" BIGINT, + "channel" VARCHAR, + "url" VARCHAR, + "date_time" TIMESTAMP, + "extra" VARCHAR +) WITH ( + ${RISEDEV_KAFKA_WITH_OPTIONS_COMMON}, + topic = 'nexmark-bid' +) FORMAT PLAIN ENCODE JSON; diff --git a/e2e_test/nexmark/drop_sources_kafka.slt.part b/e2e_test/nexmark/drop_sources_kafka.slt.part new file mode 100644 index 0000000000000..998b1b07c12d1 --- /dev/null +++ b/e2e_test/nexmark/drop_sources_kafka.slt.part @@ -0,0 +1,8 @@ +statement ok +DROP SOURCE person CASCADE; + +statement ok +DROP SOURCE auction CASCADE; + +statement ok +DROP SOURCE bid CASCADE; diff --git a/e2e_test/nexmark/produce_kafka.slt.part b/e2e_test/nexmark/produce_kafka.slt.part new file mode 100644 index 0000000000000..f922a39831e1c --- /dev/null +++ b/e2e_test/nexmark/produce_kafka.slt.part @@ -0,0 +1,58 @@ +control substitution on + +system ok +rpk topic delete -r nexmark-* || true + +system ok +rpk topic create nexmark-auction -p 4 && +rpk topic create nexmark-bid -p 4 && +rpk topic create nexmark-person -p 4 + +include ./create_tables.slt.part + +include ./insert_auction.slt.part +include ./insert_bid.slt.part +include ./insert_person.slt.part + +statement ok +flush; + +statement ok +create sink nexmark_auction FROM auction +WITH ( + ${RISEDEV_KAFKA_WITH_OPTIONS_COMMON}, + topic = 'nexmark-auction' +) FORMAT PLAIN ENCODE JSON ( + force_append_only='true' +); + +statement ok +create sink nexmark_bid FROM bid +WITH ( + ${RISEDEV_KAFKA_WITH_OPTIONS_COMMON}, + topic = 'nexmark-bid' +) FORMAT PLAIN ENCODE JSON ( + force_append_only='true' +); + +statement ok +create sink nexmark_person FROM person +WITH ( + ${RISEDEV_KAFKA_WITH_OPTIONS_COMMON}, + topic = 'nexmark-person' +) FORMAT PLAIN ENCODE JSON ( + force_append_only='true' +); + +sleep 5s + +statement ok +DROP SINK nexmark_auction; + +statement ok +DROP SINK nexmark_bid; + +statement ok +DROP SINK nexmark_person; + +include ./drop_tables.slt.part diff --git a/e2e_test/source_inline/kafka/nexmark.slt b/e2e_test/source_inline/kafka/nexmark.slt new file mode 100644 index 0000000000000..9460a0190b534 --- /dev/null +++ b/e2e_test/source_inline/kafka/nexmark.slt @@ -0,0 +1,9 @@ +include ../../nexmark/produce_kafka.slt.part +include ../../nexmark/create_sources_kafka.slt.part + +control substitution off + +include ../../streaming/nexmark/create_views.slt.part +include ../../streaming/nexmark/test_mv_result.slt.part + +include ../../nexmark/drop_sources_kafka.slt.part diff --git a/e2e_test/source_inline/kafka/tpch.slt b/e2e_test/source_inline/kafka/tpch.slt new file mode 100644 index 0000000000000..85fed5034af4d --- /dev/null +++ b/e2e_test/source_inline/kafka/tpch.slt @@ -0,0 +1,30 @@ +include ../../tpch/produce_kafka.slt.part +include ../../tpch/create_sources_kafka.slt.part + +control substitution off + +include ../../streaming/tpch/create_views.slt.part +include ../../streaming/tpch/q1.slt.part +include ../../streaming/tpch/q2.slt.part +include ../../streaming/tpch/q3.slt.part +include ../../streaming/tpch/q4.slt.part +include ../../streaming/tpch/q5.slt.part +include ../../streaming/tpch/q6.slt.part +include ../../streaming/tpch/q7.slt.part +include ../../streaming/tpch/q8.slt.part +include ../../streaming/tpch/q9.slt.part +include ../../streaming/tpch/q10.slt.part +include ../../streaming/tpch/q11.slt.part +include ../../streaming/tpch/q12.slt.part +include ../../streaming/tpch/q13.slt.part +include ../../streaming/tpch/q14.slt.part +include ../../streaming/tpch/q15.slt.part +include ../../streaming/tpch/q16.slt.part +include ../../streaming/tpch/q17.slt.part +include ../../streaming/tpch/q18.slt.part +include ../../streaming/tpch/q19.slt.part +include ../../streaming/tpch/q20.slt.part +include ../../streaming/tpch/q21.slt.part +include ../../streaming/tpch/q22.slt.part + +include ../../tpch/drop_sources_kafka.slt.part diff --git a/e2e_test/tpch/create_sources_kafka.slt.part b/e2e_test/tpch/create_sources_kafka.slt.part new file mode 100644 index 0000000000000..9dcc50e7d1f0e --- /dev/null +++ b/e2e_test/tpch/create_sources_kafka.slt.part @@ -0,0 +1,118 @@ +control substitution on + +statement ok +CREATE SOURCE supplier ( + s_suppkey INTEGER, + s_name VARCHAR, + s_address VARCHAR, + s_nationkey INTEGER, + s_phone VARCHAR, + s_acctbal NUMERIC, + s_comment VARCHAR +) WITH ( + ${RISEDEV_KAFKA_WITH_OPTIONS_COMMON}, + topic = 'tpch-supplier' +) FORMAT PLAIN ENCODE JSON; + +statement ok +CREATE SOURCE part ( + p_partkey INTEGER, + p_name VARCHAR, + p_mfgr VARCHAR, + p_brand VARCHAR, + p_type VARCHAR, + p_size INTEGER, + p_container VARCHAR, + p_retailprice NUMERIC, + p_comment VARCHAR +) WITH ( + ${RISEDEV_KAFKA_WITH_OPTIONS_COMMON}, + topic = 'tpch-part' +) FORMAT PLAIN ENCODE JSON; + +statement ok +CREATE SOURCE partsupp ( + ps_partkey INTEGER, + ps_suppkey INTEGER, + ps_availqty INTEGER, + ps_supplycost NUMERIC, + ps_comment VARCHAR +) WITH ( + ${RISEDEV_KAFKA_WITH_OPTIONS_COMMON}, + topic = 'tpch-partsupp' +) FORMAT PLAIN ENCODE JSON; + +statement ok +CREATE SOURCE customer ( + c_custkey INTEGER, + c_name VARCHAR, + c_address VARCHAR, + c_nationkey INTEGER, + c_phone VARCHAR, + c_acctbal NUMERIC, + c_mktsegment VARCHAR, + c_comment VARCHAR +) WITH ( + ${RISEDEV_KAFKA_WITH_OPTIONS_COMMON}, + topic = 'tpch-customer' +) FORMAT PLAIN ENCODE JSON; + +statement ok +CREATE SOURCE orders ( + o_orderkey BIGINT, + o_custkey INTEGER, + o_orderstatus VARCHAR, + o_totalprice NUMERIC, + o_orderdate DATE, + o_orderpriority VARCHAR, + o_clerk VARCHAR, + o_shippriority INTEGER, + o_comment VARCHAR +) WITH ( + ${RISEDEV_KAFKA_WITH_OPTIONS_COMMON}, + topic = 'tpch-orders' +) FORMAT PLAIN ENCODE JSON; + +statement ok +CREATE SOURCE lineitem ( + l_orderkey BIGINT, + l_partkey INTEGER, + l_suppkey INTEGER, + l_linenumber INTEGER, + l_quantity NUMERIC, + l_extendedprice NUMERIC, + l_discount NUMERIC, + l_tax NUMERIC, + l_returnflag VARCHAR, + l_linestatus VARCHAR, + l_shipdate DATE, + l_commitdate DATE, + l_receiptdate DATE, + l_shipinstruct VARCHAR, + l_shipmode VARCHAR, + l_comment VARCHAR +) WITH ( + ${RISEDEV_KAFKA_WITH_OPTIONS_COMMON}, + topic = 'tpch-lineitem' +) FORMAT PLAIN ENCODE JSON; + +statement ok +CREATE SOURCE nation ( + n_nationkey INTEGER, + n_name VARCHAR, + n_regionkey INTEGER, + n_comment VARCHAR +) WITH ( + ${RISEDEV_KAFKA_WITH_OPTIONS_COMMON}, + topic = 'tpch-nation' +) FORMAT PLAIN ENCODE JSON; + +statement ok +CREATE SOURCE region ( + r_regionkey INTEGER, + r_name VARCHAR, + r_comment VARCHAR +) WITH ( + ${RISEDEV_KAFKA_WITH_OPTIONS_COMMON}, + topic = 'tpch-region' +) FORMAT PLAIN ENCODE JSON; diff --git a/e2e_test/tpch/drop_sources_kafka.slt.part b/e2e_test/tpch/drop_sources_kafka.slt.part new file mode 100644 index 0000000000000..2c239fcbb422f --- /dev/null +++ b/e2e_test/tpch/drop_sources_kafka.slt.part @@ -0,0 +1,23 @@ +statement ok +DROP SOURCE supplier CASCADE; + +statement ok +DROP SOURCE region CASCADE; + +statement ok +DROP SOURCE nation CASCADE; + +statement ok +DROP SOURCE lineitem CASCADE; + +statement ok +DROP SOURCE orders CASCADE; + +statement ok +DROP SOURCE customer CASCADE; + +statement ok +DROP SOURCE partsupp CASCADE; + +statement ok +DROP SOURCE part CASCADE; diff --git a/e2e_test/tpch/produce_kafka.slt.part b/e2e_test/tpch/produce_kafka.slt.part new file mode 100644 index 0000000000000..07b476a3a9f30 --- /dev/null +++ b/e2e_test/tpch/produce_kafka.slt.part @@ -0,0 +1,131 @@ +control substitution on + +system ok +rpk topic delete -r tpch-* || true + +system ok +rpk topic create tpch-supplier -p 4 && +rpk topic create tpch-part -p 4 && +rpk topic create tpch-partsupp -p 4 && +rpk topic create tpch-customer -p 4 && +rpk topic create tpch-orders -p 4 && +rpk topic create tpch-lineitem -p 4 && +rpk topic create tpch-nation -p 4 && +rpk topic create tpch-region -p 4 + +include ./create_tables.slt.part + +include ./insert_supplier.slt.part +include ./insert_part.slt.part +include ./insert_partsupp.slt.part +include ./insert_customer.slt.part +include ./insert_orders.slt.part +include ./insert_lineitem.slt.part +include ./insert_nation.slt.part +include ./insert_region.slt.part + +statement ok +flush; + +statement ok +create sink kafka_supplier FROM supplier +WITH ( + ${RISEDEV_KAFKA_WITH_OPTIONS_COMMON}, + topic = 'tpch-supplier' +) FORMAT PLAIN ENCODE JSON ( + force_append_only='true' +); + +statement ok +create sink kafka_part FROM part +WITH ( + ${RISEDEV_KAFKA_WITH_OPTIONS_COMMON}, + topic = 'tpch-part' +) FORMAT PLAIN ENCODE JSON ( + force_append_only='true' +); + +statement ok +create sink kafka_partsupp FROM partsupp +WITH ( + ${RISEDEV_KAFKA_WITH_OPTIONS_COMMON}, + topic = 'tpch-partsupp' +) FORMAT PLAIN ENCODE JSON ( + force_append_only='true' +); + +statement ok +create sink kafka_customer FROM customer +WITH ( + ${RISEDEV_KAFKA_WITH_OPTIONS_COMMON}, + topic = 'tpch-customer' +) FORMAT PLAIN ENCODE JSON ( + force_append_only='true' +); + +# note: In source, Date format is days_since_unix_epoch. In sink, it's num_days_from_ce. +# https://github.com/risingwavelabs/risingwave/issues/16467 + +statement ok +create sink kafka_orders AS select * except(o_orderdate), o_orderdate::varchar as o_orderdate FROM orders +WITH ( + ${RISEDEV_KAFKA_WITH_OPTIONS_COMMON}, + topic = 'tpch-orders' +) FORMAT PLAIN ENCODE JSON ( + force_append_only='true' +); + +statement ok +create sink kafka_lineitem AS select * except(l_shipdate, l_commitdate, l_receiptdate), l_shipdate::varchar as l_shipdate, l_commitdate::varchar as l_commitdate, l_receiptdate::varchar as l_receiptdate FROM lineitem +WITH ( + ${RISEDEV_KAFKA_WITH_OPTIONS_COMMON}, + topic = 'tpch-lineitem' +) FORMAT PLAIN ENCODE JSON ( + force_append_only='true' +); + +statement ok +create sink kafka_nation FROM nation +WITH ( + ${RISEDEV_KAFKA_WITH_OPTIONS_COMMON}, + topic = 'tpch-nation' +) FORMAT PLAIN ENCODE JSON ( + force_append_only='true' +); + +statement ok +create sink kafka_region FROM region +WITH ( + ${RISEDEV_KAFKA_WITH_OPTIONS_COMMON}, + topic = 'tpch-region' +) FORMAT PLAIN ENCODE JSON ( + force_append_only='true' +); + +sleep 5s + +statement ok +DROP SINK kafka_supplier; + +statement ok +DROP SINK kafka_part; + +statement ok +DROP SINK kafka_partsupp; + +statement ok +DROP SINK kafka_customer; + +statement ok +DROP SINK kafka_orders; + +statement ok +DROP SINK kafka_lineitem; + +statement ok +DROP SINK kafka_nation; + +statement ok +DROP SINK kafka_region; + +include ./drop_tables.slt.part diff --git a/src/frontend/planner_test/tests/testdata/input/nexmark_source_kafka.yaml b/src/frontend/planner_test/tests/testdata/input/nexmark_source_kafka.yaml new file mode 100644 index 0000000000000..7087bff1b785b --- /dev/null +++ b/src/frontend/planner_test/tests/testdata/input/nexmark_source_kafka.yaml @@ -0,0 +1,718 @@ +- id: create_sources + sql: | + create source auction ( + id BIGINT, + item_name VARCHAR, + description VARCHAR, + initial_bid BIGINT, + reserve BIGINT, + date_time TIMESTAMP, + expires TIMESTAMP, + seller BIGINT, + category BIGINT, + extra VARCHAR) + with ( + connector = 'kafka', + properties.bootstrap.server = 'fake', + topic = 'auction' + ) FORMAT PLAIN ENCODE JSON; + + create source bid ( + auction BIGINT, + bidder BIGINT, + price BIGINT, + channel VARCHAR, + url VARCHAR, + date_time TIMESTAMP, + extra VARCHAR) + with ( + connector = 'kafka', + properties.bootstrap.server = 'fake', + topic = 'bid' + ) FORMAT PLAIN ENCODE JSON; + + create source person ( + id BIGINT, + name VARCHAR, + email_address VARCHAR, + credit_card VARCHAR, + city VARCHAR, + state VARCHAR, + date_time TIMESTAMP, + extra VARCHAR) + with ( + connector = 'kafka', + properties.bootstrap.server = 'fake', + topic = 'person' + ) FORMAT PLAIN ENCODE JSON; + expected_outputs: [] +- id: nexmark_q0 + before: + - create_sources + sql: | + SELECT auction, bidder, price, date_time FROM bid; + expected_outputs: + - stream_dist_plan + - batch_plan + - stream_plan +- id: nexmark_q1 + before: + - create_sources + sql: | + SELECT + auction, + bidder, + 0.908 * price as price, + date_time + FROM bid; + expected_outputs: + - batch_plan + - stream_plan + - stream_dist_plan +- id: nexmark_q2 + before: + - create_sources + sql: SELECT auction, price FROM bid WHERE auction = 1007 OR auction = 1020 OR auction = 2001 OR auction = 2019 OR auction = 2087; + expected_outputs: + - stream_dist_plan + - stream_plan + - batch_plan +- id: nexmark_q3 + before: + - create_sources + sql: | + SELECT + P.name, P.city, P.state, A.id + FROM + auction AS A INNER JOIN person AS P on A.seller = P.id + WHERE + A.category = 10 and (P.state = 'or' OR P.state = 'id' OR P.state = 'ca'); + expected_outputs: + - stream_plan + - batch_plan + - stream_dist_plan +- id: nexmark_q4 + before: + - create_sources + sql: | + SELECT + Q.category, + AVG(Q.final) as avg + FROM ( + SELECT MAX(B.price) AS final, A.category + FROM auction A, bid B + WHERE A.id = B.auction AND B.date_time BETWEEN A.date_time AND A.expires + GROUP BY A.id, A.category + ) Q + GROUP BY Q.category; + expected_outputs: + - batch_plan + - stream_plan + - stream_dist_plan +- id: nexmark_q5 + before: + - create_sources + sql: | + SELECT AuctionBids.auction, AuctionBids.num FROM ( + SELECT + bid.auction, + count(*) AS num, + window_start AS starttime + FROM + HOP(bid, date_time, INTERVAL '2' SECOND, INTERVAL '10' SECOND) + GROUP BY + window_start, + bid.auction + ) AS AuctionBids + JOIN ( + SELECT + max(CountBids.num) AS maxn, + CountBids.starttime_c + FROM ( + SELECT + count(*) AS num, + window_start AS starttime_c + FROM HOP(bid, date_time, INTERVAL '2' SECOND, INTERVAL '10' SECOND) + GROUP BY + bid.auction, + window_start + ) AS CountBids + GROUP BY + CountBids.starttime_c + ) AS MaxBids + ON AuctionBids.starttime = MaxBids.starttime_c AND AuctionBids.num >= MaxBids.maxn; + expected_outputs: + - logical_plan + - stream_plan + - stream_dist_plan + - batch_plan +- id: nexmark_q6 + before: + - create_sources + sql: | + SELECT + Q.seller, + AVG(Q.final) OVER + (PARTITION BY Q.seller ORDER BY Q.date_time ROWS BETWEEN 10 PRECEDING AND CURRENT ROW) + as avg + FROM ( + SELECT MAX(B.price) AS final, A.seller, B.date_time + FROM auction AS A, bid AS B + WHERE A.id = B.auction and B.date_time between A.date_time and A.expires + GROUP BY A.id, A.seller + ) AS Q; + expected_outputs: + - planner_error +- id: nexmark_q6_group_top1 + before: + - create_sources + sql: | + SELECT + Q.seller, + AVG(Q.final) OVER + (PARTITION BY Q.seller ORDER BY Q.date_time ROWS BETWEEN 10 PRECEDING AND CURRENT ROW) + as avg + FROM ( + SELECT ROW_NUMBER() OVER (PARTITION BY A.id, A.seller ORDER BY B.price) as rank, A.seller, B.price as final, B.date_time + FROM auction AS A, bid AS B + WHERE A.id = B.auction and B.date_time between A.date_time and A.expires + ) AS Q + WHERE Q.rank <= 1; + expected_outputs: + - stream_dist_plan + # - batch_plan + - stream_plan +- id: nexmark_q7 + before: + - create_sources + sql: | + SELECT + B.auction, + B.price, + B.bidder, + B.date_time + FROM + bid B + JOIN ( + SELECT + MAX(price) AS maxprice, + window_end as date_time + FROM + TUMBLE(bid, date_time, INTERVAL '10' SECOND) + GROUP BY + window_end + ) B1 ON B.price = B1.maxprice + WHERE + B.date_time BETWEEN B1.date_time - INTERVAL '10' SECOND + AND B1.date_time; + expected_outputs: + - stream_dist_plan + - batch_plan + - stream_plan +- id: nexmark_q8 + before: + - create_sources + sql: | + SELECT + P.id, + P.name, + P.starttime + FROM ( + SELECT + id, + name, + window_start AS starttime, + window_end AS endtime + FROM + TUMBLE(person, date_time, INTERVAL '10' SECOND) + GROUP BY + id, + name, + window_start, + window_end + ) P + JOIN ( + SELECT + seller, + window_start AS starttime, + window_end AS endtime + FROM + TUMBLE(auction, date_time, INTERVAL '10' SECOND) + GROUP BY + seller, + window_start, + window_end + ) A ON P.id = A.seller + AND P.starttime = A.starttime + AND P.endtime = A.endtime; + expected_outputs: + - batch_plan + - stream_plan + - stream_dist_plan +- id: nexmark_q9 + before: + - create_sources + sql: | + SELECT + id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, + auction, bidder, price, bid_date_time + FROM ( + SELECT A.*, B.auction, B.bidder, B.price, B.date_time AS bid_date_time, + ROW_NUMBER() OVER (PARTITION BY A.id ORDER BY B.price DESC, B.date_time ASC) AS rownum + FROM auction A, bid B + WHERE A.id = B.auction AND B.date_time BETWEEN A.date_time AND A.expires + ) + WHERE rownum <= 1; + expected_outputs: + - logical_plan + - stream_plan + - stream_dist_plan + - optimized_logical_plan_for_batch + - batch_plan +- id: nexmark_q10 + before: + - create_sources + sql: | + SELECT auction, bidder, price, date_time, TO_CHAR(date_time, 'YYYY-MM-DD') as date, TO_CHAR(date_time, 'HH:MI') as time FROM bid; + expected_outputs: + - stream_dist_plan + - batch_plan + - stream_plan +- id: nexmark_q11 + before: + - create_sources + sql: | + SELECT + B.bidder, + count(*) as bid_count, + SESSION_START(B.date_time, INTERVAL '10' SECOND) as starttime, + SESSION_END(B.date_time, INTERVAL '10' SECOND) as endtime + FROM bid B + GROUP BY B.bidder, SESSION(B.date_time, INTERVAL '10' SECOND); + expected_outputs: + - binder_error +- id: nexmark_q12 + before: + - create_sources + sql: | + SELECT + B.bidder, + count(*) as bid_count, + TUMBLE_START(B.p_time, INTERVAL '10' SECOND) as starttime, + TUMBLE_END(B.p_time, INTERVAL '10' SECOND) as endtime + FROM (SELECT *, PROCTIME() as p_time FROM bid) B + GROUP BY B.bidder, TUMBLE(B.p_time, INTERVAL '10' SECOND); + expected_outputs: + - binder_error +- id: nexmark_q13 + before: + - create_sources + sql: | + /* SELECT + B.auction, + B.bidder, + B.price, + B.date_time, + 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; */ + select 1; + expected_outputs: + - stream_error +- id: nexmark_q14 + before: + - create_sources + sql: | + SELECT + auction, + bidder, + 0.908 * price as price, + CASE + WHEN + extract(hour from date_time) >= 8 AND + extract(hour from date_time) <= 18 + THEN 'dayTime' + WHEN + extract(hour from date_time) <= 6 OR + extract(hour from date_time) >= 20 + THEN 'nightTime' + ELSE 'otherTime' + END AS bidTimeType, + date_time, + extra + -- ignore UDF in planner test + -- count_char(extra, 'c') AS c_counts + FROM bid + WHERE 0.908 * price > 1000000 AND 0.908 * price < 50000000; + expected_outputs: + - batch_plan + - stream_plan + - stream_dist_plan +- id: nexmark_q15 + before: + - create_sources + sql: | + SELECT + TO_CHAR(date_time, '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, + 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 to_char(date_time, 'yyyy-MM-dd'); + expected_outputs: + - stream_plan + - stream_dist_plan + - batch_plan +- id: nexmark_q15_split_distinct_agg + before: + - create_sources + sql: | + SELECT + TO_CHAR(date_time, '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, + 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 to_char(date_time, 'yyyy-MM-dd'); + expected_outputs: + - stream_plan + - stream_dist_plan + with_config_map: + RW_FORCE_SPLIT_DISTINCT_AGG: 'true' +- id: nexmark_q15_split_distinct_agg_and_force_two_phase + before: + - create_sources + sql: | + SELECT + TO_CHAR(date_time, '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, + 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 to_char(date_time, 'yyyy-MM-dd'); + expected_outputs: + - stream_plan + - stream_dist_plan + with_config_map: + RW_FORCE_SPLIT_DISTINCT_AGG: 'true' + RW_FORCE_TWO_PHASE_AGG: 'true' +- id: nexmark_q16 + before: + - create_sources + sql: | + SELECT + channel, + to_char(date_time, 'yyyy-MM-dd') AS day, + max(to_char(date_time, '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(date_time, 'yyyy-MM-dd'); + expected_outputs: + - stream_plan + - stream_dist_plan + - batch_plan +- id: nexmark_q16 + before: + - create_sources + sql: | + SELECT + channel, + to_char(date_time, 'yyyy-MM-dd') AS day, + max(to_char(date_time, '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(date_time, 'yyyy-MM-dd'); + expected_outputs: + - stream_plan + - stream_dist_plan + with_config_map: + RW_FORCE_SPLIT_DISTINCT_AGG: 'true' +- id: nexmark_q17 + before: + - create_sources + sql: | + SELECT + auction, + to_char(date_time, '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, + count(*) filter (where price >= 1000000) AS rank3_bids, + min(price) AS min_price, + max(price) AS max_price, + avg(price) AS avg_price, + sum(price) AS sum_price + FROM bid + GROUP BY auction, to_char(date_time, 'YYYY-MM-DD'); + expected_outputs: + - stream_plan + - stream_dist_plan + - batch_plan +- id: nexmark_q18 + before: + - create_sources + sql: | + SELECT auction, bidder, price, channel, url, date_time, extra + FROM (SELECT *, ROW_NUMBER() OVER (PARTITION BY bidder, auction ORDER BY date_time DESC) AS rank_number + FROM bid) + WHERE rank_number <= 1; + expected_outputs: + - stream_dist_plan + - logical_plan + - batch_plan + - stream_plan +- id: nexmark_q18_rank + before: + - create_sources + sql: | + SELECT auction, bidder, price, channel, url, date_time, extra + FROM (SELECT *, RANK() OVER (PARTITION BY bidder, auction ORDER BY date_time DESC) AS rank_number + FROM bid) + WHERE rank_number <= 1; + expected_outputs: + - stream_dist_plan + - batch_plan + - stream_plan + - logical_plan +- id: nexmark_q19 + before: + - create_sources + sql: | + SELECT * FROM + (SELECT *, ROW_NUMBER() OVER (PARTITION BY auction ORDER BY price DESC) AS rank_number FROM bid) + WHERE rank_number <= 10; + expected_outputs: + - logical_plan + - stream_plan + - stream_dist_plan + - batch_plan +- id: nexmark_q20 + before: + - create_sources + sql: | + SELECT + auction, bidder, price, channel, url, B.date_time as date_timeB, + item_name, description, initial_bid, reserve, A.date_time as date_timeA, expires, seller, category + FROM + bid B INNER JOIN auction A on B.auction = A.id + WHERE A.category = 10; + expected_outputs: + - batch_plan + - stream_dist_plan + - stream_plan +- id: nexmark_q21 + before: + - create_sources + sql: | + SELECT + auction, bidder, price, channel, + CASE + WHEN lower(channel) = 'apple' THEN '0' + WHEN lower(channel) = 'google' THEN '1' + WHEN lower(channel) = 'facebook' THEN '2' + WHEN lower(channel) = 'baidu' THEN '3' + ELSE REGEXP_MATCH(url, '(&|^)channel_id=([^&]*)')[2] + END + AS channel_id FROM bid + where REGEXP_MATCH(url, '(&|^)channel_id=([^&]*)')[2] is not null or + lower(channel) in ('apple', 'google', 'facebook', 'baidu'); + expected_outputs: + - batch_plan + - stream_dist_plan + - stream_plan +- id: nexmark_q22 + before: + - create_sources + sql: | + SELECT + auction, bidder, price, channel, + SPLIT_PART(url, '/', 4) as dir1, + SPLIT_PART(url, '/', 5) as dir2, + SPLIT_PART(url, '/', 6) as dir3 FROM bid; + expected_outputs: + - batch_plan + - stream_plan + - stream_dist_plan +- id: nexmark_q101 + before: + - create_sources + sql: | + -- A self-made query that covers outer join. + -- + -- Monitor ongoing auctions and track the current highest bid for each one in real-time. If + -- the auction has no bids, the highest bid will be NULL. + SELECT + a.id AS auction_id, + a.item_name AS auction_item_name, + b.max_price AS current_highest_bid + FROM auction a + LEFT OUTER JOIN ( + SELECT + b1.auction, + MAX(b1.price) max_price + FROM bid b1 + GROUP BY b1.auction + ) b ON a.id = b.auction; + expected_outputs: + - stream_dist_plan + - batch_plan + - stream_plan +- id: nexmark_q102 + before: + - create_sources + sql: | + -- A self-made query that covers dynamic filter. + -- + -- Show the auctions whose count of bids is greater than the overall average count of bids + -- per auction. + SELECT + a.id AS auction_id, + a.item_name AS auction_item_name, + COUNT(b.auction) AS bid_count + FROM auction a + JOIN bid b ON a.id = b.auction + GROUP BY a.id, a.item_name + HAVING COUNT(b.auction) >= ( + SELECT COUNT(*) / COUNT(DISTINCT auction) FROM bid + ) + expected_outputs: + - stream_plan + - stream_dist_plan + - batch_plan +- id: nexmark_q103 + before: + - create_sources + sql: | + -- A self-made query that covers semi join. + -- + -- Show the auctions that have at least 20 bids. + SELECT + a.id AS auction_id, + a.item_name AS auction_item_name + FROM auction a + WHERE a.id IN ( + SELECT b.auction FROM bid b + GROUP BY b.auction + HAVING COUNT(*) >= 20 + ); + expected_outputs: + - stream_dist_plan + - stream_plan + - batch_plan +- id: nexmark_q104 + before: + - create_sources + sql: | + -- A self-made query that covers anti join. + -- + -- This is the same as q103, which shows the auctions that have at least 20 bids. + SELECT + a.id AS auction_id, + a.item_name AS auction_item_name + FROM auction a + WHERE a.id NOT IN ( + SELECT b.auction FROM bid b + GROUP BY b.auction + HAVING COUNT(*) < 20 + ); + expected_outputs: + - batch_plan + - stream_plan + - stream_dist_plan +- id: nexmark_q105 + before: + - create_sources + sql: | + -- A self-made query that covers singleton top-n (and local-phase group top-n). + -- + -- Show the top 1000 auctions by the number of bids. + SELECT + a.id AS auction_id, + a.item_name AS auction_item_name, + COUNT(b.auction) AS bid_count + FROM auction a + JOIN bid b ON a.id = b.auction + GROUP BY a.id, a.item_name + ORDER BY bid_count DESC + LIMIT 1000; + expected_outputs: + - batch_plan + - stream_dist_plan + - stream_plan +- id: nexmark_q106 + before: + - create_sources + sql: | + -- A self-made query that covers two-phase stateful simple aggregation. + -- + -- Show the minimum final price of all auctions. + SELECT + MIN(final) AS min_final + FROM + ( + SELECT + auction.id, + MAX(price) AS final + FROM + auction, + bid + WHERE + bid.auction = auction.id + AND bid.date_time BETWEEN auction.date_time AND auction.expires + GROUP BY + auction.id + ) + expected_outputs: + - batch_plan + - stream_plan + - stream_dist_plan diff --git a/src/frontend/planner_test/tests/testdata/input/tpch_kafka.yaml b/src/frontend/planner_test/tests/testdata/input/tpch_kafka.yaml new file mode 100644 index 0000000000000..6372549b17573 --- /dev/null +++ b/src/frontend/planner_test/tests/testdata/input/tpch_kafka.yaml @@ -0,0 +1,902 @@ +- id: create_tables + sql: | + CREATE SOURCE supplier ( + s_suppkey INTEGER, + s_name VARCHAR, + s_address VARCHAR, + s_nationkey INTEGER, + s_phone VARCHAR, + s_acctbal NUMERIC, + s_comment VARCHAR + ) WITH ( + connector = 'kafka', + topic = 'supplier', + properties.bootstrap.server = 'fake', + ) FORMAT PLAIN ENCODE JSON; + + CREATE SOURCE part ( + p_partkey INTEGER, + p_name VARCHAR, + p_mfgr VARCHAR, + p_brand VARCHAR, + p_type VARCHAR, + p_size INTEGER, + p_container VARCHAR, + p_retailprice NUMERIC, + p_comment VARCHAR + ) WITH ( + connector = 'kafka', + topic = 'part', + properties.bootstrap.server = 'fake', + ) FORMAT PLAIN ENCODE JSON; + + CREATE SOURCE partsupp ( + ps_partkey INTEGER, + ps_suppkey INTEGER, + ps_availqty INTEGER, + ps_supplycost NUMERIC, + ps_comment VARCHAR + ) WITH ( + connector = 'kafka', + topic = 'partsupp', + properties.bootstrap.server = 'fake', + ) FORMAT PLAIN ENCODE JSON; + + CREATE SOURCE customer ( + c_custkey INTEGER, + c_name VARCHAR, + c_address VARCHAR, + c_nationkey INTEGER, + c_phone VARCHAR, + c_acctbal NUMERIC, + c_mktsegment VARCHAR, + c_comment VARCHAR + ) WITH ( + connector = 'kafka', + topic = 'customer', + properties.bootstrap.server = 'fake', + ) FORMAT PLAIN ENCODE JSON; + + CREATE SOURCE orders ( + o_orderkey BIGINT, + o_custkey INTEGER, + o_orderstatus VARCHAR, + o_totalprice NUMERIC, + o_orderdate DATE, + o_orderpriority VARCHAR, + o_clerk VARCHAR, + o_shippriority INTEGER, + o_comment VARCHAR + ) WITH ( + connector = 'kafka', + topic = 'orders', + properties.bootstrap.server = 'fake', + ) FORMAT PLAIN ENCODE JSON; + + CREATE SOURCE lineitem ( + l_orderkey BIGINT, + l_partkey INTEGER, + l_suppkey INTEGER, + l_linenumber INTEGER, + l_quantity NUMERIC, + l_extendedprice NUMERIC, + l_discount NUMERIC, + l_tax NUMERIC, + l_returnflag VARCHAR, + l_linestatus VARCHAR, + l_shipdate DATE, + l_commitdate DATE, + l_receiptdate DATE, + l_shipinstruct VARCHAR, + l_shipmode VARCHAR, + l_comment VARCHAR + ) WITH ( + connector = 'kafka', + topic = 'lineitem', + properties.bootstrap.server = 'fake', + ) FORMAT PLAIN ENCODE JSON; + + CREATE SOURCE nation ( + n_nationkey INTEGER, + n_name VARCHAR, + n_regionkey INTEGER, + n_comment VARCHAR + ) WITH ( + connector = 'kafka', + topic = 'nation', + properties.bootstrap.server = 'fake', + ) FORMAT PLAIN ENCODE JSON; + + CREATE SOURCE region ( + r_regionkey INTEGER, + r_name VARCHAR, + r_comment VARCHAR + ) WITH ( + connector = 'kafka', + topic = 'region', + properties.bootstrap.server = 'fake', + ) FORMAT PLAIN ENCODE JSON; + expected_outputs: [] +- id: tpch_q1 + before: + - create_tables + sql: | + select + l_returnflag, + l_linestatus, + sum(l_quantity) as sum_qty, + sum(l_extendedprice) as sum_base_price, + sum(l_extendedprice * (1 - l_discount)) as sum_disc_price, + sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) as sum_charge, + avg(l_quantity) as avg_qty, + avg(l_extendedprice) as avg_price, + avg(l_discount) as avg_disc, + count(*) as count_order + from + lineitem + where + l_shipdate <= date '1998-12-01' - interval '71' day + group by + l_returnflag, + l_linestatus + order by + l_returnflag, + l_linestatus + LIMIT 1; + expected_outputs: + - stream_plan +- id: tpch_q2 + before: + - create_tables + sql: | + select + s_acctbal, + s_name, + n_name, + p_partkey, + p_mfgr, + s_address, + s_phone, + s_comment + from + part, + supplier, + partsupp, + nation, + region + where + p_partkey = ps_partkey + and s_suppkey = ps_suppkey + and p_size = 4 + and p_type like '%TIN' + and s_nationkey = n_nationkey + and n_regionkey = r_regionkey + and r_name = 'AFRICA' + and ps_supplycost = ( + select + min(ps_supplycost) + from + partsupp, + supplier, + nation, + region + where + p_partkey = ps_partkey + and s_suppkey = ps_suppkey + and s_nationkey = n_nationkey + and n_regionkey = r_regionkey + and r_name = 'AFRICA' + ) + order by + s_acctbal desc, + n_name, + s_name, + p_partkey + LIMIT 100; + expected_outputs: + - stream_plan +- id: tpch_q3 + before: + - create_tables + sql: | + select + l_orderkey, + sum(l_extendedprice * (1 - l_discount)) as revenue, + o_orderdate, + o_shippriority + from + customer, + orders, + lineitem + where + c_mktsegment = 'FURNITURE' + and c_custkey = o_custkey + and l_orderkey = o_orderkey + and o_orderdate < date '1995-03-29' + and l_shipdate > date '1995-03-29' + group by + l_orderkey, + o_orderdate, + o_shippriority + order by + revenue desc, + o_orderdate + LIMIT 10; + expected_outputs: + - stream_plan +- id: tpch_q4 + before: + - create_tables + sql: | + select + o_orderpriority, + count(*) as order_count + from + orders + where + o_orderdate >= date '1997-07-01' + and o_orderdate < date '1997-07-01' + interval '3' month + and exists ( + select + * + from + lineitem + where + l_orderkey = o_orderkey + and l_commitdate < l_receiptdate + ) + group by + o_orderpriority + order by + o_orderpriority + LIMIT 1; + expected_outputs: + - stream_plan +- id: tpch_q5 + before: + - create_tables + sql: | + select + n_name, + sum(l_extendedprice * (1 - l_discount)) as revenue + from + customer, + orders, + lineitem, + supplier, + nation, + region + where + c_custkey = o_custkey + and l_orderkey = o_orderkey + and l_suppkey = s_suppkey + and c_nationkey = s_nationkey + and s_nationkey = n_nationkey + and n_regionkey = r_regionkey + and r_name = 'MIDDLE EAST' + and o_orderdate >= date '1994-01-01' + and o_orderdate < date '1994-01-01' + interval '1' year + group by + n_name + order by + revenue desc + LIMIT 1; + expected_outputs: + - stream_plan +- id: tpch_q6 + before: + - create_tables + sql: | + select + sum(l_extendedprice * l_discount) as revenue + from + lineitem + where + l_shipdate >= date '1994-01-01' + and l_shipdate < date '1994-01-01' + interval '1' year + and l_discount between 0.08 - 0.01 and 0.08 + 0.01 + and l_quantity < 24; + expected_outputs: + - stream_plan +- id: tpch_q7 + before: + - create_tables + sql: | + select + supp_nation, + cust_nation, + l_year, + sum(volume) as revenue + from + ( + select + n1.n_name as supp_nation, + n2.n_name as cust_nation, + extract(year from l_shipdate) as l_year, + l_extendedprice * (1 - l_discount) as volume + from + supplier, + lineitem, + orders, + customer, + nation n1, + nation n2 + where + s_suppkey = l_suppkey + and o_orderkey = l_orderkey + and c_custkey = o_custkey + and s_nationkey = n1.n_nationkey + and c_nationkey = n2.n_nationkey + and ( + (n1.n_name = 'ROMANIA' and n2.n_name = 'IRAN') + or (n1.n_name = 'IRAN' and n2.n_name = 'ROMANIA') + ) + and l_shipdate between date '1983-01-01' and date '2000-12-31' + ) as shipping + group by + supp_nation, + cust_nation, + l_year + order by + supp_nation, + cust_nation, + l_year + LIMIT 1; + expected_outputs: + - stream_plan +- id: tpch_q8 + before: + - create_tables + sql: | + select + o_year, + sum(case + when nation = 'IRAN' then volume + else 0 + end) / sum(volume) as mkt_share + from + ( + select + extract(year from o_orderdate) as o_year, + l_extendedprice * (1 - l_discount) as volume, + n2.n_name as nation + from + part, + supplier, + lineitem, + orders, + customer, + nation n1, + nation n2, + region + where + p_partkey = l_partkey + and s_suppkey = l_suppkey + and l_orderkey = o_orderkey + and o_custkey = c_custkey + and c_nationkey = n1.n_nationkey + and n1.n_regionkey = r_regionkey + and r_name = 'ASIA' + and s_nationkey = n2.n_nationkey + and o_orderdate between date '1995-01-01' and date '1996-12-31' + and p_type = 'PROMO ANODIZED STEEL' + ) as all_nations + group by + o_year + order by + o_year + LIMIT 1; + expected_outputs: + - stream_plan +- id: tpch_q9 + before: + - create_tables + sql: | + select + nation, + o_year, + sum(amount) as sum_profit + from + ( + select + n_name as nation, + extract(year from o_orderdate) as o_year, + l_extendedprice * (1 - l_discount) - ps_supplycost * l_quantity as amount + from + part, + supplier, + lineitem, + partsupp, + orders, + nation + where + s_suppkey = l_suppkey + and ps_suppkey = l_suppkey + and ps_partkey = l_partkey + and p_partkey = l_partkey + and o_orderkey = l_orderkey + and s_nationkey = n_nationkey + and p_name like '%yellow%' + ) as profit + group by + nation, + o_year + order by + nation, + o_year desc + LIMIT 1; + expected_outputs: + - stream_plan +- id: tpch_q10 + before: + - create_tables + sql: | + select + c_custkey, + c_name, + sum(l_extendedprice * (1.00 - l_discount)) as revenue, + c_acctbal, + n_name, + c_address, + c_phone, + c_comment + from + customer, + orders, + lineitem, + nation + where + c_custkey = o_custkey + and l_orderkey = o_orderkey + and o_orderdate >= date '1994-01-01' + and o_orderdate < date '1994-01-01' + interval '3' month + and l_returnflag = 'R' + and c_nationkey = n_nationkey + group by + c_custkey, + c_name, + c_acctbal, + c_phone, + n_name, + c_address, + c_comment + order by + revenue desc + LIMIT 20; + expected_outputs: + - stream_plan +- id: tpch_q11 + before: + - create_tables + sql: | + select + ps_partkey, + sum(ps_supplycost * ps_availqty) as value + from + partsupp, + supplier, + nation + where + ps_suppkey = s_suppkey + and s_nationkey = n_nationkey + and n_name = 'ARGENTINA' + group by + ps_partkey + having + sum(ps_supplycost * ps_availqty) > ( + select + sum(ps_supplycost * ps_availqty) * 0.0001000000 + from + partsupp, + supplier, + nation + where + ps_suppkey = s_suppkey + and s_nationkey = n_nationkey + and n_name = 'ARGENTINA' + ) + order by + value desc + LIMIT 1; + expected_outputs: + - stream_plan +- id: tpch_q12 + before: + - create_tables + sql: | + select + l_shipmode, + sum(case + when o_orderpriority = '1-URGENT' + or o_orderpriority = '2-HIGH' + then 1 + else 0 + end) as high_line_count, + sum(case + when o_orderpriority <> '1-URGENT' + and o_orderpriority <> '2-HIGH' + then 1 + else 0 + end) as low_line_count + from + orders, + lineitem + where + o_orderkey = l_orderkey + and l_shipmode in ('FOB', 'SHIP') + and l_commitdate < l_receiptdate + and l_shipdate < l_commitdate + and l_receiptdate >= date '1994-01-01' + and l_receiptdate < date '1994-01-01' + interval '1' year + group by + l_shipmode + order by + l_shipmode + LIMIT 1; + expected_outputs: + - stream_plan +- id: tpch_q13 + before: + - create_tables + sql: | + select + c_count, + count(*) as custdist + from + ( + select + c_custkey, + count(o_orderkey) as c_count + from + customer left outer join orders on + c_custkey = o_custkey + and o_comment not like '%:1%:2%' + group by + c_custkey + ) as c_orders (c_custkey, c_count) + group by + c_count + order by + custdist desc, + c_count desc + LIMIT 1; + expected_outputs: + - stream_plan +- id: tpch_q14 + before: + - create_tables + sql: | + select + 100.00 * sum(case + when p_type like 'PROMO%' + then l_extendedprice * (1 - l_discount) + else 0 + end) / sum(l_extendedprice * (1 - l_discount)) as promo_revenue + from + lineitem, + part + where + l_partkey = p_partkey + and l_shipdate >= date '1995-09-01' + and l_shipdate < date '1995-09-01' + interval '1' month; + expected_outputs: + - stream_plan +- id: tpch_q15 + before: + - create_tables + sql: | + with revenue0 (supplier_no, total_revenue) as ( + select + l_suppkey, + sum(l_extendedprice * (1 - l_discount)) + from + lineitem + where + l_shipdate >= date '1993-01-01' + and l_shipdate < date '1993-01-01' + interval '3' month + group by + l_suppkey + ) + select + s_suppkey, + s_name, + s_address, + s_phone, + total_revenue + from + supplier, + revenue0 + where + s_suppkey = supplier_no + and total_revenue = ( + select + max(total_revenue) + from + revenue0 + ) + order by + s_suppkey + LIMIT 1; + expected_outputs: + - stream_plan +- id: tpch_q16 + before: + - create_tables + sql: | + select + p_brand, + p_type, + p_size, + count(distinct ps_suppkey) as supplier_cnt + from + partsupp, + part + where + p_partkey = ps_partkey + and p_brand <> 'Brand#45' + and p_type not like 'SMALL PLATED%' + and p_size in (19, 17, 16, 23, 10, 4, 38, 11) + and ps_suppkey not in ( + select + s_suppkey + from + supplier + where + s_comment like '%Customer%Complaints%' + ) + group by + p_brand, + p_type, + p_size + order by + supplier_cnt desc, + p_brand, + p_type, + p_size + LIMIT 1; + expected_outputs: + - stream_plan +- id: tpch_q17 + before: + - create_tables + sql: | + select + sum(l_extendedprice) / 7.0 as avg_yearly + from + lineitem, + part + where + p_partkey = l_partkey + and p_brand = 'Brand#13' + and p_container = 'JUMBO PKG' + and l_quantity < ( + select + 0.2 * avg(l_quantity) + from + lineitem + where + l_partkey = p_partkey + ); + expected_outputs: + - stream_plan +- id: tpch_q18 + before: + - create_tables + sql: | + select + c_name, + c_custkey, + o_orderkey, + o_orderdate, + o_totalprice, + sum(l_quantity) quantity + from + customer, + orders, + lineitem + where + o_orderkey in ( + select + l_orderkey + from + lineitem + group by + l_orderkey + having + sum(l_quantity) > 1 + ) + and c_custkey = o_custkey + and o_orderkey = l_orderkey + group by + c_name, + c_custkey, + o_orderkey, + o_orderdate, + o_totalprice + order by + o_totalprice desc, + o_orderdate + LIMIT 100; + expected_outputs: + - stream_plan +- id: tpch_q19 + before: + - create_tables + sql: | + select + sum(l_extendedprice* (1 - l_discount)) as revenue + from + lineitem, + part + where + ( + p_partkey = l_partkey + and p_brand = 'Brand#52' + and p_container in ('SM CASE', 'SM BOX', 'SM PACK', 'SM PKG') + and l_quantity >= 1 and l_quantity <= 11 + and p_size between 1 and 5 + and l_shipmode in ('AIR', 'AIR REG') + and l_shipinstruct = 'DELIVER IN PERSON' + ) + or + ( + p_partkey = l_partkey + and p_brand = 'Brand#24' + and p_container in ('MED BAG', 'MED BOX', 'MED PKG', 'MED PACK') + and l_quantity >= 30 and l_quantity <= 40 + and p_size between 1 and 10 + and l_shipmode in ('AIR', 'AIR REG') + and l_shipinstruct = 'DELIVER IN PERSON' + ) + or + ( + p_partkey = l_partkey + and p_brand = 'Brand#32' + and p_container in ('LG CASE', 'LG BOX', 'LG PACK', 'LG PKG') + and l_quantity >= 10 and l_quantity <= 20 + and p_size between 1 and 15 + and l_shipmode in ('AIR', 'AIR REG') + and l_shipinstruct = 'DELIVER IN PERSON' + ); + expected_outputs: + - stream_plan +- id: tpch_q20 + before: + - create_tables + sql: | + select + s_name, + s_address + from + supplier, + nation + where + s_suppkey in ( + select + ps_suppkey + from + partsupp, + ( + select + l_partkey agg_partkey, + l_suppkey agg_suppkey, + 0.5 * sum(l_quantity) AS agg_quantity + from + lineitem + where + l_shipdate >= date '1994-01-01' + and l_shipdate < date '1994-01-01' + interval '1' year + group by + l_partkey, + l_suppkey + ) agg_lineitem + where + agg_partkey = ps_partkey + and agg_suppkey = ps_suppkey + and ps_partkey in ( + select + p_partkey + from + part + where + p_name like 'forest%' + ) + and ps_availqty > agg_quantity + ) + and s_nationkey = n_nationkey + and n_name = 'KENYA' + order by + s_name + LIMIT 1; + expected_outputs: + - stream_plan +- id: tpch_q21 + before: + - create_tables + sql: | + select + s_name, + count(*) as numwait + from + supplier, + lineitem l1, + orders, + nation + where + s_suppkey = l1.l_suppkey + and o_orderkey = l1.l_orderkey + and o_orderstatus = 'F' + and l1.l_receiptdate > l1.l_commitdate + and exists ( + select + * + from + lineitem l2 + where + l2.l_orderkey = l1.l_orderkey + and l2.l_suppkey <> l1.l_suppkey + ) + and not exists ( + select + * + from + lineitem l3 + where + l3.l_orderkey = l1.l_orderkey + and l3.l_suppkey <> l1.l_suppkey + and l3.l_receiptdate > l3.l_commitdate + ) + and s_nationkey = n_nationkey + and n_name = 'GERMANY' + group by + s_name + order by + numwait desc, + s_name + LIMIT 100; + expected_outputs: + - stream_plan +- id: tpch_q22 + before: + - create_tables + sql: | + select + cntrycode, + count(*) as numcust, + sum(c_acctbal) as totacctbal + from + ( + select + substring(c_phone from 1 for 2) as cntrycode, + c_acctbal + from + customer + where + substring(c_phone from 1 for 2) in + ('30', '24', '31', '38', '25', '34', '37') + and c_acctbal > ( + select + avg(c_acctbal) + from + customer + where + c_acctbal > 0.00::numeric + and substring(c_phone from 1 for 2) in + ('30', '24', '31', '38', '25', '34', '37') + ) + and not exists ( + select + * + from + orders + where + o_custkey = c_custkey + ) + ) as custsale + group by + cntrycode + order by + cntrycode + LIMIT 1; + expected_outputs: + - stream_plan 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 63daa9c43b2db..2b7f23dff932e 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] } + └─LogicalSource { source: s, columns: [id, value, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchProject { exprs: [id, value] } - └─BatchKafkaScan { source: s, columns: [id, value, _rw_kafka_timestamp, _row_id], filter: (None, None) } + └─BatchKafkaScan { source: s, columns: [id, value, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id], filter: (None, None) } create_source: format: plain encode: protobuf diff --git a/src/frontend/planner_test/tests/testdata/output/nexmark_source_kafka.yaml b/src/frontend/planner_test/tests/testdata/output/nexmark_source_kafka.yaml new file mode 100644 index 0000000000000..469d94ca87ff3 --- /dev/null +++ b/src/frontend/planner_test/tests/testdata/output/nexmark_source_kafka.yaml @@ -0,0 +1,2470 @@ +# This file is automatically generated. See `src/frontend/planner_test/README.md` for more information. +- id: create_sources + sql: | + create source auction ( + id BIGINT, + item_name VARCHAR, + description VARCHAR, + initial_bid BIGINT, + reserve BIGINT, + date_time TIMESTAMP, + expires TIMESTAMP, + seller BIGINT, + category BIGINT, + extra VARCHAR) + with ( + connector = 'kafka', + properties.bootstrap.server = 'fake', + topic = 'auction' + ) FORMAT PLAIN ENCODE JSON; + + create source bid ( + auction BIGINT, + bidder BIGINT, + price BIGINT, + channel VARCHAR, + url VARCHAR, + date_time TIMESTAMP, + extra VARCHAR) + with ( + connector = 'kafka', + properties.bootstrap.server = 'fake', + topic = 'bid' + ) FORMAT PLAIN ENCODE JSON; + + create source person ( + id BIGINT, + name VARCHAR, + email_address VARCHAR, + credit_card VARCHAR, + city VARCHAR, + state VARCHAR, + date_time TIMESTAMP, + extra VARCHAR) + with ( + connector = 'kafka', + properties.bootstrap.server = 'fake', + topic = 'person' + ) FORMAT PLAIN ENCODE JSON; +- id: nexmark_q0 + before: + - create_sources + sql: | + SELECT auction, bidder, price, date_time FROM bid; + batch_plan: |- + BatchExchange { order: [], dist: Single } + └─BatchProject { exprs: [auction, bidder, price, date_time] } + └─BatchKafkaScan { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id], filter: (None, None) } + 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] } + └─StreamRowIdGen { row_id_index: 10 } + └─StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + stream_dist_plan: |+ + Fragment 0 + StreamMaterialize { columns: [auction, bidder, price, date_time, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: NoCheck } + ├── tables: [ Materialize: 4294967294 ] + └── StreamProject { exprs: [auction, bidder, price, date_time, _row_id] } + └── StreamRowIdGen { row_id_index: 10 } + └── StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + ├── tables: [ SourceBackfill: 0 ] + └── Upstream + + Table 0 + ├── columns: [ partition_id, backfill_progress, _rw_timestamp ] + ├── primary key: [ $0 ASC ] + ├── value indices: [ 0, 1 ] + ├── distribution key: [] + └── read pk prefix len hint: 0 + + Table 4294967294 + ├── columns: [ auction, bidder, price, date_time, _row_id, _rw_timestamp ] + ├── primary key: [ $4 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4 ] + ├── distribution key: [ 4 ] + └── read pk prefix len hint: 1 + +- id: nexmark_q1 + before: + - create_sources + sql: | + SELECT + auction, + bidder, + 0.908 * price as price, + date_time + FROM bid; + batch_plan: |- + BatchExchange { order: [], dist: Single } + └─BatchProject { exprs: [auction, bidder, (0.908:Decimal * price::Decimal) as $expr1, date_time] } + └─BatchKafkaScan { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id], filter: (None, None) } + 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] } + └─StreamRowIdGen { row_id_index: 10 } + └─StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + stream_dist_plan: |+ + Fragment 0 + StreamMaterialize { columns: [auction, bidder, price, date_time, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: NoCheck } + ├── tables: [ Materialize: 4294967294 ] + └── StreamProject { exprs: [auction, bidder, (0.908:Decimal * price::Decimal) as $expr1, date_time, _row_id] } + └── StreamRowIdGen { row_id_index: 10 } + └── StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + ├── tables: [ SourceBackfill: 0 ] + └── Upstream + + Table 0 + ├── columns: [ partition_id, backfill_progress, _rw_timestamp ] + ├── primary key: [ $0 ASC ] + ├── value indices: [ 0, 1 ] + ├── distribution key: [] + └── read pk prefix len hint: 0 + + Table 4294967294 + ├── columns: [ auction, bidder, price, date_time, _row_id, _rw_timestamp ] + ├── primary key: [ $4 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4 ] + ├── distribution key: [ 4 ] + └── read pk prefix len hint: 1 + +- id: nexmark_q2 + before: + - create_sources + sql: SELECT auction, price FROM bid WHERE auction = 1007 OR auction = 1020 OR auction = 2001 OR auction = 2019 OR auction = 2087; + batch_plan: |- + 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] } + └─BatchKafkaScan { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id], filter: (None, None) } + 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] } + └─StreamFilter { predicate: ((((auction = 1007:Int32) OR (auction = 1020:Int32)) OR ((auction = 2001:Int32) OR (auction = 2019:Int32))) OR (auction = 2087:Int32)) } + └─StreamRowIdGen { row_id_index: 10 } + └─StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + stream_dist_plan: |+ + Fragment 0 + StreamMaterialize { columns: [auction, price, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: NoCheck } + ├── tables: [ Materialize: 4294967294 ] + └── StreamProject { exprs: [auction, price, _row_id] } + └── StreamFilter { predicate: ((((auction = 1007:Int32) OR (auction = 1020:Int32)) OR ((auction = 2001:Int32) OR (auction = 2019:Int32))) OR (auction = 2087:Int32)) } + └── StreamRowIdGen { row_id_index: 10 } + └── StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + ├── tables: [ SourceBackfill: 0 ] + └── Upstream + + Table 0 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + + Table 4294967294 + ├── columns: [ auction, price, _row_id, _rw_timestamp ] + ├── primary key: [ $2 ASC ] + ├── value indices: [ 0, 1, 2 ] + ├── distribution key: [ 2 ] + └── read pk prefix len hint: 1 + +- id: nexmark_q3 + before: + - create_sources + sql: | + SELECT + P.name, P.city, P.state, A.id + FROM + auction AS A INNER JOIN person AS P on A.seller = P.id + WHERE + A.category = 10 and (P.state = 'or' OR P.state = 'id' OR P.state = 'ca'); + batch_plan: |- + BatchExchange { order: [], dist: Single } + └─BatchHashJoin { type: Inner, predicate: seller = id, output: [name, city, state, id] } + ├─BatchExchange { order: [], dist: HashShard(seller) } + │ └─BatchFilter { predicate: (category = 10:Int32) } + │ └─BatchProject { exprs: [id, seller, category] } + │ └─BatchKafkaScan { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id], filter: (None, None) } + └─BatchExchange { order: [], dist: HashShard(id) } + └─BatchFilter { predicate: (((state = 'or':Varchar) OR (state = 'id':Varchar)) OR (state = 'ca':Varchar)) } + └─BatchProject { exprs: [id, name, city, state] } + └─BatchKafkaScan { source: person, columns: [id, name, email_address, credit_card, city, state, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id], filter: (None, None) } + 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) } + └─StreamHashJoin [append_only] { type: Inner, predicate: seller = id, output: [name, city, state, id, _row_id, seller, _row_id] } + ├─StreamExchange { dist: HashShard(seller) } + │ └─StreamFilter { predicate: (category = 10:Int32) } + │ └─StreamRowIdGen { row_id_index: 13 } + │ └─StreamSourceScan { columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(id) } + └─StreamFilter { predicate: (((state = 'or':Varchar) OR (state = 'id':Varchar)) OR (state = 'ca':Varchar)) } + └─StreamRowIdGen { row_id_index: 11 } + └─StreamSourceScan { columns: [id, name, email_address, credit_card, city, state, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + stream_dist_plan: |+ + Fragment 0 + 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 } + ├── tables: [ Materialize: 4294967294 ] + └── StreamExchange Hash([4, 5, 6]) from 1 + + Fragment 1 + StreamHashJoin [append_only] { type: Inner, predicate: seller = id, output: [name, city, state, id, _row_id, seller, _row_id] } + ├── tables: [ HashJoinLeft: 0, HashJoinDegreeLeft: 1, HashJoinRight: 2, HashJoinDegreeRight: 3 ] + ├── StreamExchange Hash([7]) from 2 + └── StreamExchange Hash([0]) from 3 + + Fragment 2 + StreamFilter { predicate: (category = 10:Int32) } + └── StreamRowIdGen { row_id_index: 13 } + └── StreamSourceScan { columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + ├── tables: [ SourceBackfill: 4 ] + └── Upstream + + Fragment 3 + StreamFilter { predicate: (((state = 'or':Varchar) OR (state = 'id':Varchar)) OR (state = 'ca':Varchar)) } + └── StreamRowIdGen { row_id_index: 11 } + └── StreamSourceScan { columns: [id, name, email_address, credit_card, city, state, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } { tables: [ SourceBackfill: 5 ] } + └── Upstream + + Table 0 + ├── columns: [ id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id, _rw_timestamp ] + ├── primary key: [ $7 ASC, $13 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13 ] + ├── distribution key: [ 7 ] + └── read pk prefix len hint: 1 + + Table 1 { columns: [ seller, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + + Table 2 + ├── columns: [ id, name, email_address, credit_card, city, state, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id, _rw_timestamp ] + ├── primary key: [ $0 ASC, $11 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11 ] + ├── distribution key: [ 0 ] + └── read pk prefix len hint: 1 + + Table 3 { columns: [ id, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + + Table 4 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + + Table 5 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + + Table 4294967294 + ├── columns: [ name, city, state, id, _row_id, seller, _row_id#1, _rw_timestamp ] + ├── primary key: [ $4 ASC, $6 ASC, $5 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4, 5, 6 ] + ├── distribution key: [ 4, 5, 6 ] + └── read pk prefix len hint: 3 + +- id: nexmark_q4 + before: + - create_sources + sql: | + SELECT + Q.category, + AVG(Q.final) as avg + FROM ( + SELECT MAX(B.price) AS final, A.category + FROM auction A, bid B + WHERE A.id = B.auction AND B.date_time BETWEEN A.date_time AND A.expires + GROUP BY A.id, A.category + ) Q + GROUP BY Q.category; + batch_plan: |- + BatchExchange { order: [], dist: Single } + └─BatchProject { exprs: [category, (sum(max(price)) / count(max(price))::Decimal) as $expr1] } + └─BatchHashAgg { group_key: [category], aggs: [sum(max(price)), count(max(price))] } + └─BatchExchange { order: [], dist: HashShard(category) } + └─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) } + │ └─BatchKafkaScan { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id], filter: (None, None) } + └─BatchExchange { order: [], dist: HashShard(auction) } + └─BatchKafkaScan { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id], filter: (None, None) } + 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] } + └─StreamHashAgg { group_key: [category], aggs: [sum(max(price)), count(max(price)), count] } + └─StreamExchange { dist: HashShard(category) } + └─StreamProject { exprs: [id, category, max(price)] } + └─StreamHashAgg [append_only] { group_key: [id, category], aggs: [max(price), count] } + └─StreamProject { exprs: [id, category, price, _row_id, _row_id] } + └─StreamFilter { predicate: (date_time >= date_time) AND (date_time <= expires) } + └─StreamHashJoin [append_only] { type: Inner, predicate: id = auction, output: all } + ├─StreamExchange { dist: HashShard(id) } + │ └─StreamRowIdGen { row_id_index: 13 } + │ └─StreamSourceScan { columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(auction) } + └─StreamRowIdGen { row_id_index: 10 } + └─StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + stream_dist_plan: |+ + Fragment 0 + StreamMaterialize { columns: [category, avg], stream_key: [category], pk_columns: [category], pk_conflict: NoCheck } + ├── tables: [ Materialize: 4294967294 ] + └── StreamProject { exprs: [category, (sum(max(price)) / count(max(price))::Decimal) as $expr1] } + └── StreamHashAgg { group_key: [category], aggs: [sum(max(price)), count(max(price)), count] } + ├── tables: [ HashAggState: 0 ] + └── StreamExchange Hash([1]) from 1 + + Fragment 1 + StreamProject { exprs: [id, category, max(price)] } + └── StreamHashAgg [append_only] { group_key: [id, category], aggs: [max(price), count] } + ├── tables: [ HashAggState: 1 ] + └── StreamProject { exprs: [id, category, price, _row_id, _row_id] } + └── StreamFilter { predicate: (date_time >= date_time) AND (date_time <= expires) } + └── StreamHashJoin [append_only] { type: Inner, predicate: id = auction, output: all } + ├── tables: [ HashJoinLeft: 2, HashJoinDegreeLeft: 3, HashJoinRight: 4, HashJoinDegreeRight: 5 ] + ├── StreamExchange Hash([0]) from 2 + └── StreamExchange Hash([0]) from 3 + + Fragment 2 + StreamRowIdGen { row_id_index: 13 } + └── StreamSourceScan { columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + ├── tables: [ SourceBackfill: 6 ] + └── Upstream + + Fragment 3 + StreamRowIdGen { row_id_index: 10 } + └── StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } { tables: [ SourceBackfill: 7 ] } + └── Upstream + + Table 0 { columns: [ category, sum(max(price)), count(max(price)), count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + + Table 1 { columns: [ id, category, max(price), count, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 2 } + + Table 2 + ├── columns: [ id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id, _rw_timestamp ] + ├── primary key: [ $0 ASC, $13 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13 ] + ├── distribution key: [ 0 ] + └── read pk prefix len hint: 1 + + Table 3 { columns: [ id, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + + Table 4 + ├── columns: [ auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id, _rw_timestamp ] + ├── primary key: [ $0 ASC, $10 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10 ] + ├── distribution key: [ 0 ] + └── read pk prefix len hint: 1 + + Table 5 { columns: [ auction, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + + Table 6 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + + Table 7 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + + Table 4294967294 { columns: [ category, avg, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + +- id: nexmark_q5 + before: + - create_sources + sql: | + SELECT AuctionBids.auction, AuctionBids.num FROM ( + SELECT + bid.auction, + count(*) AS num, + window_start AS starttime + FROM + HOP(bid, date_time, INTERVAL '2' SECOND, INTERVAL '10' SECOND) + GROUP BY + window_start, + bid.auction + ) AS AuctionBids + JOIN ( + SELECT + max(CountBids.num) AS maxn, + CountBids.starttime_c + FROM ( + SELECT + count(*) AS num, + window_start AS starttime_c + FROM HOP(bid, date_time, INTERVAL '2' SECOND, INTERVAL '10' SECOND) + GROUP BY + bid.auction, + window_start + ) AS CountBids + GROUP BY + CountBids.starttime_c + ) AS MaxBids + ON AuctionBids.starttime = MaxBids.starttime_c AND AuctionBids.num >= MaxBids.maxn; + logical_plan: |- + LogicalProject { exprs: [auction, count] } + └─LogicalJoin { type: Inner, on: (window_start = window_start) AND (count >= max(count)), output: all } + ├─LogicalProject { exprs: [auction, count, window_start] } + │ └─LogicalAgg { group_key: [window_start, auction], aggs: [count] } + │ └─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, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─LogicalProject { exprs: [max(count), window_start] } + └─LogicalAgg { group_key: [window_start], aggs: [max(count)] } + └─LogicalProject { exprs: [window_start, count] } + └─LogicalProject { exprs: [count, window_start] } + └─LogicalAgg { group_key: [auction, window_start], aggs: [count] } + └─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, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + batch_plan: |- + BatchExchange { order: [], dist: Single } + └─BatchHashJoin { type: Inner, predicate: window_start = window_start AND (count >= max(count)), output: [auction, count] } + ├─BatchExchange { order: [], dist: HashShard(window_start) } + │ └─BatchHashAgg { group_key: [auction, window_start], aggs: [count] } + │ └─BatchHopWindow { time_col: date_time, slide: 00:00:02, size: 00:00:10, output: [auction, window_start] } + │ └─BatchExchange { order: [], dist: HashShard(auction) } + │ └─BatchProject { exprs: [auction, date_time] } + │ └─BatchFilter { predicate: IsNotNull(date_time) } + │ └─BatchKafkaScan { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id], filter: (None, None) } + └─BatchHashAgg { group_key: [window_start], aggs: [max(count)] } + └─BatchExchange { order: [], dist: HashShard(window_start) } + └─BatchHashAgg { group_key: [auction, window_start], aggs: [count] } + └─BatchHopWindow { time_col: date_time, slide: 00:00:02, size: 00:00:10, output: [auction, window_start] } + └─BatchExchange { order: [], dist: HashShard(auction) } + └─BatchProject { exprs: [auction, date_time] } + └─BatchFilter { predicate: IsNotNull(date_time) } + └─BatchKafkaScan { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id], filter: (None, None) } + 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] } + └─StreamFilter { predicate: (count >= max(count)) } + └─StreamHashJoin { type: Inner, predicate: window_start = window_start, output: all } + ├─StreamExchange { dist: HashShard(window_start) } + │ └─StreamShare { id: 7 } + │ └─StreamHashAgg [append_only] { group_key: [auction, window_start], aggs: [count] } + │ └─StreamExchange { dist: HashShard(auction, window_start) } + │ └─StreamHopWindow { time_col: date_time, slide: 00:00:02, size: 00:00:10, output: [auction, window_start, _row_id] } + │ └─StreamProject { exprs: [auction, date_time, _row_id] } + │ └─StreamFilter { predicate: IsNotNull(date_time) } + │ └─StreamRowIdGen { row_id_index: 10 } + │ └─StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamProject { exprs: [window_start, max(count)] } + └─StreamHashAgg { group_key: [window_start], aggs: [max(count), count] } + └─StreamExchange { dist: HashShard(window_start) } + └─StreamShare { id: 7 } + └─StreamHashAgg [append_only] { group_key: [auction, window_start], aggs: [count] } + └─StreamExchange { dist: HashShard(auction, window_start) } + └─StreamHopWindow { time_col: date_time, slide: 00:00:02, size: 00:00:10, output: [auction, window_start, _row_id] } + └─StreamProject { exprs: [auction, date_time, _row_id] } + └─StreamFilter { predicate: IsNotNull(date_time) } + └─StreamRowIdGen { row_id_index: 10 } + └─StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + stream_dist_plan: |+ + Fragment 0 + StreamMaterialize { columns: [auction, num, window_start(hidden), window_start#1(hidden)], stream_key: [auction, window_start], pk_columns: [auction, window_start], pk_conflict: NoCheck } + ├── tables: [ Materialize: 4294967294 ] + └── StreamProject { exprs: [auction, count, window_start, window_start] } + └── StreamFilter { predicate: (count >= max(count)) } + └── StreamHashJoin { type: Inner, predicate: window_start = window_start, output: all } + ├── tables: [ HashJoinLeft: 0, HashJoinDegreeLeft: 1, HashJoinRight: 2, HashJoinDegreeRight: 3 ] + ├── StreamExchange Hash([1]) from 1 + └── StreamProject { exprs: [window_start, max(count)] } + └── StreamHashAgg { group_key: [window_start], aggs: [max(count), count] } { tables: [ HashAggState: 7, HashAggCall0: 6 ] } + └── StreamExchange Hash([1]) from 4 + + Fragment 1 + StreamNoOp + └── StreamExchange NoShuffle from 2 + + Fragment 2 + StreamHashAgg [append_only] { group_key: [auction, window_start], aggs: [count] } { tables: [ HashAggState: 4 ] } + └── StreamExchange Hash([0, 1]) from 3 + + Fragment 3 + StreamHopWindow { time_col: date_time, slide: 00:00:02, size: 00:00:10, output: [auction, window_start, _row_id] } + └── StreamProject { exprs: [auction, date_time, _row_id] } + └── StreamFilter { predicate: IsNotNull(date_time) } + └── StreamRowIdGen { row_id_index: 10 } + └── StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + ├── tables: [ SourceBackfill: 5 ] + └── Upstream + + Fragment 4 + StreamNoOp + └── StreamExchange NoShuffle from 2 + + Table 0 { columns: [ auction, window_start, count, _rw_timestamp ], primary key: [ $1 ASC, $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 1 ], read pk prefix len hint: 1 } + + Table 1 { columns: [ window_start, auction, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + + Table 2 { columns: [ window_start, max(count), _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + + Table 3 { columns: [ window_start, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + + Table 4 { columns: [ auction, window_start, count, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } + + Table 5 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + + Table 6 + ├── columns: [ window_start, count, auction, _rw_timestamp ] + ├── primary key: [ $0 ASC, $1 DESC, $2 ASC ] + ├── value indices: [ 0, 1, 2 ] + ├── distribution key: [ 0 ] + └── read pk prefix len hint: 1 + + Table 7 { columns: [ window_start, max(count), count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + + Table 4294967294 + ├── columns: [ auction, num, window_start, window_start#1, _rw_timestamp ] + ├── primary key: [ $0 ASC, $2 ASC ] + ├── value indices: [ 0, 1, 2, 3 ] + ├── distribution key: [ 2 ] + └── read pk prefix len hint: 2 + +- id: nexmark_q6 + before: + - create_sources + sql: | + SELECT + Q.seller, + AVG(Q.final) OVER + (PARTITION BY Q.seller ORDER BY Q.date_time ROWS BETWEEN 10 PRECEDING AND CURRENT ROW) + as avg + FROM ( + SELECT MAX(B.price) AS final, A.seller, B.date_time + FROM auction AS A, bid AS B + WHERE A.id = B.auction and B.date_time between A.date_time and A.expires + GROUP BY A.id, A.seller + ) AS Q; + planner_error: 'Invalid input syntax: column must appear in the GROUP BY clause or be used in an aggregate function' +- id: nexmark_q6_group_top1 + before: + - create_sources + sql: | + SELECT + Q.seller, + AVG(Q.final) OVER + (PARTITION BY Q.seller ORDER BY Q.date_time ROWS BETWEEN 10 PRECEDING AND CURRENT ROW) + as avg + FROM ( + SELECT ROW_NUMBER() OVER (PARTITION BY A.id, A.seller ORDER BY B.price) as rank, A.seller, B.price as final, B.date_time + FROM auction AS A, bid AS B + WHERE A.id = B.auction and B.date_time between A.date_time and A.expires + ) AS Q + WHERE Q.rank <= 1; + stream_plan: |- + StreamMaterialize { columns: [seller, avg, id(hidden)], stream_key: [id, seller], pk_columns: [id, seller], pk_conflict: NoCheck } + └─StreamProject { exprs: [seller, (sum / count::Decimal) as $expr1, id] } + └─StreamOverWindow { window_functions: [sum(price) OVER(PARTITION BY seller ORDER BY date_time ASC ROWS BETWEEN 10 PRECEDING AND CURRENT ROW), count(price) OVER(PARTITION BY seller ORDER BY date_time ASC ROWS BETWEEN 10 PRECEDING AND CURRENT ROW)] } + └─StreamExchange { dist: HashShard(seller) } + └─StreamProject { exprs: [seller, price, date_time, id] } + └─StreamGroupTopN [append_only] { order: [price ASC], limit: 1, offset: 0, group_key: [id, seller] } + └─StreamExchange { dist: HashShard(id, seller) } + └─StreamProject { exprs: [id, seller, price, date_time, _row_id, _row_id] } + └─StreamFilter { predicate: (date_time >= date_time) AND (date_time <= expires) } + └─StreamHashJoin [append_only] { type: Inner, predicate: id = auction, output: all } + ├─StreamExchange { dist: HashShard(id) } + │ └─StreamRowIdGen { row_id_index: 13 } + │ └─StreamSourceScan { columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(auction) } + └─StreamRowIdGen { row_id_index: 10 } + └─StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + stream_dist_plan: |+ + Fragment 0 + StreamMaterialize { columns: [seller, avg, id(hidden)], stream_key: [id, seller], pk_columns: [id, seller], pk_conflict: NoCheck } { tables: [ Materialize: 4294967294 ] } + └── StreamProject { exprs: [seller, (sum / count::Decimal) as $expr1, id] } + └── StreamOverWindow { window_functions: [sum(price) OVER(PARTITION BY seller ORDER BY date_time ASC ROWS BETWEEN 10 PRECEDING AND CURRENT ROW), count(price) OVER(PARTITION BY seller ORDER BY date_time ASC ROWS BETWEEN 10 PRECEDING AND CURRENT ROW)] } + ├── tables: [ OverWindow: 0 ] + └── StreamExchange Hash([0]) from 1 + + Fragment 1 + StreamProject { exprs: [seller, price, date_time, id] } + └── StreamGroupTopN [append_only] { order: [price ASC], limit: 1, offset: 0, group_key: [id, seller] } { tables: [ AppendOnlyGroupTopN: 1 ] } + └── StreamExchange Hash([0, 1]) from 2 + + Fragment 2 + StreamProject { exprs: [id, seller, price, date_time, _row_id, _row_id] } + └── StreamFilter { predicate: (date_time >= date_time) AND (date_time <= expires) } + └── StreamHashJoin [append_only] { type: Inner, predicate: id = auction, output: all } { tables: [ HashJoinLeft: 2, HashJoinDegreeLeft: 3, HashJoinRight: 4, HashJoinDegreeRight: 5 ] } + ├── StreamExchange Hash([0]) from 3 + └── StreamExchange Hash([0]) from 4 + + Fragment 3 + StreamRowIdGen { row_id_index: 13 } + └── StreamSourceScan { columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } { tables: [ SourceBackfill: 6 ] } + └── Upstream + + Fragment 4 + StreamRowIdGen { row_id_index: 10 } + └── StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } { tables: [ SourceBackfill: 7 ] } + └── Upstream + + Table 0 { columns: [ seller, price, date_time, id, sum, count, _rw_timestamp ], primary key: [ $0 ASC, $2 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + + Table 1 { columns: [ id, seller, price, date_time, _row_id, _row_id_0, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $4 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } + + Table 2 + ├── columns: [ id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id, _rw_timestamp ] + ├── primary key: [ $0 ASC, $13 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13 ] + ├── distribution key: [ 0 ] + └── read pk prefix len hint: 1 + + Table 3 { columns: [ id, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + + Table 4 + ├── columns: [ auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id, _rw_timestamp ] + ├── primary key: [ $0 ASC, $10 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10 ] + ├── distribution key: [ 0 ] + └── read pk prefix len hint: 1 + + Table 5 { columns: [ auction, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + + Table 6 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + + Table 7 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + + Table 4294967294 { columns: [ seller, avg, id, _rw_timestamp ], primary key: [ $2 ASC, $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 2 } + +- id: nexmark_q7 + before: + - create_sources + sql: | + SELECT + B.auction, + B.price, + B.bidder, + B.date_time + FROM + bid B + JOIN ( + SELECT + MAX(price) AS maxprice, + window_end as date_time + FROM + TUMBLE(bid, date_time, INTERVAL '10' SECOND) + GROUP BY + window_end + ) B1 ON B.price = B1.maxprice + WHERE + B.date_time BETWEEN B1.date_time - INTERVAL '10' SECOND + AND B1.date_time; + batch_plan: |- + 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) } + │ └─BatchKafkaScan { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id], filter: (None, None) } + └─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] } + └─BatchKafkaScan { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id], filter: (None, None) } + 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] } + └─StreamFilter { predicate: (date_time >= $expr2) AND (date_time <= $expr1) } + └─StreamHashJoin { type: Inner, predicate: price = max(price), output: all } + ├─StreamExchange { dist: HashShard(price) } + │ └─StreamShare { id: 3 } + │ └─StreamProject { exprs: [auction, bidder, price, date_time, _row_id] } + │ └─StreamRowIdGen { row_id_index: 10 } + │ └─StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(max(price)) } + └─StreamProject { exprs: [$expr1, max(price), ($expr1 - '00:00:10':Interval) as $expr2] } + └─StreamHashAgg [append_only] { group_key: [$expr1], aggs: [max(price), count] } + └─StreamExchange { dist: HashShard($expr1) } + └─StreamProject { exprs: [(TumbleStart(date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr1, price, _row_id] } + └─StreamShare { id: 3 } + └─StreamProject { exprs: [auction, bidder, price, date_time, _row_id] } + └─StreamRowIdGen { row_id_index: 10 } + └─StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + stream_dist_plan: |+ + Fragment 0 + 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 } + ├── tables: [ Materialize: 4294967294 ] + └── StreamProject { exprs: [auction, price, bidder, date_time, _row_id, $expr1] } + └── StreamFilter { predicate: (date_time >= $expr2) AND (date_time <= $expr1) } + └── StreamHashJoin { type: Inner, predicate: price = max(price), output: all } { tables: [ HashJoinLeft: 0, HashJoinDegreeLeft: 1, HashJoinRight: 2, HashJoinDegreeRight: 3 ] } + ├── StreamExchange Hash([2]) from 1 + └── StreamExchange Hash([1]) from 3 + + Fragment 1 + StreamNoOp + └── StreamExchange NoShuffle from 2 + + Fragment 2 + StreamProject { exprs: [auction, bidder, price, date_time, _row_id] } + └── StreamRowIdGen { row_id_index: 10 } + └── StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + ├── tables: [ SourceBackfill: 4 ] + └── Upstream + + Fragment 3 + StreamProject { exprs: [$expr1, max(price), ($expr1 - '00:00:10':Interval) as $expr2] } + └── StreamHashAgg [append_only] { group_key: [$expr1], aggs: [max(price), count] } { tables: [ HashAggState: 5 ] } + └── StreamExchange Hash([0]) from 4 + + Fragment 4 + StreamProject { exprs: [(TumbleStart(date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr1, price, _row_id] } + └── StreamExchange NoShuffle from 2 + + Table 0 + ├── columns: [ auction, bidder, price, date_time, _row_id, _rw_timestamp ] + ├── primary key: [ $2 ASC, $4 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4 ] + ├── distribution key: [ 2 ] + └── read pk prefix len hint: 1 + + Table 1 { columns: [ price, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + + Table 2 { columns: [ $expr1, max(price), $expr2, _rw_timestamp ], primary key: [ $1 ASC, $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 1 ], read pk prefix len hint: 1 } + + Table 3 { columns: [ max(price), $expr1, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + + Table 4 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + + Table 5 { columns: [ $expr1, max(price), count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + + Table 4294967294 + ├── columns: [ auction, price, bidder, date_time, _row_id, $expr1, _rw_timestamp ] + ├── primary key: [ $4 ASC, $5 ASC, $1 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4, 5 ] + ├── distribution key: [ 1 ] + └── read pk prefix len hint: 3 + +- id: nexmark_q8 + before: + - create_sources + sql: | + SELECT + P.id, + P.name, + P.starttime + FROM ( + SELECT + id, + name, + window_start AS starttime, + window_end AS endtime + FROM + TUMBLE(person, date_time, INTERVAL '10' SECOND) + GROUP BY + id, + name, + window_start, + window_end + ) P + JOIN ( + SELECT + seller, + window_start AS starttime, + window_end AS endtime + FROM + TUMBLE(auction, date_time, INTERVAL '10' SECOND) + GROUP BY + seller, + window_start, + window_end + ) A ON P.id = A.seller + AND P.starttime = A.starttime + AND P.endtime = A.endtime; + batch_plan: |- + BatchExchange { order: [], dist: Single } + └─BatchHashJoin { type: Inner, predicate: id = seller AND $expr1 = $expr3 AND $expr2 = $expr4, output: [id, name, $expr1] } + ├─BatchExchange { order: [], dist: HashShard(id, $expr1, $expr2) } + │ └─BatchHashAgg { group_key: [id, name, $expr1, $expr2], aggs: [] } + │ └─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, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id, TumbleStart(date_time, '00:00:10':Interval) as $expr1] } + │ └─BatchKafkaScan { source: person, columns: [id, name, email_address, credit_card, city, state, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id], filter: (None, None) } + └─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, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id, TumbleStart(date_time, '00:00:10':Interval) as $expr3] } + └─BatchKafkaScan { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id], filter: (None, None) } + 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) } + └─StreamHashJoin [append_only] { type: Inner, predicate: id = seller AND $expr1 = $expr3 AND $expr2 = $expr4, output: all } + ├─StreamExchange { dist: HashShard(id, $expr1, $expr2) } + │ └─StreamAppendOnlyDedup { dedup_cols: [id, name, $expr1, $expr2] } + │ └─StreamExchange { dist: HashShard(id, name, $expr1, $expr2) } + │ └─StreamProject { exprs: [id, name, $expr1, ($expr1 + '00:00:10':Interval) as $expr2] } + │ └─StreamProject { exprs: [id, name, email_address, credit_card, city, state, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id, TumbleStart(date_time, '00:00:10':Interval) as $expr1] } + │ └─StreamRowIdGen { row_id_index: 11 } + │ └─StreamSourceScan { columns: [id, name, email_address, credit_card, city, state, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamAppendOnlyDedup { dedup_cols: [seller, $expr3, $expr4] } + └─StreamExchange { dist: HashShard(seller, $expr3, $expr4) } + └─StreamProject { exprs: [seller, $expr3, ($expr3 + '00:00:10':Interval) as $expr4] } + └─StreamProject { exprs: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id, TumbleStart(date_time, '00:00:10':Interval) as $expr3] } + └─StreamRowIdGen { row_id_index: 13 } + └─StreamSourceScan { columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + stream_dist_plan: |+ + Fragment 0 + 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 } + ├── tables: [ Materialize: 4294967294 ] + └── StreamExchange Hash([0, 1, 2, 3]) from 1 + + Fragment 1 + StreamHashJoin [append_only] { type: Inner, predicate: id = seller AND $expr1 = $expr3 AND $expr2 = $expr4, output: all } { tables: [ HashJoinLeft: 0, HashJoinDegreeLeft: 1, HashJoinRight: 2, HashJoinDegreeRight: 3 ] } + ├── StreamExchange Hash([0, 2, 3]) from 2 + └── StreamAppendOnlyDedup { dedup_cols: [seller, $expr3, $expr4] } { tables: [ AppendOnlyDedup: 6 ] } + └── StreamExchange Hash([0, 1, 2]) from 4 + + Fragment 2 + StreamAppendOnlyDedup { dedup_cols: [id, name, $expr1, $expr2] } { tables: [ AppendOnlyDedup: 4 ] } + └── StreamExchange Hash([0, 1, 2, 3]) from 3 + + Fragment 3 + StreamProject { exprs: [id, name, $expr1, ($expr1 + '00:00:10':Interval) as $expr2] } + └── StreamProject { exprs: [id, name, email_address, credit_card, city, state, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id, TumbleStart(date_time, '00:00:10':Interval) as $expr1] } + └── StreamRowIdGen { row_id_index: 11 } + └── StreamSourceScan { columns: [id, name, email_address, credit_card, city, state, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } { tables: [ SourceBackfill: 5 ] } + └── Upstream + + Fragment 4 + StreamProject { exprs: [seller, $expr3, ($expr3 + '00:00:10':Interval) as $expr4] } + └── StreamProject { exprs: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id, TumbleStart(date_time, '00:00:10':Interval) as $expr3] } + └── StreamRowIdGen { row_id_index: 13 } + └── StreamSourceScan { columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } { tables: [ SourceBackfill: 7 ] } + └── Upstream + + Table 0 { columns: [ id, name, $expr1, $expr2, _rw_timestamp ], primary key: [ $0 ASC, $2 ASC, $3 ASC, $1 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0, 2, 3 ], read pk prefix len hint: 3 } + + Table 1 { columns: [ id, $expr1, $expr2, name, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } + + Table 2 { columns: [ seller, $expr3, $expr4, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } + + Table 3 { columns: [ seller, $expr3, $expr4, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } + + Table 4 { columns: [ id, name, $expr1, $expr2, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0, 1, 2, 3 ], read pk prefix len hint: 4 } + + Table 5 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + + Table 6 { columns: [ seller, $expr3, $expr4, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } + + Table 7 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + + Table 4294967294 + ├── columns: [ id, name, starttime, $expr2, seller, $expr3, $expr4, _rw_timestamp ] + ├── primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4, 5, 6 ] + ├── distribution key: [ 0, 1, 2, 3 ] + └── read pk prefix len hint: 4 + +- id: nexmark_q9 + before: + - create_sources + sql: | + SELECT + id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, + auction, bidder, price, bid_date_time + FROM ( + SELECT A.*, B.auction, B.bidder, B.price, B.date_time AS bid_date_time, + ROW_NUMBER() OVER (PARTITION BY A.id ORDER BY B.price DESC, B.date_time ASC) AS rownum + FROM auction A, bid B + WHERE A.id = B.auction AND B.date_time BETWEEN A.date_time AND A.expires + ) + WHERE rownum <= 1; + logical_plan: |- + LogicalProject { exprs: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, auction, bidder, price, date_time] } + └─LogicalFilter { predicate: (row_number <= 1:Int32) } + └─LogicalProject { exprs: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, auction, bidder, price, date_time, row_number] } + └─LogicalOverWindow { window_functions: [row_number() OVER(PARTITION BY id ORDER BY price DESC, date_time ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } + └─LogicalProject { exprs: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id, auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _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, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─LogicalSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _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] } + ├─LogicalKafkaScan { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id], time_range: (Unbounded, Unbounded) } + └─LogicalKafkaScan { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id], time_range: (Unbounded, Unbounded) } + 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) } + │ └─BatchKafkaScan { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id], filter: (None, None) } + └─BatchExchange { order: [], dist: HashShard(auction) } + └─BatchKafkaScan { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id], filter: (None, None) } + 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] } + └─StreamProject { exprs: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, auction, bidder, price, date_time, _row_id, _row_id] } + └─StreamFilter { predicate: (date_time >= date_time) AND (date_time <= expires) } + └─StreamHashJoin [append_only] { type: Inner, predicate: id = auction, output: all } + ├─StreamExchange { dist: HashShard(id) } + │ └─StreamRowIdGen { row_id_index: 13 } + │ └─StreamSourceScan { columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(auction) } + └─StreamRowIdGen { row_id_index: 10 } + └─StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + stream_dist_plan: |+ + Fragment 0 + 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 } + ├── tables: [ Materialize: 4294967294 ] + └── StreamGroupTopN [append_only] { order: [price DESC, date_time ASC], limit: 1, offset: 0, group_key: [id] } { tables: [ AppendOnlyGroupTopN: 0 ] } + └── StreamProject { exprs: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, auction, bidder, price, date_time, _row_id, _row_id] } + └── StreamFilter { predicate: (date_time >= date_time) AND (date_time <= expires) } + └── StreamHashJoin [append_only] { type: Inner, predicate: id = auction, output: all } { tables: [ HashJoinLeft: 1, HashJoinDegreeLeft: 2, HashJoinRight: 3, HashJoinDegreeRight: 4 ] } + ├── StreamExchange Hash([0]) from 1 + └── StreamExchange Hash([0]) from 2 + + Fragment 1 + StreamRowIdGen { row_id_index: 13 } + └── StreamSourceScan { columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } { tables: [ SourceBackfill: 5 ] } + └── Upstream + + Fragment 2 + StreamRowIdGen { row_id_index: 10 } + └── StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } { tables: [ SourceBackfill: 6 ] } + └── Upstream + + Table 0 + ├── columns: [ id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, auction, bidder, price, date_time_0, _row_id, _row_id_0, _rw_timestamp ] + ├── primary key: [ $0 ASC, $11 DESC, $12 ASC, $13 ASC, $14 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14 ] + ├── distribution key: [ 0 ] + └── read pk prefix len hint: 1 + + Table 1 + ├── columns: [ id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id, _rw_timestamp ] + ├── primary key: [ $0 ASC, $13 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13 ] + ├── distribution key: [ 0 ] + └── read pk prefix len hint: 1 + + Table 2 { columns: [ id, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + + Table 3 + ├── columns: [ auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id, _rw_timestamp ] + ├── primary key: [ $0 ASC, $10 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10 ] + ├── distribution key: [ 0 ] + └── read pk prefix len hint: 1 + + Table 4 { columns: [ auction, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + + Table 5 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + + Table 6 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + + Table 4294967294 + ├── columns: [ id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, auction, bidder, price, bid_date_time, _row_id, _row_id#1, _rw_timestamp ] + ├── primary key: [ $0 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14 ] + ├── distribution key: [ 0 ] + └── read pk prefix len hint: 1 + +- id: nexmark_q10 + before: + - create_sources + sql: | + SELECT auction, bidder, price, date_time, TO_CHAR(date_time, 'YYYY-MM-DD') as date, TO_CHAR(date_time, 'HH:MI') as time FROM bid; + 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] } + └─BatchKafkaScan { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id], filter: (None, None) } + 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] } + └─StreamRowIdGen { row_id_index: 10 } + └─StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + stream_dist_plan: |+ + Fragment 0 + StreamMaterialize { columns: [auction, bidder, price, date_time, date, time, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: NoCheck } + ├── tables: [ Materialize: 4294967294 ] + └── 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] } + └── StreamRowIdGen { row_id_index: 10 } + └── StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + ├── tables: [ SourceBackfill: 0 ] + └── Upstream + + Table 0 + ├── columns: [ partition_id, backfill_progress, _rw_timestamp ] + ├── primary key: [ $0 ASC ] + ├── value indices: [ 0, 1 ] + ├── distribution key: [] + └── read pk prefix len hint: 0 + + Table 4294967294 + ├── columns: [ auction, bidder, price, date_time, date, time, _row_id, _rw_timestamp ] + ├── primary key: [ $6 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4, 5, 6 ] + ├── distribution key: [ 6 ] + └── read pk prefix len hint: 1 + +- id: nexmark_q11 + before: + - create_sources + sql: | + SELECT + B.bidder, + count(*) as bid_count, + SESSION_START(B.date_time, INTERVAL '10' SECOND) as starttime, + SESSION_END(B.date_time, INTERVAL '10' SECOND) as endtime + FROM bid B + GROUP BY B.bidder, SESSION(B.date_time, INTERVAL '10' SECOND); + binder_error: | + Failed to bind expression: SESSION_START(B.date_time, INTERVAL '10' SECOND) + + Caused by: + function session_start(timestamp without time zone, interval) does not exist +- id: nexmark_q12 + before: + - create_sources + sql: | + SELECT + B.bidder, + count(*) as bid_count, + TUMBLE_START(B.p_time, INTERVAL '10' SECOND) as starttime, + TUMBLE_END(B.p_time, INTERVAL '10' SECOND) as endtime + FROM (SELECT *, PROCTIME() as p_time FROM bid) B + GROUP BY B.bidder, TUMBLE(B.p_time, INTERVAL '10' SECOND); + binder_error: | + Failed to bind expression: PROCTIME() + + Caused by: + Invalid input syntax: Function `PROCTIME()` is only allowed in CREATE TABLE/SOURCE. Is `NOW()` what you want? +- id: nexmark_q13 + before: + - create_sources + sql: | + /* SELECT + B.auction, + B.bidder, + B.price, + B.date_time, + 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; */ + select 1; + stream_error: 'Bind error: An alias must be specified for the 1st expression (counting from 1) in result relation' +- id: nexmark_q14 + before: + - create_sources + sql: | + SELECT + auction, + bidder, + 0.908 * price as price, + CASE + WHEN + extract(hour from date_time) >= 8 AND + extract(hour from date_time) <= 18 + THEN 'dayTime' + WHEN + extract(hour from date_time) <= 6 OR + extract(hour from date_time) >= 20 + THEN 'nightTime' + ELSE 'otherTime' + END AS bidTimeType, + date_time, + extra + -- ignore UDF in planner test + -- count_char(extra, 'c') AS c_counts + FROM bid + WHERE 0.908 * price > 1000000 AND 0.908 * price < 50000000; + batch_plan: |- + BatchExchange { order: [], dist: Single } + └─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] } + └─BatchKafkaScan { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id], filter: (None, None) } + 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] } + └─StreamFilter { predicate: ((0.908:Decimal * price::Decimal) > 1000000:Decimal) AND ((0.908:Decimal * price::Decimal) < 50000000:Decimal) } + └─StreamRowIdGen { row_id_index: 10 } + └─StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + stream_dist_plan: |+ + Fragment 0 + StreamMaterialize { columns: [auction, bidder, price, bidtimetype, date_time, extra, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: NoCheck } { tables: [ Materialize: 4294967294 ] } + └── 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] } + └── StreamFilter { predicate: ((0.908:Decimal * price::Decimal) > 1000000:Decimal) AND ((0.908:Decimal * price::Decimal) < 50000000:Decimal) } + └── StreamRowIdGen { row_id_index: 10 } + └── StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } { tables: [ SourceBackfill: 0 ] } + └── Upstream + + Table 0 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + + Table 4294967294 { columns: [ auction, bidder, price, bidtimetype, date_time, extra, _row_id, _rw_timestamp ], primary key: [ $6 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6 ], distribution key: [ 6 ], read pk prefix len hint: 1 } + +- id: nexmark_q15 + before: + - create_sources + sql: | + SELECT + TO_CHAR(date_time, '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, + 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 to_char(date_time, 'yyyy-MM-dd'); + batch_plan: |- + BatchExchange { order: [], dist: Single } + └─BatchHashAgg { group_key: [$expr1_expanded], aggs: [sum0(count) filter((flag = 0:Int64)), sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bidder_expanded) filter((flag = 1:Int64)), count(bidder_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(auction_expanded) filter((flag = 2:Int64)), count(auction_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))] } + └─BatchExchange { order: [], dist: HashShard($expr1_expanded) } + └─BatchHashAgg { group_key: [$expr1_expanded, bidder_expanded, auction_expanded, flag], aggs: [count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32))] } + └─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] } + └─BatchKafkaScan { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id], filter: (None, None) } + 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))] } + └─StreamExchange { dist: HashShard($expr1) } + └─StreamProject { exprs: [ToChar(date_time, 'yyyy-MM-dd':Varchar) as $expr1, price, bidder, auction, _row_id] } + └─StreamRowIdGen { row_id_index: 10 } + └─StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + stream_dist_plan: |+ + Fragment 0 + 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 } { tables: [ Materialize: 4294967294 ] } + └── 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))] } + ├── tables: [ HashAggState: 0, HashAggDedupForCol2: 1, HashAggDedupForCol3: 2 ] + └── StreamExchange Hash([0]) from 1 + + Fragment 1 + StreamProject { exprs: [ToChar(date_time, 'yyyy-MM-dd':Varchar) as $expr1, price, bidder, auction, _row_id] } + └── StreamRowIdGen { row_id_index: 10 } + └── StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } { tables: [ SourceBackfill: 3 ] } + └── Upstream + + Table 0 + ├── columns: [ $expr1, 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)), _rw_timestamp ] + ├── primary key: [ $0 ASC ] + ├── value indices: [ 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12 ] + ├── distribution key: [ 0 ] + └── read pk prefix len hint: 1 + + Table 1 { columns: [ $expr1, bidder, count_for_agg_call_4, count_for_agg_call_5, count_for_agg_call_6, count_for_agg_call_7, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 2 } + + Table 2 { columns: [ $expr1, auction, count_for_agg_call_8, count_for_agg_call_9, count_for_agg_call_10, count_for_agg_call_11, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 2 } + + Table 3 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + + Table 4294967294 { 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, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + +- id: nexmark_q15_split_distinct_agg + before: + - create_sources + sql: | + SELECT + TO_CHAR(date_time, '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, + 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 to_char(date_time, 'yyyy-MM-dd'); + 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 } + └─StreamProject { exprs: [$expr1_expanded, sum0(count) filter((flag = 0:Int64)), sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bidder_expanded) filter((flag = 1:Int64)), count(bidder_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(auction_expanded) filter((flag = 2:Int64)), count(auction_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))] } + └─StreamHashAgg { group_key: [$expr1_expanded], aggs: [sum0(count) filter((flag = 0:Int64)), sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bidder_expanded) filter((flag = 1:Int64)), count(bidder_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(auction_expanded) filter((flag = 2:Int64)), count(auction_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count] } + └─StreamExchange { dist: HashShard($expr1_expanded) } + └─StreamHashAgg [append_only] { group_key: [$expr1_expanded, bidder_expanded, auction_expanded, flag], aggs: [count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32))] } + └─StreamExchange { dist: HashShard($expr1_expanded, bidder_expanded, auction_expanded, flag) } + └─StreamExpand { column_subsets: [[$expr1], [$expr1, bidder], [$expr1, auction]] } + └─StreamProject { exprs: [ToChar(date_time, 'yyyy-MM-dd':Varchar) as $expr1, price, bidder, auction, _row_id] } + └─StreamRowIdGen { row_id_index: 10 } + └─StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + stream_dist_plan: |+ + Fragment 0 + 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 } { tables: [ Materialize: 4294967294 ] } + └── StreamProject { exprs: [$expr1_expanded, sum0(count) filter((flag = 0:Int64)), sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bidder_expanded) filter((flag = 1:Int64)), count(bidder_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(auction_expanded) filter((flag = 2:Int64)), count(auction_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))] } + └── StreamHashAgg { group_key: [$expr1_expanded], aggs: [sum0(count) filter((flag = 0:Int64)), sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bidder_expanded) filter((flag = 1:Int64)), count(bidder_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(auction_expanded) filter((flag = 2:Int64)), count(auction_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count] } + ├── tables: [ HashAggState: 0 ] + └── StreamExchange Hash([0]) from 1 + + Fragment 1 + StreamHashAgg [append_only] { group_key: [$expr1_expanded, bidder_expanded, auction_expanded, flag], aggs: [count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32))] } { tables: [ HashAggState: 1 ] } + └── StreamExchange Hash([0, 2, 3, 10]) from 2 + + Fragment 2 + StreamExpand { column_subsets: [[$expr1], [$expr1, bidder], [$expr1, auction]] } + └── StreamProject { exprs: [ToChar(date_time, 'yyyy-MM-dd':Varchar) as $expr1, price, bidder, auction, _row_id] } + └── StreamRowIdGen { row_id_index: 10 } + └── StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } { tables: [ SourceBackfill: 2 ] } + └── Upstream + + Table 0 + ├── columns: [ $expr1_expanded, sum0(count) filter((flag = 0:Int64)), sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bidder_expanded) filter((flag = 1:Int64)), count(bidder_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(auction_expanded) filter((flag = 2:Int64)), count(auction_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count, _rw_timestamp ] + ├── primary key: [ $0 ASC ] + ├── value indices: [ 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13 ] + ├── distribution key: [ 0 ] + └── read pk prefix len hint: 1 + + Table 1 { columns: [ $expr1_expanded, bidder_expanded, auction_expanded, flag, count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4, 5, 6, 7 ], distribution key: [ 0, 1, 2, 3 ], read pk prefix len hint: 4 } + + Table 2 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + + Table 4294967294 { 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, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + + with_config_map: + RW_FORCE_SPLIT_DISTINCT_AGG: 'true' +- id: nexmark_q15_split_distinct_agg_and_force_two_phase + before: + - create_sources + sql: | + SELECT + TO_CHAR(date_time, '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, + 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 to_char(date_time, 'yyyy-MM-dd'); + 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 } + └─StreamProject { exprs: [$expr1_expanded, sum0(sum0(count) filter((flag = 0:Int64))), sum0(sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64))), sum0(sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64))), sum0(sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64))), sum0(count(bidder_expanded) filter((flag = 1:Int64))), sum0(count(bidder_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64))), sum0(count(bidder_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64))), sum0(count(bidder_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64))), sum0(count(auction_expanded) filter((flag = 2:Int64))), sum0(count(auction_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64))), sum0(count(auction_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))), sum0(count(auction_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)))] } + └─StreamHashAgg { group_key: [$expr1_expanded], aggs: [sum0(sum0(count) filter((flag = 0:Int64))), sum0(sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64))), sum0(sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64))), sum0(sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64))), sum0(count(bidder_expanded) filter((flag = 1:Int64))), sum0(count(bidder_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64))), sum0(count(bidder_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64))), sum0(count(bidder_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64))), sum0(count(auction_expanded) filter((flag = 2:Int64))), sum0(count(auction_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64))), sum0(count(auction_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))), sum0(count(auction_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))), count] } + └─StreamExchange { dist: HashShard($expr1_expanded) } + └─StreamHashAgg { group_key: [$expr1_expanded, _vnode], aggs: [sum0(count) filter((flag = 0:Int64)), sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bidder_expanded) filter((flag = 1:Int64)), count(bidder_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(auction_expanded) filter((flag = 2:Int64)), count(auction_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count] } + └─StreamProject { exprs: [$expr1_expanded, bidder_expanded, auction_expanded, flag, count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), Vnode($expr1_expanded, bidder_expanded, auction_expanded, flag) as _vnode] } + └─StreamHashAgg [append_only] { group_key: [$expr1_expanded, bidder_expanded, auction_expanded, flag], aggs: [count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32))] } + └─StreamExchange { dist: HashShard($expr1_expanded, bidder_expanded, auction_expanded, flag) } + └─StreamExpand { column_subsets: [[$expr1], [$expr1, bidder], [$expr1, auction]] } + └─StreamProject { exprs: [ToChar(date_time, 'yyyy-MM-dd':Varchar) as $expr1, price, bidder, auction, _row_id] } + └─StreamRowIdGen { row_id_index: 10 } + └─StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + stream_dist_plan: |+ + Fragment 0 + 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 } { tables: [ Materialize: 4294967294 ] } + └── StreamProject { exprs: [$expr1_expanded, sum0(sum0(count) filter((flag = 0:Int64))), sum0(sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64))), sum0(sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64))), sum0(sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64))), sum0(count(bidder_expanded) filter((flag = 1:Int64))), sum0(count(bidder_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64))), sum0(count(bidder_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64))), sum0(count(bidder_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64))), sum0(count(auction_expanded) filter((flag = 2:Int64))), sum0(count(auction_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64))), sum0(count(auction_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))), sum0(count(auction_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)))] } + └── StreamHashAgg { group_key: [$expr1_expanded], aggs: [sum0(sum0(count) filter((flag = 0:Int64))), sum0(sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64))), sum0(sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64))), sum0(sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64))), sum0(count(bidder_expanded) filter((flag = 1:Int64))), sum0(count(bidder_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64))), sum0(count(bidder_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64))), sum0(count(bidder_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64))), sum0(count(auction_expanded) filter((flag = 2:Int64))), sum0(count(auction_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64))), sum0(count(auction_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))), sum0(count(auction_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))), count] } + ├── tables: [ HashAggState: 0 ] + └── StreamExchange Hash([0]) from 1 + + Fragment 1 + StreamHashAgg { group_key: [$expr1_expanded, _vnode], aggs: [sum0(count) filter((flag = 0:Int64)), sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bidder_expanded) filter((flag = 1:Int64)), count(bidder_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(auction_expanded) filter((flag = 2:Int64)), count(auction_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count] } { tables: [ HashAggState: 1 ] } + └── StreamProject { exprs: [$expr1_expanded, bidder_expanded, auction_expanded, flag, count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), Vnode($expr1_expanded, bidder_expanded, auction_expanded, flag) as _vnode] } + └── StreamHashAgg [append_only] { group_key: [$expr1_expanded, bidder_expanded, auction_expanded, flag], aggs: [count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32))] } { tables: [ HashAggState: 2 ] } + └── StreamExchange Hash([0, 2, 3, 10]) from 2 + + Fragment 2 + StreamExpand { column_subsets: [[$expr1], [$expr1, bidder], [$expr1, auction]] } + └── StreamProject { exprs: [ToChar(date_time, 'yyyy-MM-dd':Varchar) as $expr1, price, bidder, auction, _row_id] } + └── StreamRowIdGen { row_id_index: 10 } + └── StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } { tables: [ SourceBackfill: 3 ] } + └── Upstream + + Table 0 + ├── columns: [ $expr1_expanded, sum0(sum0(count) filter((flag = 0:Int64))), sum0(sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64))), sum0(sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64))), sum0(sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64))), sum0(count(bidder_expanded) filter((flag = 1:Int64))), sum0(count(bidder_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64))), sum0(count(bidder_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64))), sum0(count(bidder_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64))), sum0(count(auction_expanded) filter((flag = 2:Int64))), sum0(count(auction_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64))), sum0(count(auction_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))), sum0(count(auction_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))), count, _rw_timestamp ] + ├── primary key: [ $0 ASC ] + ├── value indices: [ 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13 ] + ├── distribution key: [ 0 ] + └── read pk prefix len hint: 1 + + Table 1 + ├── columns: [ $expr1_expanded, _vnode, sum0(count) filter((flag = 0:Int64)), sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bidder_expanded) filter((flag = 1:Int64)), count(bidder_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(auction_expanded) filter((flag = 2:Int64)), count(auction_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count, _rw_timestamp ] + ├── primary key: [ $0 ASC, $1 ASC ] + ├── value indices: [ 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14 ] + ├── distribution key: [] + ├── read pk prefix len hint: 2 + └── vnode column idx: 1 + + Table 2 { columns: [ $expr1_expanded, bidder_expanded, auction_expanded, flag, count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4, 5, 6, 7 ], distribution key: [ 0, 1, 2, 3 ], read pk prefix len hint: 4 } + + Table 3 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + + Table 4294967294 { 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, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + + with_config_map: + RW_FORCE_SPLIT_DISTINCT_AGG: 'true' + RW_FORCE_TWO_PHASE_AGG: 'true' +- id: nexmark_q16 + before: + - create_sources + sql: | + SELECT + channel, + to_char(date_time, 'yyyy-MM-dd') AS day, + max(to_char(date_time, '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(date_time, 'yyyy-MM-dd'); + batch_plan: |- + BatchExchange { order: [], dist: Single } + └─BatchHashAgg { group_key: [channel_expanded, $expr1_expanded], aggs: [max(max($expr2_expanded)) filter((flag = 0:Int64)), sum0(count) filter((flag = 0:Int64)), sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bidder_expanded) filter((flag = 1:Int64)), count(bidder_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(auction_expanded) filter((flag = 2:Int64)), count(auction_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))] } + └─BatchExchange { order: [], dist: HashShard(channel_expanded, $expr1_expanded) } + └─BatchHashAgg { group_key: [channel_expanded, $expr1_expanded, bidder_expanded, auction_expanded, flag], aggs: [max($expr2_expanded), count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32))] } + └─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] } + └─BatchKafkaScan { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id], filter: (None, None) } + 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))] } + └─StreamExchange { dist: HashShard(channel, $expr1) } + └─StreamProject { exprs: [channel, ToChar(date_time, 'yyyy-MM-dd':Varchar) as $expr1, ToChar(date_time, 'HH:mm':Varchar) as $expr2, price, bidder, auction, _row_id] } + └─StreamRowIdGen { row_id_index: 10 } + └─StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + stream_dist_plan: |+ + Fragment 0 + 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 } { tables: [ Materialize: 4294967294 ] } + └── 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))] } + ├── tables: [ HashAggState: 0, HashAggDedupForCol4: 1, HashAggDedupForCol5: 2 ] + └── StreamExchange Hash([0, 1]) from 1 + + Fragment 1 + StreamProject { exprs: [channel, ToChar(date_time, 'yyyy-MM-dd':Varchar) as $expr1, ToChar(date_time, 'HH:mm':Varchar) as $expr2, price, bidder, auction, _row_id] } + └── StreamRowIdGen { row_id_index: 10 } + └── StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } { tables: [ SourceBackfill: 3 ] } + └── Upstream + + Table 0 + ├── columns: [ channel, $expr1, 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)), _rw_timestamp ] + ├── primary key: [ $0 ASC, $1 ASC ] + ├── value indices: [ 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14 ] + ├── distribution key: [ 0, 1 ] + └── read pk prefix len hint: 2 + + Table 1 { columns: [ channel, $expr1, bidder, count_for_agg_call_5, count_for_agg_call_6, count_for_agg_call_7, count_for_agg_call_8, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3, 4, 5, 6 ], distribution key: [ 0, 1 ], read pk prefix len hint: 3 } + + Table 2 { columns: [ channel, $expr1, auction, count_for_agg_call_9, count_for_agg_call_10, count_for_agg_call_11, count_for_agg_call_12, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3, 4, 5, 6 ], distribution key: [ 0, 1 ], read pk prefix len hint: 3 } + + Table 3 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + + Table 4294967294 { 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, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } + +- id: nexmark_q16 + before: + - create_sources + sql: | + SELECT + channel, + to_char(date_time, 'yyyy-MM-dd') AS day, + max(to_char(date_time, '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(date_time, 'yyyy-MM-dd'); + 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 } + └─StreamProject { exprs: [channel_expanded, $expr1_expanded, max(max($expr2_expanded)) filter((flag = 0:Int64)), sum0(count) filter((flag = 0:Int64)), sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bidder_expanded) filter((flag = 1:Int64)), count(bidder_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(auction_expanded) filter((flag = 2:Int64)), count(auction_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))] } + └─StreamHashAgg { group_key: [channel_expanded, $expr1_expanded], aggs: [max(max($expr2_expanded)) filter((flag = 0:Int64)), sum0(count) filter((flag = 0:Int64)), sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bidder_expanded) filter((flag = 1:Int64)), count(bidder_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(auction_expanded) filter((flag = 2:Int64)), count(auction_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count] } + └─StreamExchange { dist: HashShard(channel_expanded, $expr1_expanded) } + └─StreamHashAgg [append_only] { group_key: [channel_expanded, $expr1_expanded, bidder_expanded, auction_expanded, flag], aggs: [max($expr2_expanded), count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32))] } + └─StreamExchange { dist: HashShard(channel_expanded, $expr1_expanded, bidder_expanded, auction_expanded, flag) } + └─StreamExpand { column_subsets: [[channel, $expr1, $expr2], [channel, $expr1, bidder], [channel, $expr1, auction]] } + └─StreamProject { exprs: [channel, ToChar(date_time, 'yyyy-MM-dd':Varchar) as $expr1, ToChar(date_time, 'HH:mm':Varchar) as $expr2, price, bidder, auction, _row_id] } + └─StreamRowIdGen { row_id_index: 10 } + └─StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + stream_dist_plan: |+ + Fragment 0 + 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 } { tables: [ Materialize: 4294967294 ] } + └── StreamProject { exprs: [channel_expanded, $expr1_expanded, max(max($expr2_expanded)) filter((flag = 0:Int64)), sum0(count) filter((flag = 0:Int64)), sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bidder_expanded) filter((flag = 1:Int64)), count(bidder_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(auction_expanded) filter((flag = 2:Int64)), count(auction_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))] } + └── StreamHashAgg { group_key: [channel_expanded, $expr1_expanded], aggs: [max(max($expr2_expanded)) filter((flag = 0:Int64)), sum0(count) filter((flag = 0:Int64)), sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bidder_expanded) filter((flag = 1:Int64)), count(bidder_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(auction_expanded) filter((flag = 2:Int64)), count(auction_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count] } + ├── tables: [ HashAggState: 1, HashAggCall0: 0 ] + └── StreamExchange Hash([0, 1]) from 1 + + Fragment 1 + StreamHashAgg [append_only] { group_key: [channel_expanded, $expr1_expanded, bidder_expanded, auction_expanded, flag], aggs: [max($expr2_expanded), count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32))] } { tables: [ HashAggState: 2 ] } + └── StreamExchange Hash([0, 1, 4, 5, 14]) from 2 + + Fragment 2 + StreamExpand { column_subsets: [[channel, $expr1, $expr2], [channel, $expr1, bidder], [channel, $expr1, auction]] } + └── StreamProject { exprs: [channel, ToChar(date_time, 'yyyy-MM-dd':Varchar) as $expr1, ToChar(date_time, 'HH:mm':Varchar) as $expr2, price, bidder, auction, _row_id] } + └── StreamRowIdGen { row_id_index: 10 } + └── StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } { tables: [ SourceBackfill: 3 ] } + └── Upstream + + Table 0 { columns: [ channel_expanded, $expr1_expanded, max($expr2_expanded), bidder_expanded, auction_expanded, flag, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 DESC, $3 ASC, $4 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } + + Table 1 + ├── columns: [ channel_expanded, $expr1_expanded, max(max($expr2_expanded)) filter((flag = 0:Int64)), sum0(count) filter((flag = 0:Int64)), sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bidder_expanded) filter((flag = 1:Int64)), count(bidder_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(auction_expanded) filter((flag = 2:Int64)), count(auction_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count, _rw_timestamp ] + ├── primary key: [ $0 ASC, $1 ASC ] + ├── value indices: [ 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15 ] + ├── distribution key: [ 0, 1 ] + └── read pk prefix len hint: 2 + + Table 2 { columns: [ channel_expanded, $expr1_expanded, bidder_expanded, auction_expanded, flag, max($expr2_expanded), count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC ], value indices: [ 5, 6, 7, 8, 9 ], distribution key: [ 0, 1, 2, 3, 4 ], read pk prefix len hint: 5 } + + Table 3 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + + Table 4294967294 { 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, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } + + with_config_map: + RW_FORCE_SPLIT_DISTINCT_AGG: 'true' +- id: nexmark_q17 + before: + - create_sources + sql: | + SELECT + auction, + to_char(date_time, '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, + count(*) filter (where price >= 1000000) AS rank3_bids, + min(price) AS min_price, + max(price) AS max_price, + avg(price) AS avg_price, + sum(price) AS sum_price + FROM bid + GROUP BY auction, to_char(date_time, 'YYYY-MM-DD'); + batch_plan: |- + BatchExchange { order: [], dist: Single } + └─BatchProject { 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)] } + └─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] } + └─BatchKafkaScan { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id], filter: (None, None) } + 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)] } + └─StreamHashAgg [append_only] { 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)] } + └─StreamExchange { dist: HashShard(auction, $expr1) } + └─StreamProject { exprs: [auction, ToChar(date_time, 'YYYY-MM-DD':Varchar) as $expr1, price, _row_id] } + └─StreamRowIdGen { row_id_index: 10 } + └─StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + stream_dist_plan: |+ + Fragment 0 + 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 } { tables: [ Materialize: 4294967294 ] } + └── 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)] } + └── StreamHashAgg [append_only] { 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)] } + ├── tables: [ HashAggState: 0 ] + └── StreamExchange Hash([0, 1]) from 1 + + Fragment 1 + StreamProject { exprs: [auction, ToChar(date_time, 'YYYY-MM-DD':Varchar) as $expr1, price, _row_id] } + └── StreamRowIdGen { row_id_index: 10 } + └── StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } { tables: [ SourceBackfill: 1 ] } + └── Upstream + + Table 0 + ├── columns: [ 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), _rw_timestamp ] + ├── primary key: [ $0 ASC, $1 ASC ] + ├── value indices: [ 2, 3, 4, 5, 6, 7, 8, 9 ] + ├── distribution key: [ 0, 1 ] + └── read pk prefix len hint: 2 + + Table 1 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + + Table 4294967294 + ├── columns: [ auction, day, total_bids, rank1_bids, rank2_bids, rank3_bids, min_price, max_price, avg_price, sum_price, _rw_timestamp ] + ├── primary key: [ $0 ASC, $1 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9 ] + ├── distribution key: [ 0, 1 ] + └── read pk prefix len hint: 2 + +- id: nexmark_q18 + before: + - create_sources + sql: | + SELECT auction, bidder, price, channel, url, date_time, extra + FROM (SELECT *, ROW_NUMBER() OVER (PARTITION BY bidder, auction ORDER BY date_time DESC) AS rank_number + FROM bid) + WHERE rank_number <= 1; + logical_plan: |- + LogicalProject { exprs: [auction, bidder, price, channel, url, date_time, extra] } + └─LogicalFilter { predicate: (row_number <= 1:Int32) } + └─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, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─LogicalSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _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] } + └─BatchKafkaScan { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id], filter: (None, None) } + 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] } + └─StreamExchange { dist: HashShard(bidder, auction) } + └─StreamProject { exprs: [auction, bidder, price, channel, url, date_time, extra, _row_id] } + └─StreamRowIdGen { row_id_index: 10 } + └─StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + stream_dist_plan: |+ + Fragment 0 + StreamMaterialize { columns: [auction, bidder, price, channel, url, date_time, extra, _row_id(hidden)], stream_key: [bidder, auction], pk_columns: [bidder, auction], pk_conflict: NoCheck } + ├── tables: [ Materialize: 4294967294 ] + └── StreamGroupTopN [append_only] { order: [date_time DESC], limit: 1, offset: 0, group_key: [bidder, auction] } { tables: [ AppendOnlyGroupTopN: 0 ] } + └── StreamExchange Hash([1, 0]) from 1 + + Fragment 1 + StreamProject { exprs: [auction, bidder, price, channel, url, date_time, extra, _row_id] } + └── StreamRowIdGen { row_id_index: 10 } + └── StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + ├── tables: [ SourceBackfill: 1 ] + └── Upstream + + Table 0 + ├── columns: [ auction, bidder, price, channel, url, date_time, extra, _row_id, _rw_timestamp ] + ├── primary key: [ $1 ASC, $0 ASC, $5 DESC, $7 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ] + ├── distribution key: [ 1, 0 ] + └── read pk prefix len hint: 2 + + Table 1 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + + Table 4294967294 + ├── columns: [ auction, bidder, price, channel, url, date_time, extra, _row_id, _rw_timestamp ] + ├── primary key: [ $1 ASC, $0 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ] + ├── distribution key: [ 1, 0 ] + └── read pk prefix len hint: 2 + +- id: nexmark_q18_rank + before: + - create_sources + sql: | + SELECT auction, bidder, price, channel, url, date_time, extra + FROM (SELECT *, RANK() OVER (PARTITION BY bidder, auction ORDER BY date_time DESC) AS rank_number + FROM bid) + WHERE rank_number <= 1; + logical_plan: |- + LogicalProject { exprs: [auction, bidder, price, channel, url, date_time, extra] } + └─LogicalFilter { predicate: (rank <= 1:Int32) } + └─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, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─LogicalSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _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] } + └─BatchKafkaScan { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id], filter: (None, None) } + 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] } + └─StreamExchange { dist: HashShard(bidder, auction) } + └─StreamProject { exprs: [auction, bidder, price, channel, url, date_time, extra, _row_id] } + └─StreamRowIdGen { row_id_index: 10 } + └─StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + stream_dist_plan: |+ + Fragment 0 + 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 } + ├── tables: [ Materialize: 4294967294 ] + └── StreamGroupTopN [append_only] { order: [date_time DESC], limit: 1, offset: 0, with_ties: true, group_key: [bidder, auction] } { tables: [ AppendOnlyGroupTopN: 0 ] } + └── StreamExchange Hash([1, 0]) from 1 + + Fragment 1 + StreamProject { exprs: [auction, bidder, price, channel, url, date_time, extra, _row_id] } + └── StreamRowIdGen { row_id_index: 10 } + └── StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } { tables: [ SourceBackfill: 1 ] } + └── Upstream + + Table 0 + ├── columns: [ auction, bidder, price, channel, url, date_time, extra, _row_id, _rw_timestamp ] + ├── primary key: [ $1 ASC, $0 ASC, $5 DESC, $7 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ] + ├── distribution key: [ 1, 0 ] + └── read pk prefix len hint: 2 + + Table 1 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + + Table 4294967294 + ├── columns: [ auction, bidder, price, channel, url, date_time, extra, _row_id, _rw_timestamp ] + ├── primary key: [ $1 ASC, $0 ASC, $7 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ] + ├── distribution key: [ 1, 0 ] + └── read pk prefix len hint: 3 + +- id: nexmark_q19 + before: + - create_sources + sql: | + SELECT * FROM + (SELECT *, ROW_NUMBER() OVER (PARTITION BY auction ORDER BY price DESC) AS rank_number FROM bid) + WHERE rank_number <= 10; + logical_plan: |- + LogicalProject { exprs: [auction, bidder, price, channel, url, date_time, extra, row_number] } + └─LogicalFilter { predicate: (row_number <= 10:Int32) } + └─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, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─LogicalSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _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)] } + └─BatchSort { order: [auction ASC, price DESC] } + └─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] } + └─BatchKafkaScan { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id], filter: (None, None) } + 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)] } + └─StreamGroupTopN [append_only] { order: [price DESC], limit: 10, offset: 0, group_key: [auction] } + └─StreamExchange { dist: HashShard(auction) } + └─StreamProject { exprs: [auction, bidder, price, channel, url, date_time, extra, _row_id] } + └─StreamRowIdGen { row_id_index: 10 } + └─StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + stream_dist_plan: |+ + Fragment 0 + 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 } + ├── tables: [ Materialize: 4294967294 ] + └── StreamOverWindow { window_functions: [row_number() OVER(PARTITION BY auction ORDER BY price DESC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } { tables: [ OverWindow: 0 ] } + └── StreamGroupTopN [append_only] { order: [price DESC], limit: 10, offset: 0, group_key: [auction] } { tables: [ AppendOnlyGroupTopN: 1 ] } + └── StreamExchange Hash([0]) from 1 + + Fragment 1 + StreamProject { exprs: [auction, bidder, price, channel, url, date_time, extra, _row_id] } + └── StreamRowIdGen { row_id_index: 10 } + └── StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } { tables: [ SourceBackfill: 2 ] } + └── Upstream + + Table 0 + ├── columns: [ auction, bidder, price, channel, url, date_time, extra, _row_id, row_number, _rw_timestamp ] + ├── primary key: [ $0 ASC, $2 DESC, $7 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8 ] + ├── distribution key: [ 0 ] + └── read pk prefix len hint: 1 + + Table 1 + ├── columns: [ auction, bidder, price, channel, url, date_time, extra, _row_id, _rw_timestamp ] + ├── primary key: [ $0 ASC, $2 DESC, $7 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ] + ├── distribution key: [ 0 ] + └── read pk prefix len hint: 1 + + Table 2 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + + Table 4294967294 + ├── columns: [ auction, bidder, price, channel, url, date_time, extra, _row_id, rank_number, _rw_timestamp ] + ├── primary key: [ $0 ASC, $7 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8 ] + ├── distribution key: [ 0 ] + └── read pk prefix len hint: 2 + +- id: nexmark_q20 + before: + - create_sources + sql: | + SELECT + auction, bidder, price, channel, url, B.date_time as date_timeB, + item_name, description, initial_bid, reserve, A.date_time as date_timeA, expires, seller, category + FROM + bid B INNER JOIN auction A on B.auction = A.id + WHERE A.category = 10; + batch_plan: |- + 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) } + │ └─BatchKafkaScan { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id], filter: (None, None) } + └─BatchExchange { order: [], dist: HashShard(id) } + └─BatchFilter { predicate: (category = 10:Int32) } + └─BatchProject { exprs: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category] } + └─BatchKafkaScan { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id], filter: (None, None) } + 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) } + └─StreamHashJoin [append_only] { type: Inner, predicate: auction = id, output: [auction, bidder, price, channel, url, date_time, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id, _row_id] } + ├─StreamExchange { dist: HashShard(auction) } + │ └─StreamRowIdGen { row_id_index: 10 } + │ └─StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(id) } + └─StreamFilter { predicate: (category = 10:Int32) } + └─StreamRowIdGen { row_id_index: 13 } + └─StreamSourceScan { columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + stream_dist_plan: |+ + Fragment 0 + 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 } + ├── tables: [ Materialize: 4294967294 ] + └── StreamExchange Hash([0, 14, 15]) from 1 + + Fragment 1 + StreamHashJoin [append_only] { type: Inner, predicate: auction = id, output: [auction, bidder, price, channel, url, date_time, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id, _row_id] } + ├── tables: [ HashJoinLeft: 0, HashJoinDegreeLeft: 1, HashJoinRight: 2, HashJoinDegreeRight: 3 ] + ├── StreamExchange Hash([0]) from 2 + └── StreamExchange Hash([0]) from 3 + + Fragment 2 + StreamRowIdGen { row_id_index: 10 } + └── StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } { tables: [ SourceBackfill: 4 ] } + └── Upstream + + Fragment 3 + StreamFilter { predicate: (category = 10:Int32) } + └── StreamRowIdGen { row_id_index: 13 } + └── StreamSourceScan { columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } { tables: [ SourceBackfill: 5 ] } + └── Upstream + + Table 0 { columns: [ auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id, _rw_timestamp ], primary key: [ $0 ASC, $10 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + + Table 1 { columns: [ auction, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + + Table 2 + ├── columns: [ id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id, _rw_timestamp ] + ├── primary key: [ $0 ASC, $13 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13 ] + ├── distribution key: [ 0 ] + └── read pk prefix len hint: 1 + + Table 3 { columns: [ id, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + + Table 4 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + + Table 5 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + + Table 4294967294 + ├── columns: [ auction, bidder, price, channel, url, date_timeb, item_name, description, initial_bid, reserve, date_timea, expires, seller, category, _row_id, _row_id#1, _rw_timestamp ] + ├── primary key: [ $14 ASC, $15 ASC, $0 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15 ] + ├── distribution key: [ 0, 14, 15 ] + └── read pk prefix len hint: 3 + +- id: nexmark_q21 + before: + - create_sources + sql: | + SELECT + auction, bidder, price, channel, + CASE + WHEN lower(channel) = 'apple' THEN '0' + WHEN lower(channel) = 'google' THEN '1' + WHEN lower(channel) = 'facebook' THEN '2' + WHEN lower(channel) = 'baidu' THEN '3' + ELSE REGEXP_MATCH(url, '(&|^)channel_id=([^&]*)')[2] + END + AS channel_id FROM bid + where REGEXP_MATCH(url, '(&|^)channel_id=([^&]*)')[2] is not null or + lower(channel) in ('apple', 'google', 'facebook', 'baidu'); + batch_plan: |- + BatchExchange { order: [], dist: Single } + └─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] } + └─BatchKafkaScan { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id], filter: (None, None) } + 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] } + └─StreamFilter { predicate: (IsNotNull(ArrayAccess(RegexpMatch(url, '(&|^)channel_id=([^&]*)':Varchar), 2:Int32)) OR In(Lower(channel), 'apple':Varchar, 'google':Varchar, 'facebook':Varchar, 'baidu':Varchar)) } + └─StreamRowIdGen { row_id_index: 10 } + └─StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + stream_dist_plan: |+ + Fragment 0 + StreamMaterialize { columns: [auction, bidder, price, channel, channel_id, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: NoCheck } { tables: [ Materialize: 4294967294 ] } + └── 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] } + └── StreamFilter { predicate: (IsNotNull(ArrayAccess(RegexpMatch(url, '(&|^)channel_id=([^&]*)':Varchar), 2:Int32)) OR In(Lower(channel), 'apple':Varchar, 'google':Varchar, 'facebook':Varchar, 'baidu':Varchar)) } + └── StreamRowIdGen { row_id_index: 10 } + └── StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } { tables: [ SourceBackfill: 0 ] } + └── Upstream + + Table 0 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + + Table 4294967294 { columns: [ auction, bidder, price, channel, channel_id, _row_id, _rw_timestamp ], primary key: [ $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 5 ], read pk prefix len hint: 1 } + +- id: nexmark_q22 + before: + - create_sources + sql: | + SELECT + auction, bidder, price, channel, + SPLIT_PART(url, '/', 4) as dir1, + SPLIT_PART(url, '/', 5) as dir2, + SPLIT_PART(url, '/', 6) as dir3 FROM bid; + 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] } + └─BatchKafkaScan { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id], filter: (None, None) } + 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] } + └─StreamRowIdGen { row_id_index: 10 } + └─StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + stream_dist_plan: |+ + Fragment 0 + StreamMaterialize { columns: [auction, bidder, price, channel, dir1, dir2, dir3, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: NoCheck } { tables: [ Materialize: 4294967294 ] } + └── 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] } + └── StreamRowIdGen { row_id_index: 10 } + └── StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } { tables: [ SourceBackfill: 0 ] } + └── Upstream + + Table 0 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + + Table 4294967294 + ├── columns: [ auction, bidder, price, channel, dir1, dir2, dir3, _row_id, _rw_timestamp ] + ├── primary key: [ $7 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ] + ├── distribution key: [ 7 ] + └── read pk prefix len hint: 1 + +- id: nexmark_q101 + before: + - create_sources + sql: | + -- A self-made query that covers outer join. + -- + -- Monitor ongoing auctions and track the current highest bid for each one in real-time. If + -- the auction has no bids, the highest bid will be NULL. + SELECT + a.id AS auction_id, + a.item_name AS auction_item_name, + b.max_price AS current_highest_bid + FROM auction a + LEFT OUTER JOIN ( + SELECT + b1.auction, + MAX(b1.price) max_price + FROM bid b1 + GROUP BY b1.auction + ) b ON a.id = b.auction; + batch_plan: |- + BatchExchange { order: [], dist: Single } + └─BatchHashJoin { type: LeftOuter, predicate: id = auction, output: [id, item_name, max(price)] } + ├─BatchExchange { order: [], dist: HashShard(id) } + │ └─BatchKafkaScan { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id], filter: (None, None) } + └─BatchHashAgg { group_key: [auction], aggs: [max(price)] } + └─BatchExchange { order: [], dist: HashShard(auction) } + └─BatchKafkaScan { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id], filter: (None, None) } + 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) } + └─StreamHashJoin { type: LeftOuter, predicate: id = auction, output: [id, item_name, max(price), _row_id, auction] } + ├─StreamExchange { dist: HashShard(id) } + │ └─StreamRowIdGen { row_id_index: 13 } + │ └─StreamSourceScan { columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamProject { exprs: [auction, max(price)] } + └─StreamHashAgg [append_only] { group_key: [auction], aggs: [max(price), count] } + └─StreamExchange { dist: HashShard(auction) } + └─StreamRowIdGen { row_id_index: 10 } + └─StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + stream_dist_plan: |+ + Fragment 0 + 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 } + ├── tables: [ Materialize: 4294967294 ] + └── StreamExchange Hash([0, 3]) from 1 + + Fragment 1 + StreamHashJoin { type: LeftOuter, predicate: id = auction, output: [id, item_name, max(price), _row_id, auction] } + ├── tables: [ HashJoinLeft: 0, HashJoinDegreeLeft: 1, HashJoinRight: 2, HashJoinDegreeRight: 3 ] + ├── StreamExchange Hash([0]) from 2 + └── StreamProject { exprs: [auction, max(price)] } + └── StreamHashAgg [append_only] { group_key: [auction], aggs: [max(price), count] } { tables: [ HashAggState: 5 ] } + └── StreamExchange Hash([0]) from 3 + + Fragment 2 + StreamRowIdGen { row_id_index: 13 } + └── StreamSourceScan { columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + ├── tables: [ SourceBackfill: 4 ] + └── Upstream + + Fragment 3 + StreamRowIdGen { row_id_index: 10 } + └── StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } { tables: [ SourceBackfill: 6 ] } + └── Upstream + + Table 0 + ├── columns: [ id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id, _rw_timestamp ] + ├── primary key: [ $0 ASC, $13 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13 ] + ├── distribution key: [ 0 ] + └── read pk prefix len hint: 1 + + Table 1 { columns: [ id, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + + Table 2 { columns: [ auction, max(price), _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + + Table 3 { columns: [ auction, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + + Table 4 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + + Table 5 { columns: [ auction, max(price), count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + + Table 6 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + + Table 4294967294 + ├── columns: [ auction_id, auction_item_name, current_highest_bid, _row_id, auction, _rw_timestamp ] + ├── primary key: [ $3 ASC, $0 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4 ] + ├── distribution key: [ 0, 3 ] + └── read pk prefix len hint: 2 + +- id: nexmark_q102 + before: + - create_sources + sql: | + -- A self-made query that covers dynamic filter. + -- + -- Show the auctions whose count of bids is greater than the overall average count of bids + -- per auction. + SELECT + a.id AS auction_id, + a.item_name AS auction_item_name, + COUNT(b.auction) AS bid_count + FROM auction a + JOIN bid b ON a.id = b.auction + GROUP BY a.id, a.item_name + HAVING COUNT(b.auction) >= ( + SELECT COUNT(*) / COUNT(DISTINCT auction) FROM bid + ) + batch_plan: |- + BatchNestedLoopJoin { type: Inner, predicate: (count(auction) >= $expr1), output: [id, item_name, count(auction)] } + ├─BatchExchange { order: [], dist: Single } + │ └─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) } + │ │ └─BatchKafkaScan { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id], filter: (None, None) } + │ └─BatchExchange { order: [], dist: HashShard(auction) } + │ └─BatchKafkaScan { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id], filter: (None, None) } + └─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) } + └─BatchKafkaScan { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id], filter: (None, None) } + 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)] } + ├─StreamProject { exprs: [id, item_name, count(auction)] } + │ └─StreamHashAgg [append_only] { group_key: [id, item_name], aggs: [count(auction), count] } + │ └─StreamHashJoin [append_only] { type: Inner, predicate: id = auction, output: [id, item_name, auction, _row_id, _row_id] } + │ ├─StreamExchange { dist: HashShard(id) } + │ │ └─StreamRowIdGen { row_id_index: 13 } + │ │ └─StreamSourceScan { columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ └─StreamExchange { dist: HashShard(auction) } + │ └─StreamShare { id: 5 } + │ └─StreamProject { exprs: [auction, _row_id] } + │ └─StreamRowIdGen { row_id_index: 10 } + │ └─StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: Broadcast } + └─StreamProject { exprs: [(sum0(sum0(count)) / sum0(count(auction))) as $expr1] } + └─StreamSimpleAgg { aggs: [sum0(sum0(count)), sum0(count(auction)), count] } + └─StreamExchange { dist: Single } + └─StreamStatelessSimpleAgg { aggs: [sum0(count), count(auction)] } + └─StreamHashAgg [append_only] { group_key: [auction], aggs: [count] } + └─StreamExchange { dist: HashShard(auction) } + └─StreamShare { id: 5 } + └─StreamProject { exprs: [auction, _row_id] } + └─StreamRowIdGen { row_id_index: 10 } + └─StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + stream_dist_plan: |+ + Fragment 0 + 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 } + ├── tables: [ Materialize: 4294967294 ] + └── StreamDynamicFilter { predicate: (count(auction) >= $expr1), output: [id, item_name, count(auction)] } { tables: [ DynamicFilterLeft: 0, DynamicFilterRight: 1 ] } + ├── StreamProject { exprs: [id, item_name, count(auction)] } + │ └── StreamHashAgg [append_only] { group_key: [id, item_name], aggs: [count(auction), count] } { tables: [ HashAggState: 2 ] } + │ └── StreamHashJoin [append_only] { type: Inner, predicate: id = auction, output: [id, item_name, auction, _row_id, _row_id] } + │ ├── tables: [ HashJoinLeft: 3, HashJoinDegreeLeft: 4, HashJoinRight: 5, HashJoinDegreeRight: 6 ] + │ ├── StreamExchange Hash([0]) from 1 + │ └── StreamExchange Hash([0]) from 2 + └── StreamExchange Broadcast from 4 + + Fragment 1 + StreamRowIdGen { row_id_index: 13 } + └── StreamSourceScan { columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + ├── tables: [ SourceBackfill: 7 ] + └── Upstream + + Fragment 2 + StreamNoOp + └── StreamExchange NoShuffle from 3 + + Fragment 3 + StreamProject { exprs: [auction, _row_id] } + └── StreamRowIdGen { row_id_index: 10 } + └── StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } { tables: [ SourceBackfill: 8 ] } + └── Upstream + + Fragment 4 + StreamProject { exprs: [(sum0(sum0(count)) / sum0(count(auction))) as $expr1] } + └── StreamSimpleAgg { aggs: [sum0(sum0(count)), sum0(count(auction)), count] } { tables: [ SimpleAggState: 9 ] } + └── StreamExchange Single from 5 + + Fragment 5 + StreamStatelessSimpleAgg { aggs: [sum0(count), count(auction)] } + └── StreamHashAgg [append_only] { group_key: [auction], aggs: [count] } { tables: [ HashAggState: 10 ] } + └── StreamExchange Hash([0]) from 6 + + Fragment 6 + StreamNoOp + └── StreamExchange NoShuffle from 3 + + Table 0 { columns: [ id, item_name, count(auction), _rw_timestamp ], primary key: [ $2 ASC, $0 ASC, $1 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + + Table 1 { columns: [ $expr1, _rw_timestamp ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } + + Table 2 { columns: [ id, item_name, count(auction), count, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 2 } + + Table 3 + ├── columns: [ id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id, _rw_timestamp ] + ├── primary key: [ $0 ASC, $13 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13 ] + ├── distribution key: [ 0 ] + └── read pk prefix len hint: 1 + + Table 4 { columns: [ id, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + + Table 5 { columns: [ auction, _row_id, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + + Table 6 { columns: [ auction, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + + Table 7 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + + Table 8 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + + Table 9 { columns: [ sum0(sum0(count)), sum0(count(auction)), count, _rw_timestamp ], primary key: [], value indices: [ 0, 1, 2 ], distribution key: [], read pk prefix len hint: 0 } + + Table 10 { columns: [ auction, count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + + Table 4294967294 + ├── columns: [ auction_id, auction_item_name, bid_count, _rw_timestamp ] + ├── primary key: [ $0 ASC, $1 ASC ] + ├── value indices: [ 0, 1, 2 ] + ├── distribution key: [ 0 ] + └── read pk prefix len hint: 2 + +- id: nexmark_q103 + before: + - create_sources + sql: | + -- A self-made query that covers semi join. + -- + -- Show the auctions that have at least 20 bids. + SELECT + a.id AS auction_id, + a.item_name AS auction_item_name + FROM auction a + WHERE a.id IN ( + SELECT b.auction FROM bid b + GROUP BY b.auction + HAVING COUNT(*) >= 20 + ); + batch_plan: |- + BatchExchange { order: [], dist: Single } + └─BatchHashJoin { type: LeftSemi, predicate: id = auction, output: [id, item_name] } + ├─BatchExchange { order: [], dist: HashShard(id) } + │ └─BatchKafkaScan { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id], filter: (None, None) } + └─BatchProject { exprs: [auction] } + └─BatchFilter { predicate: (count >= 20:Int32) } + └─BatchHashAgg { group_key: [auction], aggs: [count] } + └─BatchExchange { order: [], dist: HashShard(auction) } + └─BatchKafkaScan { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id], filter: (None, None) } + 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) } + └─StreamHashJoin { type: LeftSemi, predicate: id = auction, output: [id, item_name, _row_id] } + ├─StreamExchange { dist: HashShard(id) } + │ └─StreamRowIdGen { row_id_index: 13 } + │ └─StreamSourceScan { columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamProject { exprs: [auction] } + └─StreamFilter { predicate: (count >= 20:Int32) } + └─StreamHashAgg [append_only] { group_key: [auction], aggs: [count] } + └─StreamExchange { dist: HashShard(auction) } + └─StreamRowIdGen { row_id_index: 10 } + └─StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + stream_dist_plan: |+ + Fragment 0 + 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 } + ├── tables: [ Materialize: 4294967294 ] + └── StreamExchange Hash([0, 2]) from 1 + + Fragment 1 + StreamHashJoin { type: LeftSemi, predicate: id = auction, output: [id, item_name, _row_id] } + ├── tables: [ HashJoinLeft: 0, HashJoinDegreeLeft: 1, HashJoinRight: 2, HashJoinDegreeRight: 3 ] + ├── StreamExchange Hash([0]) from 2 + └── StreamProject { exprs: [auction] } + └── StreamFilter { predicate: (count >= 20:Int32) } + └── StreamHashAgg [append_only] { group_key: [auction], aggs: [count] } { tables: [ HashAggState: 5 ] } + └── StreamExchange Hash([0]) from 3 + + Fragment 2 + StreamRowIdGen { row_id_index: 13 } + └── StreamSourceScan { columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + ├── tables: [ SourceBackfill: 4 ] + └── Upstream + + Fragment 3 + StreamRowIdGen { row_id_index: 10 } + └── StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } { tables: [ SourceBackfill: 6 ] } + └── Upstream + + Table 0 + ├── columns: [ id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id, _rw_timestamp ] + ├── primary key: [ $0 ASC, $13 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13 ] + ├── distribution key: [ 0 ] + └── read pk prefix len hint: 1 + + Table 1 { columns: [ id, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + + Table 2 { columns: [ auction, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + + Table 3 { columns: [ auction, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + + Table 4 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + + Table 5 { columns: [ auction, count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + + Table 6 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + + Table 4294967294 + ├── columns: [ auction_id, auction_item_name, _row_id, _rw_timestamp ] + ├── primary key: [ $2 ASC, $0 ASC ] + ├── value indices: [ 0, 1, 2 ] + ├── distribution key: [ 0, 2 ] + └── read pk prefix len hint: 2 + +- id: nexmark_q104 + before: + - create_sources + sql: | + -- A self-made query that covers anti join. + -- + -- This is the same as q103, which shows the auctions that have at least 20 bids. + SELECT + a.id AS auction_id, + a.item_name AS auction_item_name + FROM auction a + WHERE a.id NOT IN ( + SELECT b.auction FROM bid b + GROUP BY b.auction + HAVING COUNT(*) < 20 + ); + batch_plan: |- + BatchExchange { order: [], dist: Single } + └─BatchHashJoin { type: LeftAnti, predicate: id = auction, output: [id, item_name] } + ├─BatchExchange { order: [], dist: HashShard(id) } + │ └─BatchKafkaScan { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id], filter: (None, None) } + └─BatchProject { exprs: [auction] } + └─BatchFilter { predicate: (count < 20:Int32) } + └─BatchHashAgg { group_key: [auction], aggs: [count] } + └─BatchExchange { order: [], dist: HashShard(auction) } + └─BatchKafkaScan { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id], filter: (None, None) } + 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) } + └─StreamHashJoin { type: LeftAnti, predicate: id = auction, output: [id, item_name, _row_id] } + ├─StreamExchange { dist: HashShard(id) } + │ └─StreamRowIdGen { row_id_index: 13 } + │ └─StreamSourceScan { columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamProject { exprs: [auction] } + └─StreamFilter { predicate: (count < 20:Int32) } + └─StreamHashAgg [append_only] { group_key: [auction], aggs: [count] } + └─StreamExchange { dist: HashShard(auction) } + └─StreamRowIdGen { row_id_index: 10 } + └─StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + stream_dist_plan: |+ + Fragment 0 + 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 } + ├── tables: [ Materialize: 4294967294 ] + └── StreamExchange Hash([0, 2]) from 1 + + Fragment 1 + StreamHashJoin { type: LeftAnti, predicate: id = auction, output: [id, item_name, _row_id] } + ├── tables: [ HashJoinLeft: 0, HashJoinDegreeLeft: 1, HashJoinRight: 2, HashJoinDegreeRight: 3 ] + ├── StreamExchange Hash([0]) from 2 + └── StreamProject { exprs: [auction] } + └── StreamFilter { predicate: (count < 20:Int32) } + └── StreamHashAgg [append_only] { group_key: [auction], aggs: [count] } { tables: [ HashAggState: 5 ] } + └── StreamExchange Hash([0]) from 3 + + Fragment 2 + StreamRowIdGen { row_id_index: 13 } + └── StreamSourceScan { columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + ├── tables: [ SourceBackfill: 4 ] + └── Upstream + + Fragment 3 + StreamRowIdGen { row_id_index: 10 } + └── StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } { tables: [ SourceBackfill: 6 ] } + └── Upstream + + Table 0 + ├── columns: [ id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id, _rw_timestamp ] + ├── primary key: [ $0 ASC, $13 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13 ] + ├── distribution key: [ 0 ] + └── read pk prefix len hint: 1 + + Table 1 { columns: [ id, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + + Table 2 { columns: [ auction, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + + Table 3 { columns: [ auction, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + + Table 4 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + + Table 5 { columns: [ auction, count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + + Table 6 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + + Table 4294967294 + ├── columns: [ auction_id, auction_item_name, _row_id, _rw_timestamp ] + ├── primary key: [ $2 ASC, $0 ASC ] + ├── value indices: [ 0, 1, 2 ] + ├── distribution key: [ 0, 2 ] + └── read pk prefix len hint: 2 + +- id: nexmark_q105 + before: + - create_sources + sql: | + -- A self-made query that covers singleton top-n (and local-phase group top-n). + -- + -- Show the top 1000 auctions by the number of bids. + SELECT + a.id AS auction_id, + a.item_name AS auction_item_name, + COUNT(b.auction) AS bid_count + FROM auction a + JOIN bid b ON a.id = b.auction + GROUP BY a.id, a.item_name + ORDER BY bid_count DESC + LIMIT 1000; + batch_plan: |- + BatchTopN { order: [count(auction) DESC], limit: 1000, offset: 0 } + └─BatchExchange { order: [], dist: Single } + └─BatchTopN { order: [count(auction) DESC], limit: 1000, offset: 0 } + └─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) } + │ └─BatchKafkaScan { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id], filter: (None, None) } + └─BatchExchange { order: [], dist: HashShard(auction) } + └─BatchKafkaScan { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id], filter: (None, None) } + 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)] } + └─StreamTopN { order: [count(auction) DESC], limit: 1000, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [count(auction) DESC], limit: 1000, offset: 0, group_key: [$expr1] } + └─StreamProject { exprs: [id, item_name, count(auction), Vnode(id) as $expr1] } + └─StreamHashAgg [append_only] { group_key: [id, item_name], aggs: [count(auction), count] } + └─StreamHashJoin [append_only] { type: Inner, predicate: id = auction, output: [id, item_name, auction, _row_id, _row_id] } + ├─StreamExchange { dist: HashShard(id) } + │ └─StreamRowIdGen { row_id_index: 13 } + │ └─StreamSourceScan { columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(auction) } + └─StreamRowIdGen { row_id_index: 10 } + └─StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + stream_dist_plan: |+ + Fragment 0 + 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 } + ├── tables: [ Materialize: 4294967294 ] + └── StreamProject { exprs: [id, item_name, count(auction)] } + └── StreamTopN { order: [count(auction) DESC], limit: 1000, offset: 0 } { tables: [ TopN: 0 ] } + └── StreamExchange Single from 1 + + Fragment 1 + StreamGroupTopN { order: [count(auction) DESC], limit: 1000, offset: 0, group_key: [$expr1] } { tables: [ GroupTopN: 1 ] } + └── StreamProject { exprs: [id, item_name, count(auction), Vnode(id) as $expr1] } + └── StreamHashAgg [append_only] { group_key: [id, item_name], aggs: [count(auction), count] } { tables: [ HashAggState: 2 ] } + └── StreamHashJoin [append_only] { type: Inner, predicate: id = auction, output: [id, item_name, auction, _row_id, _row_id] } + ├── tables: [ HashJoinLeft: 3, HashJoinDegreeLeft: 4, HashJoinRight: 5, HashJoinDegreeRight: 6 ] + ├── StreamExchange Hash([0]) from 2 + └── StreamExchange Hash([0]) from 3 + + Fragment 2 + StreamRowIdGen { row_id_index: 13 } + └── StreamSourceScan { columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + ├── tables: [ SourceBackfill: 7 ] + └── Upstream + + Fragment 3 + StreamRowIdGen { row_id_index: 10 } + └── StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } { tables: [ SourceBackfill: 8 ] } + └── Upstream + + Table 0 + ├── columns: [ id, item_name, count(auction), $expr1, _rw_timestamp ] + ├── primary key: [ $2 DESC, $0 ASC, $1 ASC ] + ├── value indices: [ 0, 1, 2, 3 ] + ├── distribution key: [] + └── read pk prefix len hint: 0 + + Table 1 + ├── columns: [ id, item_name, count(auction), $expr1, _rw_timestamp ] + ├── primary key: [ $3 ASC, $2 DESC, $0 ASC, $1 ASC ] + ├── value indices: [ 0, 1, 2, 3 ] + ├── distribution key: [ 0 ] + ├── read pk prefix len hint: 1 + └── vnode column idx: 3 + + Table 2 { columns: [ id, item_name, count(auction), count, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 2 } + + Table 3 + ├── columns: [ id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id, _rw_timestamp ] + ├── primary key: [ $0 ASC, $13 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13 ] + ├── distribution key: [ 0 ] + └── read pk prefix len hint: 1 + + Table 4 { columns: [ id, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + + Table 5 + ├── columns: [ auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id, _rw_timestamp ] + ├── primary key: [ $0 ASC, $10 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10 ] + ├── distribution key: [ 0 ] + └── read pk prefix len hint: 1 + + Table 6 { columns: [ auction, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + + Table 7 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + + Table 8 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + + Table 4294967294 + ├── columns: [ auction_id, auction_item_name, bid_count, _rw_timestamp ] + ├── primary key: [ $2 DESC, $0 ASC, $1 ASC ] + ├── value indices: [ 0, 1, 2 ] + ├── distribution key: [] + └── read pk prefix len hint: 3 + +- id: nexmark_q106 + before: + - create_sources + sql: | + -- A self-made query that covers two-phase stateful simple aggregation. + -- + -- Show the minimum final price of all auctions. + SELECT + MIN(final) AS min_final + FROM + ( + SELECT + auction.id, + MAX(price) AS final + FROM + auction, + bid + WHERE + bid.auction = auction.id + AND bid.date_time BETWEEN auction.date_time AND auction.expires + GROUP BY + auction.id + ) + batch_plan: |- + BatchSimpleAgg { aggs: [min(min(max(price)))] } + └─BatchExchange { order: [], dist: Single } + └─BatchSimpleAgg { aggs: [min(max(price))] } + └─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) } + │ └─BatchKafkaScan { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id], filter: (None, None) } + └─BatchExchange { order: [], dist: HashShard(auction) } + └─BatchKafkaScan { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id], filter: (None, None) } + stream_plan: |- + StreamMaterialize { columns: [min_final], stream_key: [], pk_columns: [], pk_conflict: NoCheck } + └─StreamProject { exprs: [min(min(max(price)))] } + └─StreamSimpleAgg { aggs: [min(min(max(price))), count] } + └─StreamExchange { dist: Single } + └─StreamHashAgg { group_key: [$expr1], aggs: [min(max(price)), count] } + └─StreamProject { exprs: [id, max(price), Vnode(id) as $expr1] } + └─StreamHashAgg [append_only] { group_key: [id], aggs: [max(price), count] } + └─StreamProject { exprs: [id, price, _row_id, _row_id] } + └─StreamFilter { predicate: (date_time >= date_time) AND (date_time <= expires) } + └─StreamHashJoin [append_only] { type: Inner, predicate: id = auction, output: all } + ├─StreamExchange { dist: HashShard(id) } + │ └─StreamRowIdGen { row_id_index: 13 } + │ └─StreamSourceScan { columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(auction) } + └─StreamRowIdGen { row_id_index: 10 } + └─StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + stream_dist_plan: |+ + Fragment 0 + StreamMaterialize { columns: [min_final], stream_key: [], pk_columns: [], pk_conflict: NoCheck } + ├── tables: [ Materialize: 4294967294 ] + └── StreamProject { exprs: [min(min(max(price)))] } + └── StreamSimpleAgg { aggs: [min(min(max(price))), count] } + ├── tables: [ SimpleAggState: 1, SimpleAggCall0: 0 ] + └── StreamExchange Single from 1 + + Fragment 1 + StreamHashAgg { group_key: [$expr1], aggs: [min(max(price)), count] } + ├── tables: [ HashAggState: 3, HashAggCall0: 2 ] + └── StreamProject { exprs: [id, max(price), Vnode(id) as $expr1] } + └── StreamHashAgg [append_only] { group_key: [id], aggs: [max(price), count] } + ├── tables: [ HashAggState: 4 ] + └── StreamProject { exprs: [id, price, _row_id, _row_id] } + └── StreamFilter { predicate: (date_time >= date_time) AND (date_time <= expires) } + └── StreamHashJoin [append_only] { type: Inner, predicate: id = auction, output: all } + ├── tables: + │ ┌── HashJoinLeft: 5 + │ ├── HashJoinDegreeLeft: 6 + │ ├── HashJoinRight: 7 + │ └── HashJoinDegreeRight: 8 + ├── StreamExchange Hash([0]) from 2 + └── StreamExchange Hash([0]) from 3 + + Fragment 2 + StreamRowIdGen { row_id_index: 13 } + └── StreamSourceScan { columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + ├── tables: [ SourceBackfill: 9 ] + └── Upstream + + Fragment 3 + StreamRowIdGen { row_id_index: 10 } + └── StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } { tables: [ SourceBackfill: 10 ] } + └── Upstream + + Table 0 { columns: [ min(max(price)), $expr1, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + + Table 1 { columns: [ min(min(max(price))), count, _rw_timestamp ], primary key: [], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + + Table 2 + ├── columns: [ $expr1, max(price), id, _rw_timestamp ] + ├── primary key: [ $0 ASC, $1 ASC, $2 ASC ] + ├── value indices: [ 1, 2 ] + ├── distribution key: [ 2 ] + ├── read pk prefix len hint: 1 + └── vnode column idx: 0 + + Table 3 { columns: [ $expr1, min(max(price)), count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [], read pk prefix len hint: 1, vnode column idx: 0 } + + Table 4 { columns: [ id, max(price), count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + + Table 5 + ├── columns: [ id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id, _rw_timestamp ] + ├── primary key: [ $0 ASC, $13 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13 ] + ├── distribution key: [ 0 ] + └── read pk prefix len hint: 1 + + Table 6 { columns: [ id, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + + Table 7 + ├── columns: [ auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id, _rw_timestamp ] + ├── primary key: [ $0 ASC, $10 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10 ] + ├── distribution key: [ 0 ] + └── read pk prefix len hint: 1 + + Table 8 { columns: [ auction, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + + Table 9 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + + Table 10 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + + Table 4294967294 { columns: [ min_final, _rw_timestamp ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } + 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 83fde26bfc7d2..fa75610ff09e3 100644 --- a/src/frontend/planner_test/tests/testdata/output/shared_source.yml +++ b/src/frontend/planner_test/tests/testdata/output/shared_source.yml @@ -68,12 +68,12 @@ batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchProject { exprs: [x, y] } - └─BatchKafkaScan { source: s, columns: [x, y, _rw_kafka_timestamp, _row_id], filter: (None, None) } + └─BatchKafkaScan { source: s, columns: [x, y, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _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] } - └─StreamRowIdGen { row_id_index: 3 } - └─StreamSourceScan { columns: [x, y, _rw_kafka_timestamp, _row_id, _rw_kafka_partition, _rw_kafka_offset] } + └─StreamRowIdGen { row_id_index: 5 } + └─StreamSourceScan { columns: [x, y, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } with_config_map: streaming_use_shared_source: 'true' - before: @@ -84,11 +84,11 @@ batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchProject { exprs: [x, y] } - └─BatchKafkaScan { source: s, columns: [x, y, _rw_kafka_timestamp, _row_id], filter: (None, None) } + └─BatchKafkaScan { source: s, columns: [x, y, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _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] } - └─StreamRowIdGen { row_id_index: 3 } - └─StreamSourceScan { columns: [x, y, _rw_kafka_timestamp, _row_id, _rw_kafka_partition, _rw_kafka_offset] } + └─StreamRowIdGen { row_id_index: 5 } + └─StreamSourceScan { columns: [x, y, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } with_config_map: streaming_use_shared_source: 'true' diff --git a/src/frontend/planner_test/tests/testdata/output/tpch_kafka.yaml b/src/frontend/planner_test/tests/testdata/output/tpch_kafka.yaml new file mode 100644 index 0000000000000..b1f9c6d5dc88c --- /dev/null +++ b/src/frontend/planner_test/tests/testdata/output/tpch_kafka.yaml @@ -0,0 +1,1513 @@ +# This file is automatically generated. See `src/frontend/planner_test/README.md` for more information. +- id: create_tables + sql: | + CREATE SOURCE supplier ( + s_suppkey INTEGER, + s_name VARCHAR, + s_address VARCHAR, + s_nationkey INTEGER, + s_phone VARCHAR, + s_acctbal NUMERIC, + s_comment VARCHAR + ) WITH ( + connector = 'kafka', + topic = 'supplier', + properties.bootstrap.server = 'fake', + ) FORMAT PLAIN ENCODE JSON; + + CREATE SOURCE part ( + p_partkey INTEGER, + p_name VARCHAR, + p_mfgr VARCHAR, + p_brand VARCHAR, + p_type VARCHAR, + p_size INTEGER, + p_container VARCHAR, + p_retailprice NUMERIC, + p_comment VARCHAR + ) WITH ( + connector = 'kafka', + topic = 'part', + properties.bootstrap.server = 'fake', + ) FORMAT PLAIN ENCODE JSON; + + CREATE SOURCE partsupp ( + ps_partkey INTEGER, + ps_suppkey INTEGER, + ps_availqty INTEGER, + ps_supplycost NUMERIC, + ps_comment VARCHAR + ) WITH ( + connector = 'kafka', + topic = 'partsupp', + properties.bootstrap.server = 'fake', + ) FORMAT PLAIN ENCODE JSON; + + CREATE SOURCE customer ( + c_custkey INTEGER, + c_name VARCHAR, + c_address VARCHAR, + c_nationkey INTEGER, + c_phone VARCHAR, + c_acctbal NUMERIC, + c_mktsegment VARCHAR, + c_comment VARCHAR + ) WITH ( + connector = 'kafka', + topic = 'customer', + properties.bootstrap.server = 'fake', + ) FORMAT PLAIN ENCODE JSON; + + CREATE SOURCE orders ( + o_orderkey BIGINT, + o_custkey INTEGER, + o_orderstatus VARCHAR, + o_totalprice NUMERIC, + o_orderdate DATE, + o_orderpriority VARCHAR, + o_clerk VARCHAR, + o_shippriority INTEGER, + o_comment VARCHAR + ) WITH ( + connector = 'kafka', + topic = 'orders', + properties.bootstrap.server = 'fake', + ) FORMAT PLAIN ENCODE JSON; + + CREATE SOURCE lineitem ( + l_orderkey BIGINT, + l_partkey INTEGER, + l_suppkey INTEGER, + l_linenumber INTEGER, + l_quantity NUMERIC, + l_extendedprice NUMERIC, + l_discount NUMERIC, + l_tax NUMERIC, + l_returnflag VARCHAR, + l_linestatus VARCHAR, + l_shipdate DATE, + l_commitdate DATE, + l_receiptdate DATE, + l_shipinstruct VARCHAR, + l_shipmode VARCHAR, + l_comment VARCHAR + ) WITH ( + connector = 'kafka', + topic = 'lineitem', + properties.bootstrap.server = 'fake', + ) FORMAT PLAIN ENCODE JSON; + + CREATE SOURCE nation ( + n_nationkey INTEGER, + n_name VARCHAR, + n_regionkey INTEGER, + n_comment VARCHAR + ) WITH ( + connector = 'kafka', + topic = 'nation', + properties.bootstrap.server = 'fake', + ) FORMAT PLAIN ENCODE JSON; + + CREATE SOURCE region ( + r_regionkey INTEGER, + r_name VARCHAR, + r_comment VARCHAR + ) WITH ( + connector = 'kafka', + topic = 'region', + properties.bootstrap.server = 'fake', + ) FORMAT PLAIN ENCODE JSON; +- id: tpch_q1 + before: + - create_tables + sql: | + select + l_returnflag, + l_linestatus, + sum(l_quantity) as sum_qty, + sum(l_extendedprice) as sum_base_price, + sum(l_extendedprice * (1 - l_discount)) as sum_disc_price, + sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) as sum_charge, + avg(l_quantity) as avg_qty, + avg(l_extendedprice) as avg_price, + avg(l_discount) as avg_disc, + count(*) as count_order + from + lineitem + where + l_shipdate <= date '1998-12-01' - interval '71' day + group by + l_returnflag, + l_linestatus + order by + l_returnflag, + l_linestatus + LIMIT 1; + stream_plan: |- + StreamMaterialize { columns: [l_returnflag, l_linestatus, sum_qty, sum_base_price, sum_disc_price, sum_charge, avg_qty, avg_price, avg_disc, count_order], stream_key: [], pk_columns: [l_returnflag, l_linestatus], pk_conflict: NoCheck } + └─StreamProject { exprs: [l_returnflag, l_linestatus, sum(l_quantity), sum(l_extendedprice), sum($expr1), sum($expr2), $expr3, $expr4, $expr5, count] } + └─StreamTopN { order: [l_returnflag ASC, l_linestatus ASC], limit: 1, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [l_returnflag ASC, l_linestatus ASC], limit: 1, offset: 0, group_key: [$expr6] } + └─StreamProject { exprs: [l_returnflag, l_linestatus, sum(l_quantity), sum(l_extendedprice), sum($expr1), sum($expr2), (sum(l_quantity) / count(l_quantity)::Decimal) as $expr3, (sum(l_extendedprice) / count(l_extendedprice)::Decimal) as $expr4, (sum(l_discount) / count(l_discount)::Decimal) as $expr5, count, Vnode(l_returnflag, l_linestatus) as $expr6] } + └─StreamHashAgg [append_only] { group_key: [l_returnflag, l_linestatus], aggs: [sum(l_quantity), sum(l_extendedprice), sum($expr1), sum($expr2), count(l_quantity), count(l_extendedprice), sum(l_discount), count(l_discount), count] } + └─StreamExchange { dist: HashShard(l_returnflag, l_linestatus) } + └─StreamProject { exprs: [l_returnflag, l_linestatus, l_quantity, l_extendedprice, $expr1, ($expr1 * (1:Decimal + l_tax)) as $expr2, l_discount, _row_id] } + └─StreamProject { 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, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id, (l_extendedprice * (1:Decimal - l_discount)) as $expr1] } + └─StreamFilter { predicate: (l_shipdate <= '1998-09-21 00:00:00':Timestamp) } + └─StreamRowIdGen { row_id_index: 19 } + └─StreamSourceScan { 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, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } +- id: tpch_q2 + before: + - create_tables + sql: | + select + s_acctbal, + s_name, + n_name, + p_partkey, + p_mfgr, + s_address, + s_phone, + s_comment + from + part, + supplier, + partsupp, + nation, + region + where + p_partkey = ps_partkey + and s_suppkey = ps_suppkey + and p_size = 4 + and p_type like '%TIN' + and s_nationkey = n_nationkey + and n_regionkey = r_regionkey + and r_name = 'AFRICA' + and ps_supplycost = ( + select + min(ps_supplycost) + from + partsupp, + supplier, + nation, + region + where + p_partkey = ps_partkey + and s_suppkey = ps_suppkey + and s_nationkey = n_nationkey + and n_regionkey = r_regionkey + and r_name = 'AFRICA' + ) + order by + s_acctbal desc, + n_name, + s_name, + p_partkey + LIMIT 100; + 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 } + └─StreamProject { exprs: [s_acctbal, s_name, n_name, p_partkey, p_mfgr, s_address, s_phone, s_comment, _row_id, min(ps_supplycost), ps_partkey, _row_id, _row_id, r_regionkey, _row_id, n_nationkey, _row_id, s_suppkey] } + └─StreamTopN { order: [s_acctbal DESC, n_name ASC, s_name ASC, p_partkey ASC], limit: 100, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [s_acctbal DESC, n_name ASC, s_name ASC, p_partkey ASC], limit: 100, offset: 0, group_key: [_vnode] } + └─StreamProject { exprs: [s_acctbal, s_name, n_name, p_partkey, p_mfgr, s_address, s_phone, s_comment, _row_id, min(ps_supplycost), ps_partkey, _row_id, _row_id, r_regionkey, _row_id, n_nationkey, _row_id, s_suppkey, Vnode(ps_partkey, min(ps_supplycost)) as _vnode] } + └─StreamHashJoin { type: Inner, predicate: p_partkey = ps_partkey AND min(ps_supplycost) = ps_supplycost AND ps_partkey = ps_partkey, output: [s_acctbal, s_name, n_name, p_partkey, p_mfgr, s_address, s_phone, s_comment, _row_id, min(ps_supplycost), ps_partkey, _row_id, _row_id, r_regionkey, _row_id, n_nationkey, _row_id, s_suppkey] } + ├─StreamExchange { dist: HashShard(ps_partkey, min(ps_supplycost)) } + │ └─StreamHashJoin { type: Inner, predicate: p_partkey = ps_partkey, output: [p_partkey, p_mfgr, ps_partkey, min(ps_supplycost), _row_id] } + │ ├─StreamExchange { dist: HashShard(p_partkey) } + │ │ └─StreamFilter { predicate: (p_size = 4:Int32) AND Like(p_type, '%TIN':Varchar) } + │ │ └─StreamRowIdGen { row_id_index: 12 } + │ │ └─StreamSourceScan { columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ └─StreamProject { exprs: [ps_partkey, min(ps_supplycost)] } + │ └─StreamHashAgg [append_only] { group_key: [ps_partkey], aggs: [min(ps_supplycost), count] } + │ └─StreamExchange { dist: HashShard(ps_partkey) } + │ └─StreamHashJoin [append_only] { type: Inner, predicate: s_nationkey = n_nationkey, output: [ps_supplycost, ps_partkey, _row_id, _row_id, ps_suppkey, s_nationkey, _row_id, _row_id, r_regionkey] } + │ ├─StreamExchange { dist: HashShard(s_nationkey) } + │ │ └─StreamHashJoin [append_only] { type: Inner, predicate: ps_suppkey = s_suppkey, output: [ps_partkey, ps_supplycost, s_nationkey, _row_id, ps_suppkey, _row_id] } + │ │ ├─StreamExchange { dist: HashShard(ps_suppkey) } + │ │ │ └─StreamShare { id: 6 } + │ │ │ └─StreamProject { exprs: [ps_partkey, ps_suppkey, ps_supplycost, _row_id] } + │ │ │ └─StreamRowIdGen { row_id_index: 8 } + │ │ │ └─StreamSourceScan { columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ │ └─StreamExchange { dist: HashShard(s_suppkey) } + │ │ └─StreamShare { id: 10 } + │ │ └─StreamProject { exprs: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } + │ │ └─StreamRowIdGen { row_id_index: 10 } + │ │ └─StreamSourceScan { columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ └─StreamExchange { dist: HashShard(n_nationkey) } + │ └─StreamHashJoin [append_only] { type: Inner, predicate: r_regionkey = n_regionkey, output: [n_nationkey, _row_id, r_regionkey, _row_id] } + │ ├─StreamExchange { dist: HashShard(r_regionkey) } + │ │ └─StreamFilter { predicate: (r_name = 'AFRICA':Varchar) } + │ │ └─StreamShare { id: 17 } + │ │ └─StreamProject { exprs: [r_regionkey, r_name, _row_id] } + │ │ └─StreamFilter { predicate: (r_name = 'AFRICA':Varchar) } + │ │ └─StreamRowIdGen { row_id_index: 6 } + │ │ └─StreamSourceScan { columns: [r_regionkey, r_name, r_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ └─StreamExchange { dist: HashShard(n_regionkey) } + │ └─StreamShare { id: 22 } + │ └─StreamProject { exprs: [n_nationkey, n_name, n_regionkey, _row_id] } + │ └─StreamRowIdGen { row_id_index: 7 } + │ └─StreamSourceScan { columns: [n_nationkey, n_name, n_regionkey, n_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(ps_partkey, ps_supplycost) } + └─StreamHashJoin [append_only] { type: Inner, predicate: s_suppkey = ps_suppkey, output: [n_name, s_name, s_address, s_phone, s_acctbal, s_comment, ps_partkey, ps_supplycost, _row_id, _row_id, r_regionkey, _row_id, n_nationkey, s_suppkey, _row_id] } + ├─StreamExchange { dist: HashShard(s_suppkey) } + │ └─StreamHashJoin [append_only] { type: Inner, predicate: n_nationkey = s_nationkey, output: [n_name, s_suppkey, s_name, s_address, s_phone, s_acctbal, s_comment, _row_id, _row_id, r_regionkey, n_nationkey, _row_id] } + │ ├─StreamExchange { dist: HashShard(n_nationkey) } + │ │ └─StreamHashJoin [append_only] { type: Inner, predicate: r_regionkey = n_regionkey, output: [n_nationkey, n_name, _row_id, r_regionkey, _row_id] } + │ │ ├─StreamExchange { dist: HashShard(r_regionkey) } + │ │ │ └─StreamFilter { predicate: (r_name = 'AFRICA':Varchar) } + │ │ │ └─StreamShare { id: 17 } + │ │ │ └─StreamProject { exprs: [r_regionkey, r_name, _row_id] } + │ │ │ └─StreamFilter { predicate: (r_name = 'AFRICA':Varchar) } + │ │ │ └─StreamRowIdGen { row_id_index: 6 } + │ │ │ └─StreamSourceScan { columns: [r_regionkey, r_name, r_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ │ └─StreamExchange { dist: HashShard(n_regionkey) } + │ │ └─StreamShare { id: 22 } + │ │ └─StreamProject { exprs: [n_nationkey, n_name, n_regionkey, _row_id] } + │ │ └─StreamRowIdGen { row_id_index: 7 } + │ │ └─StreamSourceScan { columns: [n_nationkey, n_name, n_regionkey, n_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ └─StreamExchange { dist: HashShard(s_nationkey) } + │ └─StreamShare { id: 10 } + │ └─StreamProject { exprs: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } + │ └─StreamRowIdGen { row_id_index: 10 } + │ └─StreamSourceScan { columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(ps_suppkey) } + └─StreamFilter { predicate: IsNotNull(ps_partkey) } + └─StreamShare { id: 6 } + └─StreamProject { exprs: [ps_partkey, ps_suppkey, ps_supplycost, _row_id] } + └─StreamRowIdGen { row_id_index: 8 } + └─StreamSourceScan { columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } +- id: tpch_q3 + before: + - create_tables + sql: | + select + l_orderkey, + sum(l_extendedprice * (1 - l_discount)) as revenue, + o_orderdate, + o_shippriority + from + customer, + orders, + lineitem + where + c_mktsegment = 'FURNITURE' + and c_custkey = o_custkey + and l_orderkey = o_orderkey + and o_orderdate < date '1995-03-29' + and l_shipdate > date '1995-03-29' + group by + l_orderkey, + o_orderdate, + o_shippriority + order by + revenue desc, + o_orderdate + LIMIT 10; + stream_plan: |- + StreamMaterialize { columns: [l_orderkey, revenue, o_orderdate, o_shippriority], stream_key: [l_orderkey, o_orderdate, o_shippriority], pk_columns: [revenue, o_orderdate, l_orderkey, o_shippriority], pk_conflict: NoCheck } + └─StreamProject { exprs: [l_orderkey, sum($expr1), o_orderdate, o_shippriority] } + └─StreamTopN { order: [sum($expr1) DESC, o_orderdate ASC], limit: 10, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [sum($expr1) DESC, o_orderdate ASC], limit: 10, offset: 0, group_key: [$expr2] } + └─StreamProject { exprs: [l_orderkey, sum($expr1), o_orderdate, o_shippriority, Vnode(l_orderkey, o_orderdate, o_shippriority) as $expr2] } + └─StreamHashAgg [append_only] { group_key: [l_orderkey, o_orderdate, o_shippriority], aggs: [sum($expr1), count] } + └─StreamExchange { dist: HashShard(l_orderkey, o_orderdate, o_shippriority) } + └─StreamProject { exprs: [l_orderkey, o_orderdate, o_shippriority, (l_extendedprice * (1:Decimal - l_discount)) as $expr1, _row_id, _row_id, c_custkey, _row_id, o_orderkey] } + └─StreamHashJoin [append_only] { type: Inner, predicate: o_orderkey = l_orderkey, output: [o_orderdate, o_shippriority, l_orderkey, l_extendedprice, l_discount, _row_id, _row_id, c_custkey, o_orderkey, _row_id] } + ├─StreamExchange { dist: HashShard(o_orderkey) } + │ └─StreamHashJoin [append_only] { type: Inner, predicate: c_custkey = o_custkey, output: [o_orderkey, o_orderdate, o_shippriority, _row_id, c_custkey, _row_id] } + │ ├─StreamExchange { dist: HashShard(c_custkey) } + │ │ └─StreamFilter { predicate: (c_mktsegment = 'FURNITURE':Varchar) } + │ │ └─StreamRowIdGen { row_id_index: 11 } + │ │ └─StreamSourceScan { columns: [c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ └─StreamExchange { dist: HashShard(o_custkey) } + │ └─StreamFilter { predicate: (o_orderdate < '1995-03-29':Date) } + │ └─StreamRowIdGen { row_id_index: 12 } + │ └─StreamSourceScan { columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(l_orderkey) } + └─StreamFilter { predicate: (l_shipdate > '1995-03-29':Date) } + └─StreamRowIdGen { row_id_index: 19 } + └─StreamSourceScan { 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, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } +- id: tpch_q4 + before: + - create_tables + sql: | + select + o_orderpriority, + count(*) as order_count + from + orders + where + o_orderdate >= date '1997-07-01' + and o_orderdate < date '1997-07-01' + interval '3' month + and exists ( + select + * + from + lineitem + where + l_orderkey = o_orderkey + and l_commitdate < l_receiptdate + ) + group by + o_orderpriority + order by + o_orderpriority + LIMIT 1; + stream_plan: |- + StreamMaterialize { columns: [o_orderpriority, order_count], stream_key: [], pk_columns: [o_orderpriority], pk_conflict: NoCheck } + └─StreamProject { exprs: [o_orderpriority, count] } + └─StreamTopN { order: [o_orderpriority ASC], limit: 1, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [o_orderpriority ASC], limit: 1, offset: 0, group_key: [_vnode] } + └─StreamProject { exprs: [o_orderpriority, count, Vnode(o_orderpriority) as _vnode] } + └─StreamHashAgg { group_key: [o_orderpriority], aggs: [count] } + └─StreamExchange { dist: HashShard(o_orderpriority) } + └─StreamHashJoin { type: LeftSemi, predicate: o_orderkey = l_orderkey, output: [o_orderpriority, _row_id, o_orderkey] } + ├─StreamExchange { dist: HashShard(o_orderkey) } + │ └─StreamFilter { predicate: (o_orderdate >= '1997-07-01':Date) AND (o_orderdate < '1997-10-01 00:00:00':Timestamp) } + │ └─StreamRowIdGen { row_id_index: 12 } + │ └─StreamSourceScan { columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(l_orderkey) } + └─StreamProject { exprs: [l_orderkey, _row_id] } + └─StreamFilter { predicate: (l_commitdate < l_receiptdate) } + └─StreamRowIdGen { row_id_index: 19 } + └─StreamSourceScan { 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, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } +- id: tpch_q5 + before: + - create_tables + sql: | + select + n_name, + sum(l_extendedprice * (1 - l_discount)) as revenue + from + customer, + orders, + lineitem, + supplier, + nation, + region + where + c_custkey = o_custkey + and l_orderkey = o_orderkey + and l_suppkey = s_suppkey + and c_nationkey = s_nationkey + and s_nationkey = n_nationkey + and n_regionkey = r_regionkey + and r_name = 'MIDDLE EAST' + and o_orderdate >= date '1994-01-01' + and o_orderdate < date '1994-01-01' + interval '1' year + group by + n_name + order by + revenue desc + LIMIT 1; + stream_plan: |- + StreamMaterialize { columns: [n_name, revenue], stream_key: [], pk_columns: [revenue], pk_conflict: NoCheck } + └─StreamProject { exprs: [n_name, sum($expr1)] } + └─StreamTopN { order: [sum($expr1) DESC], limit: 1, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [sum($expr1) DESC], limit: 1, offset: 0, group_key: [$expr2] } + └─StreamProject { exprs: [n_name, sum($expr1), Vnode(n_name) as $expr2] } + └─StreamHashAgg [append_only] { group_key: [n_name], aggs: [sum($expr1), count] } + └─StreamExchange { dist: HashShard(n_name) } + └─StreamProject { exprs: [n_name, (l_extendedprice * (1:Decimal - l_discount)) as $expr1, _row_id, _row_id, r_regionkey, _row_id, _row_id, o_custkey, _row_id, _row_id, l_suppkey, o_orderkey, n_nationkey] } + └─StreamHashJoin [append_only] { type: Inner, predicate: n_nationkey = s_nationkey AND n_nationkey = c_nationkey, output: [l_extendedprice, l_discount, n_name, _row_id, _row_id, r_regionkey, n_nationkey, _row_id, _row_id, o_custkey, _row_id, _row_id, l_suppkey, o_orderkey, c_nationkey] } + ├─StreamExchange { dist: HashShard(n_nationkey, n_nationkey) } + │ └─StreamHashJoin [append_only] { type: Inner, predicate: r_regionkey = n_regionkey, output: [n_nationkey, n_name, _row_id, r_regionkey, _row_id] } + │ ├─StreamExchange { dist: HashShard(r_regionkey) } + │ │ └─StreamFilter { predicate: (r_name = 'MIDDLE EAST':Varchar) } + │ │ └─StreamRowIdGen { row_id_index: 6 } + │ │ └─StreamSourceScan { columns: [r_regionkey, r_name, r_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ └─StreamExchange { dist: HashShard(n_regionkey) } + │ └─StreamFilter { predicate: IsNotNull(n_nationkey) } + │ └─StreamRowIdGen { row_id_index: 7 } + │ └─StreamSourceScan { columns: [n_nationkey, n_name, n_regionkey, n_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(c_nationkey, s_nationkey) } + └─StreamHashJoin [append_only] { type: Inner, predicate: o_orderkey = l_orderkey AND c_nationkey = s_nationkey, output: [c_nationkey, l_extendedprice, l_discount, s_nationkey, _row_id, _row_id, o_custkey, o_orderkey, _row_id, _row_id, l_suppkey] } + ├─StreamExchange { dist: HashShard(o_orderkey, c_nationkey) } + │ └─StreamHashJoin [append_only] { type: Inner, predicate: o_custkey = c_custkey, output: [o_orderkey, c_nationkey, _row_id, o_custkey, _row_id] } + │ ├─StreamExchange { dist: HashShard(o_custkey) } + │ │ └─StreamFilter { predicate: (o_orderdate >= '1994-01-01':Date) AND (o_orderdate < '1995-01-01 00:00:00':Timestamp) } + │ │ └─StreamRowIdGen { row_id_index: 12 } + │ │ └─StreamSourceScan { columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ └─StreamExchange { dist: HashShard(c_custkey) } + │ └─StreamRowIdGen { row_id_index: 11 } + │ └─StreamSourceScan { columns: [c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(l_orderkey, s_nationkey) } + └─StreamHashJoin [append_only] { type: Inner, predicate: l_suppkey = s_suppkey, output: [l_orderkey, l_extendedprice, l_discount, s_nationkey, _row_id, l_suppkey, _row_id] } + ├─StreamExchange { dist: HashShard(l_suppkey) } + │ └─StreamRowIdGen { row_id_index: 19 } + │ └─StreamSourceScan { 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, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(s_suppkey) } + └─StreamRowIdGen { row_id_index: 10 } + └─StreamSourceScan { columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } +- id: tpch_q6 + before: + - create_tables + sql: | + select + sum(l_extendedprice * l_discount) as revenue + from + lineitem + where + l_shipdate >= date '1994-01-01' + and l_shipdate < date '1994-01-01' + interval '1' year + and l_discount between 0.08 - 0.01 and 0.08 + 0.01 + and l_quantity < 24; + stream_plan: |- + StreamMaterialize { columns: [revenue], stream_key: [], pk_columns: [], pk_conflict: NoCheck } + └─StreamProject { exprs: [sum(sum($expr1))] } + └─StreamSimpleAgg [append_only] { aggs: [sum(sum($expr1)), count] } + └─StreamExchange { dist: Single } + └─StreamStatelessSimpleAgg { aggs: [sum($expr1)] } + └─StreamProject { exprs: [(l_extendedprice * l_discount) as $expr1, _row_id] } + └─StreamFilter { predicate: (l_shipdate >= '1994-01-01':Date) AND (l_shipdate < '1995-01-01 00:00:00':Timestamp) AND (l_discount >= 0.07:Decimal) AND (l_discount <= 0.09:Decimal) AND (l_quantity < 24:Decimal) } + └─StreamRowIdGen { row_id_index: 19 } + └─StreamSourceScan { 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, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } +- id: tpch_q7 + before: + - create_tables + sql: | + select + supp_nation, + cust_nation, + l_year, + sum(volume) as revenue + from + ( + select + n1.n_name as supp_nation, + n2.n_name as cust_nation, + extract(year from l_shipdate) as l_year, + l_extendedprice * (1 - l_discount) as volume + from + supplier, + lineitem, + orders, + customer, + nation n1, + nation n2 + where + s_suppkey = l_suppkey + and o_orderkey = l_orderkey + and c_custkey = o_custkey + and s_nationkey = n1.n_nationkey + and c_nationkey = n2.n_nationkey + and ( + (n1.n_name = 'ROMANIA' and n2.n_name = 'IRAN') + or (n1.n_name = 'IRAN' and n2.n_name = 'ROMANIA') + ) + and l_shipdate between date '1983-01-01' and date '2000-12-31' + ) as shipping + group by + supp_nation, + cust_nation, + l_year + order by + supp_nation, + cust_nation, + l_year + LIMIT 1; + stream_plan: |- + StreamMaterialize { columns: [supp_nation, cust_nation, l_year, revenue], stream_key: [], pk_columns: [supp_nation, cust_nation, l_year], pk_conflict: NoCheck } + └─StreamProject { exprs: [n_name, n_name, $expr1, sum($expr2)] } + └─StreamTopN { order: [n_name ASC, n_name ASC, $expr1 ASC], limit: 1, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [n_name ASC, n_name ASC, $expr1 ASC], limit: 1, offset: 0, group_key: [$expr3] } + └─StreamProject { exprs: [n_name, n_name, $expr1, sum($expr2), Vnode(n_name, n_name, $expr1) as $expr3] } + └─StreamHashAgg [append_only] { group_key: [n_name, n_name, $expr1], aggs: [sum($expr2), count] } + └─StreamExchange { dist: HashShard(n_name, n_name, $expr1) } + └─StreamProject { exprs: [n_name, n_name, Extract('YEAR':Varchar, l_shipdate) as $expr1, (l_extendedprice * (1:Decimal - l_discount)) as $expr2, _row_id, _row_id, n_nationkey, _row_id, s_suppkey, _row_id, _row_id, n_nationkey, _row_id, c_custkey, l_orderkey] } + └─StreamFilter { predicate: (((n_name = 'ROMANIA':Varchar) AND (n_name = 'IRAN':Varchar)) OR ((n_name = 'IRAN':Varchar) AND (n_name = 'ROMANIA':Varchar))) } + └─StreamHashJoin [append_only] { type: Inner, predicate: l_orderkey = o_orderkey, output: all } + ├─StreamExchange { dist: HashShard(l_orderkey) } + │ └─StreamHashJoin [append_only] { type: Inner, predicate: s_suppkey = l_suppkey, output: [n_name, l_orderkey, l_extendedprice, l_discount, l_shipdate, _row_id, _row_id, n_nationkey, s_suppkey, _row_id] } + │ ├─StreamExchange { dist: HashShard(s_suppkey) } + │ │ └─StreamHashJoin [append_only] { type: Inner, predicate: n_nationkey = s_nationkey, output: [n_name, s_suppkey, _row_id, n_nationkey, _row_id] } + │ │ ├─StreamExchange { dist: HashShard(n_nationkey) } + │ │ │ └─StreamShare { id: 3 } + │ │ │ └─StreamProject { exprs: [n_nationkey, n_name, _row_id] } + │ │ │ └─StreamRowIdGen { row_id_index: 7 } + │ │ │ └─StreamSourceScan { columns: [n_nationkey, n_name, n_regionkey, n_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ │ └─StreamExchange { dist: HashShard(s_nationkey) } + │ │ └─StreamRowIdGen { row_id_index: 10 } + │ │ └─StreamSourceScan { columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ └─StreamExchange { dist: HashShard(l_suppkey) } + │ └─StreamFilter { predicate: (l_shipdate >= '1983-01-01':Date) AND (l_shipdate <= '2000-12-31':Date) } + │ └─StreamRowIdGen { row_id_index: 19 } + │ └─StreamSourceScan { 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, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(o_orderkey) } + └─StreamHashJoin [append_only] { type: Inner, predicate: c_custkey = o_custkey, output: [n_name, o_orderkey, _row_id, _row_id, n_nationkey, c_custkey, _row_id] } + ├─StreamExchange { dist: HashShard(c_custkey) } + │ └─StreamHashJoin [append_only] { type: Inner, predicate: n_nationkey = c_nationkey, output: [n_name, c_custkey, _row_id, n_nationkey, _row_id] } + │ ├─StreamExchange { dist: HashShard(n_nationkey) } + │ │ └─StreamShare { id: 3 } + │ │ └─StreamProject { exprs: [n_nationkey, n_name, _row_id] } + │ │ └─StreamRowIdGen { row_id_index: 7 } + │ │ └─StreamSourceScan { columns: [n_nationkey, n_name, n_regionkey, n_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ └─StreamExchange { dist: HashShard(c_nationkey) } + │ └─StreamRowIdGen { row_id_index: 11 } + │ └─StreamSourceScan { columns: [c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(o_custkey) } + └─StreamRowIdGen { row_id_index: 12 } + └─StreamSourceScan { columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } +- id: tpch_q8 + before: + - create_tables + sql: | + select + o_year, + sum(case + when nation = 'IRAN' then volume + else 0 + end) / sum(volume) as mkt_share + from + ( + select + extract(year from o_orderdate) as o_year, + l_extendedprice * (1 - l_discount) as volume, + n2.n_name as nation + from + part, + supplier, + lineitem, + orders, + customer, + nation n1, + nation n2, + region + where + p_partkey = l_partkey + and s_suppkey = l_suppkey + and l_orderkey = o_orderkey + and o_custkey = c_custkey + and c_nationkey = n1.n_nationkey + and n1.n_regionkey = r_regionkey + and r_name = 'ASIA' + and s_nationkey = n2.n_nationkey + and o_orderdate between date '1995-01-01' and date '1996-12-31' + and p_type = 'PROMO ANODIZED STEEL' + ) as all_nations + group by + o_year + order by + o_year + LIMIT 1; + stream_plan: |- + StreamMaterialize { columns: [o_year, mkt_share], stream_key: [], pk_columns: [o_year], pk_conflict: NoCheck } + └─StreamProject { exprs: [$expr1, $expr4] } + └─StreamTopN { order: [$expr1 ASC], limit: 1, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [$expr1 ASC], limit: 1, offset: 0, group_key: [$expr5] } + └─StreamProject { exprs: [$expr1, (sum($expr3) / sum($expr2)) as $expr4, Vnode($expr1) as $expr5] } + └─StreamHashAgg [append_only] { group_key: [$expr1], aggs: [sum($expr3), sum($expr2), count] } + └─StreamExchange { dist: HashShard($expr1) } + └─StreamProject { exprs: [$expr1, Case((n_name = 'IRAN':Varchar), $expr2, 0:Decimal) as $expr3, $expr2, _row_id, _row_id, r_regionkey, _row_id, n_nationkey, _row_id, _row_id, n_nationkey, _row_id, _row_id, p_partkey, s_suppkey, _row_id, l_orderkey, c_custkey] } + └─StreamProject { exprs: [Extract('YEAR':Varchar, o_orderdate) as $expr1, (l_extendedprice * (1:Decimal - l_discount)) as $expr2, n_name, _row_id, _row_id, r_regionkey, _row_id, n_nationkey, _row_id, _row_id, n_nationkey, _row_id, _row_id, p_partkey, s_suppkey, _row_id, l_orderkey, c_custkey] } + └─StreamHashJoin [append_only] { type: Inner, predicate: c_custkey = o_custkey, output: [l_extendedprice, l_discount, o_orderdate, n_name, _row_id, _row_id, r_regionkey, _row_id, n_nationkey, c_custkey, _row_id, _row_id, n_nationkey, _row_id, _row_id, p_partkey, s_suppkey, _row_id, l_orderkey] } + ├─StreamExchange { dist: HashShard(c_custkey) } + │ └─StreamHashJoin [append_only] { type: Inner, predicate: n_nationkey = c_nationkey, output: [c_custkey, _row_id, _row_id, r_regionkey, n_nationkey, _row_id] } + │ ├─StreamExchange { dist: HashShard(n_nationkey) } + │ │ └─StreamHashJoin [append_only] { type: Inner, predicate: r_regionkey = n_regionkey, output: [n_nationkey, _row_id, r_regionkey, _row_id] } + │ │ ├─StreamExchange { dist: HashShard(r_regionkey) } + │ │ │ └─StreamFilter { predicate: (r_name = 'ASIA':Varchar) } + │ │ │ └─StreamRowIdGen { row_id_index: 6 } + │ │ │ └─StreamSourceScan { columns: [r_regionkey, r_name, r_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ │ └─StreamExchange { dist: HashShard(n_regionkey) } + │ │ └─StreamShare { id: 6 } + │ │ └─StreamProject { exprs: [n_nationkey, n_name, n_regionkey, _row_id] } + │ │ └─StreamRowIdGen { row_id_index: 7 } + │ │ └─StreamSourceScan { columns: [n_nationkey, n_name, n_regionkey, n_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ └─StreamExchange { dist: HashShard(c_nationkey) } + │ └─StreamRowIdGen { row_id_index: 11 } + │ └─StreamSourceScan { columns: [c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(o_custkey) } + └─StreamHashJoin [append_only] { type: Inner, predicate: l_orderkey = o_orderkey, output: [n_name, l_extendedprice, l_discount, o_custkey, o_orderdate, _row_id, _row_id, n_nationkey, _row_id, _row_id, p_partkey, s_suppkey, l_orderkey, _row_id] } + ├─StreamExchange { dist: HashShard(l_orderkey) } + │ └─StreamHashJoin [append_only] { type: Inner, predicate: s_suppkey = l_suppkey, output: [n_name, l_orderkey, l_extendedprice, l_discount, _row_id, _row_id, n_nationkey, s_suppkey, _row_id, _row_id, p_partkey] } + │ ├─StreamExchange { dist: HashShard(s_suppkey) } + │ │ └─StreamHashJoin [append_only] { type: Inner, predicate: n_nationkey = s_nationkey, output: [n_name, s_suppkey, _row_id, n_nationkey, _row_id] } + │ │ ├─StreamExchange { dist: HashShard(n_nationkey) } + │ │ │ └─StreamShare { id: 6 } + │ │ │ └─StreamProject { exprs: [n_nationkey, n_name, n_regionkey, _row_id] } + │ │ │ └─StreamRowIdGen { row_id_index: 7 } + │ │ │ └─StreamSourceScan { columns: [n_nationkey, n_name, n_regionkey, n_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ │ └─StreamExchange { dist: HashShard(s_nationkey) } + │ │ └─StreamRowIdGen { row_id_index: 10 } + │ │ └─StreamSourceScan { columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ └─StreamExchange { dist: HashShard(l_suppkey) } + │ └─StreamHashJoin [append_only] { type: Inner, predicate: p_partkey = l_partkey, output: [l_orderkey, l_suppkey, l_extendedprice, l_discount, _row_id, p_partkey, _row_id] } + │ ├─StreamExchange { dist: HashShard(p_partkey) } + │ │ └─StreamFilter { predicate: (p_type = 'PROMO ANODIZED STEEL':Varchar) } + │ │ └─StreamRowIdGen { row_id_index: 12 } + │ │ └─StreamSourceScan { columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ └─StreamExchange { dist: HashShard(l_partkey) } + │ └─StreamRowIdGen { row_id_index: 19 } + │ └─StreamSourceScan { 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, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(o_orderkey) } + └─StreamFilter { predicate: (o_orderdate >= '1995-01-01':Date) AND (o_orderdate <= '1996-12-31':Date) } + └─StreamRowIdGen { row_id_index: 12 } + └─StreamSourceScan { columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } +- id: tpch_q9 + before: + - create_tables + sql: | + select + nation, + o_year, + sum(amount) as sum_profit + from + ( + select + n_name as nation, + extract(year from o_orderdate) as o_year, + l_extendedprice * (1 - l_discount) - ps_supplycost * l_quantity as amount + from + part, + supplier, + lineitem, + partsupp, + orders, + nation + where + s_suppkey = l_suppkey + and ps_suppkey = l_suppkey + and ps_partkey = l_partkey + and p_partkey = l_partkey + and o_orderkey = l_orderkey + and s_nationkey = n_nationkey + and p_name like '%yellow%' + ) as profit + group by + nation, + o_year + order by + nation, + o_year desc + LIMIT 1; + stream_plan: |- + StreamMaterialize { columns: [nation, o_year, sum_profit], stream_key: [], pk_columns: [nation, o_year], pk_conflict: NoCheck } + └─StreamProject { exprs: [n_name, $expr1, sum($expr2)] } + └─StreamTopN { order: [n_name ASC, $expr1 DESC], limit: 1, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [n_name ASC, $expr1 DESC], limit: 1, offset: 0, group_key: [$expr3] } + └─StreamProject { exprs: [n_name, $expr1, sum($expr2), Vnode(n_name, $expr1) as $expr3] } + └─StreamHashAgg [append_only] { group_key: [n_name, $expr1], aggs: [sum($expr2), count] } + └─StreamExchange { dist: HashShard(n_name, $expr1) } + └─StreamProject { exprs: [n_name, Extract('YEAR':Varchar, o_orderdate) as $expr1, ((l_extendedprice * (1:Decimal - l_discount)) - (ps_supplycost * l_quantity)) as $expr2, _row_id, _row_id, p_partkey, _row_id, _row_id, n_nationkey, _row_id, _row_id, o_orderkey, ps_suppkey, ps_partkey] } + └─StreamHashJoin [append_only] { type: Inner, predicate: p_partkey = l_partkey AND ps_suppkey = l_suppkey AND ps_partkey = l_partkey AND ps_suppkey = s_suppkey, output: [l_quantity, l_extendedprice, l_discount, ps_supplycost, o_orderdate, n_name, _row_id, _row_id, p_partkey, ps_suppkey, ps_partkey, _row_id, _row_id, n_nationkey, _row_id, _row_id, o_orderkey, s_suppkey] } + ├─StreamExchange { dist: HashShard(ps_suppkey) } + │ └─StreamHashJoin [append_only] { type: Inner, predicate: p_partkey = ps_partkey, output: [p_partkey, ps_partkey, ps_suppkey, ps_supplycost, _row_id, _row_id] } + │ ├─StreamExchange { dist: HashShard(p_partkey) } + │ │ └─StreamFilter { predicate: Like(p_name, '%yellow%':Varchar) } + │ │ └─StreamRowIdGen { row_id_index: 12 } + │ │ └─StreamSourceScan { columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ └─StreamExchange { dist: HashShard(ps_partkey) } + │ └─StreamFilter { predicate: IsNotNull(ps_suppkey) } + │ └─StreamRowIdGen { row_id_index: 8 } + │ └─StreamSourceScan { columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamHashJoin [append_only] { type: Inner, predicate: s_suppkey = l_suppkey, output: [n_name, s_suppkey, o_orderdate, l_partkey, l_suppkey, l_quantity, l_extendedprice, l_discount, _row_id, _row_id, n_nationkey, _row_id, _row_id, o_orderkey] } + ├─StreamExchange { dist: HashShard(s_suppkey) } + │ └─StreamHashJoin [append_only] { type: Inner, predicate: n_nationkey = s_nationkey, output: [n_name, s_suppkey, _row_id, n_nationkey, _row_id] } + │ ├─StreamExchange { dist: HashShard(n_nationkey) } + │ │ └─StreamRowIdGen { row_id_index: 7 } + │ │ └─StreamSourceScan { columns: [n_nationkey, n_name, n_regionkey, n_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ └─StreamExchange { dist: HashShard(s_nationkey) } + │ └─StreamRowIdGen { row_id_index: 10 } + │ └─StreamSourceScan { columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(l_suppkey) } + └─StreamHashJoin [append_only] { type: Inner, predicate: o_orderkey = l_orderkey, output: [o_orderdate, l_partkey, l_suppkey, l_quantity, l_extendedprice, l_discount, _row_id, o_orderkey, _row_id] } + ├─StreamExchange { dist: HashShard(o_orderkey) } + │ └─StreamRowIdGen { row_id_index: 12 } + │ └─StreamSourceScan { columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(l_orderkey) } + └─StreamFilter { predicate: IsNotNull(l_partkey) } + └─StreamRowIdGen { row_id_index: 19 } + └─StreamSourceScan { 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, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } +- id: tpch_q10 + before: + - create_tables + sql: | + select + c_custkey, + c_name, + sum(l_extendedprice * (1.00 - l_discount)) as revenue, + c_acctbal, + n_name, + c_address, + c_phone, + c_comment + from + customer, + orders, + lineitem, + nation + where + c_custkey = o_custkey + and l_orderkey = o_orderkey + and o_orderdate >= date '1994-01-01' + and o_orderdate < date '1994-01-01' + interval '3' month + and l_returnflag = 'R' + and c_nationkey = n_nationkey + group by + c_custkey, + c_name, + c_acctbal, + c_phone, + n_name, + c_address, + c_comment + order by + revenue desc + LIMIT 20; + stream_plan: |- + StreamMaterialize { columns: [c_custkey, c_name, revenue, c_acctbal, n_name, c_address, c_phone, c_comment], stream_key: [c_custkey, c_name, c_acctbal, c_phone, n_name, c_address, c_comment], pk_columns: [revenue, c_custkey, c_name, c_acctbal, c_phone, n_name, c_address, c_comment], pk_conflict: NoCheck } + └─StreamProject { exprs: [c_custkey, c_name, sum($expr1), c_acctbal, n_name, c_address, c_phone, c_comment] } + └─StreamTopN { order: [sum($expr1) DESC], limit: 20, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [sum($expr1) DESC], limit: 20, offset: 0, group_key: [$expr2] } + └─StreamProject { exprs: [c_custkey, c_name, sum($expr1), c_acctbal, n_name, c_address, c_phone, c_comment, Vnode(c_custkey) as $expr2] } + └─StreamHashAgg [append_only] { group_key: [c_custkey, c_name, c_acctbal, c_phone, n_name, c_address, c_comment], aggs: [sum($expr1), count] } + └─StreamProject { exprs: [c_custkey, c_name, c_acctbal, c_phone, n_name, c_address, c_comment, (l_extendedprice * (1.00:Decimal - l_discount)) as $expr1, _row_id, _row_id, n_nationkey, _row_id, _row_id, l_orderkey] } + └─StreamHashJoin [append_only] { type: Inner, predicate: c_custkey = o_custkey, output: [c_custkey, c_name, c_address, c_phone, c_acctbal, c_comment, l_extendedprice, l_discount, n_name, _row_id, _row_id, n_nationkey, _row_id, _row_id, l_orderkey] } + ├─StreamExchange { dist: HashShard(c_custkey) } + │ └─StreamHashJoin [append_only] { type: Inner, predicate: n_nationkey = c_nationkey, output: [n_name, c_custkey, c_name, c_address, c_phone, c_acctbal, c_comment, _row_id, n_nationkey, _row_id] } + │ ├─StreamExchange { dist: HashShard(n_nationkey) } + │ │ └─StreamRowIdGen { row_id_index: 7 } + │ │ └─StreamSourceScan { columns: [n_nationkey, n_name, n_regionkey, n_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ └─StreamExchange { dist: HashShard(c_nationkey) } + │ └─StreamRowIdGen { row_id_index: 11 } + │ └─StreamSourceScan { columns: [c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(o_custkey) } + └─StreamHashJoin [append_only] { type: Inner, predicate: l_orderkey = o_orderkey, output: [l_extendedprice, l_discount, o_custkey, _row_id, l_orderkey, _row_id] } + ├─StreamExchange { dist: HashShard(l_orderkey) } + │ └─StreamFilter { predicate: (l_returnflag = 'R':Varchar) } + │ └─StreamRowIdGen { row_id_index: 19 } + │ └─StreamSourceScan { 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, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(o_orderkey) } + └─StreamFilter { predicate: (o_orderdate >= '1994-01-01':Date) AND (o_orderdate < '1994-04-01 00:00:00':Timestamp) } + └─StreamRowIdGen { row_id_index: 12 } + └─StreamSourceScan { columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } +- id: tpch_q11 + before: + - create_tables + sql: | + select + ps_partkey, + sum(ps_supplycost * ps_availqty) as value + from + partsupp, + supplier, + nation + where + ps_suppkey = s_suppkey + and s_nationkey = n_nationkey + and n_name = 'ARGENTINA' + group by + ps_partkey + having + sum(ps_supplycost * ps_availqty) > ( + select + sum(ps_supplycost * ps_availqty) * 0.0001000000 + from + partsupp, + supplier, + nation + where + ps_suppkey = s_suppkey + and s_nationkey = n_nationkey + and n_name = 'ARGENTINA' + ) + order by + value desc + LIMIT 1; + stream_plan: |- + StreamMaterialize { columns: [ps_partkey, value], stream_key: [], pk_columns: [value], pk_conflict: NoCheck } + └─StreamProject { exprs: [ps_partkey, sum($expr1)] } + └─StreamTopN { order: [sum($expr1) DESC], limit: 1, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [sum($expr1) DESC], limit: 1, offset: 0, group_key: [_vnode] } + └─StreamProject { exprs: [ps_partkey, sum($expr1), Vnode(ps_partkey) as _vnode] } + └─StreamDynamicFilter { predicate: (sum($expr1) > $expr3), output: [ps_partkey, sum($expr1)] } + ├─StreamProject { exprs: [ps_partkey, sum($expr1)] } + │ └─StreamHashAgg [append_only] { group_key: [ps_partkey], aggs: [sum($expr1), count] } + │ └─StreamExchange { dist: HashShard(ps_partkey) } + │ └─StreamProject { exprs: [ps_partkey, (ps_supplycost * ps_availqty::Decimal) as $expr1, _row_id, _row_id, ps_suppkey, _row_id, s_nationkey] } + │ └─StreamShare { id: 11 } + │ └─StreamHashJoin [append_only] { type: Inner, predicate: s_nationkey = n_nationkey, output: [ps_partkey, ps_availqty, ps_supplycost, _row_id, _row_id, ps_suppkey, s_nationkey, _row_id] } + │ ├─StreamExchange { dist: HashShard(s_nationkey) } + │ │ └─StreamHashJoin [append_only] { type: Inner, predicate: ps_suppkey = s_suppkey, output: [ps_partkey, ps_availqty, ps_supplycost, s_nationkey, _row_id, ps_suppkey, _row_id] } + │ │ ├─StreamExchange { dist: HashShard(ps_suppkey) } + │ │ │ └─StreamRowIdGen { row_id_index: 8 } + │ │ │ └─StreamSourceScan { columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ │ └─StreamExchange { dist: HashShard(s_suppkey) } + │ │ └─StreamRowIdGen { row_id_index: 10 } + │ │ └─StreamSourceScan { columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ └─StreamExchange { dist: HashShard(n_nationkey) } + │ └─StreamFilter { predicate: (n_name = 'ARGENTINA':Varchar) } + │ └─StreamRowIdGen { row_id_index: 7 } + │ └─StreamSourceScan { columns: [n_nationkey, n_name, n_regionkey, n_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: Broadcast } + └─StreamProject { exprs: [(sum(sum($expr2)) * 0.0001000000:Decimal) as $expr3] } + └─StreamSimpleAgg [append_only] { aggs: [sum(sum($expr2)), count] } + └─StreamExchange { dist: Single } + └─StreamStatelessSimpleAgg { aggs: [sum($expr2)] } + └─StreamProject { exprs: [(ps_supplycost * ps_availqty::Decimal) as $expr2, _row_id, _row_id, ps_suppkey, _row_id, s_nationkey] } + └─StreamShare { id: 11 } + └─StreamHashJoin [append_only] { type: Inner, predicate: s_nationkey = n_nationkey, output: [ps_partkey, ps_availqty, ps_supplycost, _row_id, _row_id, ps_suppkey, s_nationkey, _row_id] } + ├─StreamExchange { dist: HashShard(s_nationkey) } + │ └─StreamHashJoin [append_only] { type: Inner, predicate: ps_suppkey = s_suppkey, output: [ps_partkey, ps_availqty, ps_supplycost, s_nationkey, _row_id, ps_suppkey, _row_id] } + │ ├─StreamExchange { dist: HashShard(ps_suppkey) } + │ │ └─StreamRowIdGen { row_id_index: 8 } + │ │ └─StreamSourceScan { columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ └─StreamExchange { dist: HashShard(s_suppkey) } + │ └─StreamRowIdGen { row_id_index: 10 } + │ └─StreamSourceScan { columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(n_nationkey) } + └─StreamFilter { predicate: (n_name = 'ARGENTINA':Varchar) } + └─StreamRowIdGen { row_id_index: 7 } + └─StreamSourceScan { columns: [n_nationkey, n_name, n_regionkey, n_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } +- id: tpch_q12 + before: + - create_tables + sql: | + select + l_shipmode, + sum(case + when o_orderpriority = '1-URGENT' + or o_orderpriority = '2-HIGH' + then 1 + else 0 + end) as high_line_count, + sum(case + when o_orderpriority <> '1-URGENT' + and o_orderpriority <> '2-HIGH' + then 1 + else 0 + end) as low_line_count + from + orders, + lineitem + where + o_orderkey = l_orderkey + and l_shipmode in ('FOB', 'SHIP') + and l_commitdate < l_receiptdate + and l_shipdate < l_commitdate + and l_receiptdate >= date '1994-01-01' + and l_receiptdate < date '1994-01-01' + interval '1' year + group by + l_shipmode + order by + l_shipmode + LIMIT 1; + stream_plan: |- + StreamMaterialize { columns: [l_shipmode, high_line_count, low_line_count], stream_key: [], pk_columns: [l_shipmode], pk_conflict: NoCheck } + └─StreamProject { exprs: [l_shipmode, sum($expr1), sum($expr2)] } + └─StreamTopN { order: [l_shipmode ASC], limit: 1, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [l_shipmode ASC], limit: 1, offset: 0, group_key: [$expr3] } + └─StreamProject { exprs: [l_shipmode, sum($expr1), sum($expr2), Vnode(l_shipmode) as $expr3] } + └─StreamHashAgg [append_only] { group_key: [l_shipmode], aggs: [sum($expr1), sum($expr2), count] } + └─StreamExchange { dist: HashShard(l_shipmode) } + └─StreamProject { exprs: [l_shipmode, Case(((o_orderpriority = '1-URGENT':Varchar) OR (o_orderpriority = '2-HIGH':Varchar)), 1:Int32, 0:Int32) as $expr1, Case(((o_orderpriority <> '1-URGENT':Varchar) AND (o_orderpriority <> '2-HIGH':Varchar)), 1:Int32, 0:Int32) as $expr2, _row_id, _row_id, o_orderkey] } + └─StreamHashJoin [append_only] { type: Inner, predicate: o_orderkey = l_orderkey, output: [o_orderpriority, l_shipmode, _row_id, o_orderkey, _row_id] } + ├─StreamExchange { dist: HashShard(o_orderkey) } + │ └─StreamRowIdGen { row_id_index: 12 } + │ └─StreamSourceScan { columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(l_orderkey) } + └─StreamFilter { predicate: In(l_shipmode, 'FOB':Varchar, 'SHIP':Varchar) AND (l_commitdate < l_receiptdate) AND (l_shipdate < l_commitdate) AND (l_receiptdate >= '1994-01-01':Date) AND (l_receiptdate < '1995-01-01 00:00:00':Timestamp) } + └─StreamRowIdGen { row_id_index: 19 } + └─StreamSourceScan { 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, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } +- id: tpch_q13 + before: + - create_tables + sql: | + select + c_count, + count(*) as custdist + from + ( + select + c_custkey, + count(o_orderkey) as c_count + from + customer left outer join orders on + c_custkey = o_custkey + and o_comment not like '%:1%:2%' + group by + c_custkey + ) as c_orders (c_custkey, c_count) + group by + c_count + order by + custdist desc, + c_count desc + LIMIT 1; + stream_plan: |- + StreamMaterialize { columns: [c_count, custdist], stream_key: [], pk_columns: [custdist, c_count], pk_conflict: NoCheck } + └─StreamProject { exprs: [count(o_orderkey), count] } + └─StreamTopN { order: [count DESC, count(o_orderkey) DESC], limit: 1, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [count DESC, count(o_orderkey) DESC], limit: 1, offset: 0, group_key: [_vnode] } + └─StreamProject { exprs: [count(o_orderkey), count, Vnode(count(o_orderkey)) as _vnode] } + └─StreamHashAgg { group_key: [count(o_orderkey)], aggs: [count] } + └─StreamExchange { dist: HashShard(count(o_orderkey)) } + └─StreamProject { exprs: [c_custkey, count(o_orderkey)] } + └─StreamHashAgg { group_key: [c_custkey], aggs: [count(o_orderkey), count] } + └─StreamHashJoin { type: LeftOuter, predicate: c_custkey = o_custkey, output: [c_custkey, o_orderkey, _row_id, _row_id] } + ├─StreamExchange { dist: HashShard(c_custkey) } + │ └─StreamRowIdGen { row_id_index: 11 } + │ └─StreamSourceScan { columns: [c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(o_custkey) } + └─StreamProject { exprs: [o_orderkey, o_custkey, _row_id] } + └─StreamFilter { predicate: Not(Like(o_comment, '%:1%:2%':Varchar)) } + └─StreamRowIdGen { row_id_index: 12 } + └─StreamSourceScan { columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } +- id: tpch_q14 + before: + - create_tables + sql: | + select + 100.00 * sum(case + when p_type like 'PROMO%' + then l_extendedprice * (1 - l_discount) + else 0 + end) / sum(l_extendedprice * (1 - l_discount)) as promo_revenue + from + lineitem, + part + where + l_partkey = p_partkey + and l_shipdate >= date '1995-09-01' + and l_shipdate < date '1995-09-01' + interval '1' month; + stream_plan: |- + StreamMaterialize { columns: [promo_revenue], stream_key: [], pk_columns: [], pk_conflict: NoCheck } + └─StreamProject { exprs: [((100.00:Decimal * sum(sum($expr1))) / sum(sum($expr2))) as $expr3] } + └─StreamSimpleAgg [append_only] { aggs: [sum(sum($expr1)), sum(sum($expr2)), count] } + └─StreamExchange { dist: Single } + └─StreamStatelessSimpleAgg { aggs: [sum($expr1), sum($expr2)] } + └─StreamProject { exprs: [Case(Like(p_type, 'PROMO%':Varchar), (l_extendedprice * (1:Decimal - l_discount)), 0:Decimal) as $expr1, (l_extendedprice * (1:Decimal - l_discount)) as $expr2, _row_id, _row_id, l_partkey] } + └─StreamHashJoin [append_only] { type: Inner, predicate: l_partkey = p_partkey, output: [l_extendedprice, l_discount, p_type, _row_id, l_partkey, _row_id] } + ├─StreamExchange { dist: HashShard(l_partkey) } + │ └─StreamFilter { predicate: (l_shipdate >= '1995-09-01':Date) AND (l_shipdate < '1995-10-01 00:00:00':Timestamp) } + │ └─StreamRowIdGen { row_id_index: 19 } + │ └─StreamSourceScan { 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, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(p_partkey) } + └─StreamRowIdGen { row_id_index: 12 } + └─StreamSourceScan { columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } +- id: tpch_q15 + before: + - create_tables + sql: | + with revenue0 (supplier_no, total_revenue) as ( + select + l_suppkey, + sum(l_extendedprice * (1 - l_discount)) + from + lineitem + where + l_shipdate >= date '1993-01-01' + and l_shipdate < date '1993-01-01' + interval '3' month + group by + l_suppkey + ) + select + s_suppkey, + s_name, + s_address, + s_phone, + total_revenue + from + supplier, + revenue0 + where + s_suppkey = supplier_no + and total_revenue = ( + select + max(total_revenue) + from + revenue0 + ) + order by + s_suppkey + LIMIT 1; + stream_plan: |- + StreamMaterialize { columns: [s_suppkey, s_name, s_address, s_phone, total_revenue, _row_id(hidden)], stream_key: [], pk_columns: [s_suppkey], pk_conflict: NoCheck } + └─StreamProject { exprs: [s_suppkey, s_name, s_address, s_phone, sum($expr1), _row_id] } + └─StreamTopN { order: [s_suppkey ASC], limit: 1, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [s_suppkey ASC], limit: 1, offset: 0, group_key: [_vnode] } + └─StreamProject { exprs: [s_suppkey, s_name, s_address, s_phone, sum($expr1), _row_id, Vnode(sum($expr1)) as _vnode] } + └─StreamHashJoin { type: Inner, predicate: sum($expr1) = max(max(sum($expr1))), output: [s_suppkey, s_name, s_address, s_phone, sum($expr1), _row_id] } + ├─StreamExchange { dist: HashShard(sum($expr1)) } + │ └─StreamHashJoin { type: Inner, predicate: s_suppkey = l_suppkey, output: [s_suppkey, s_name, s_address, s_phone, sum($expr1), _row_id, l_suppkey] } + │ ├─StreamExchange { dist: HashShard(s_suppkey) } + │ │ └─StreamRowIdGen { row_id_index: 10 } + │ │ └─StreamSourceScan { columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ └─StreamShare { id: 9 } + │ └─StreamProject { exprs: [l_suppkey, sum($expr1)] } + │ └─StreamHashAgg [append_only] { group_key: [l_suppkey], aggs: [sum($expr1), count] } + │ └─StreamExchange { dist: HashShard(l_suppkey) } + │ └─StreamProject { exprs: [l_suppkey, (l_extendedprice * (1:Decimal - l_discount)) as $expr1, _row_id] } + │ └─StreamFilter { predicate: (l_shipdate >= '1993-01-01':Date) AND (l_shipdate < '1993-04-01 00:00:00':Timestamp) } + │ └─StreamRowIdGen { row_id_index: 19 } + │ └─StreamSourceScan { 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, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(max(max(sum($expr1)))) } + └─StreamProject { exprs: [max(max(sum($expr1)))] } + └─StreamSimpleAgg { aggs: [max(max(sum($expr1))), count] } + └─StreamExchange { dist: Single } + └─StreamHashAgg { group_key: [_vnode], aggs: [max(sum($expr1)), count] } + └─StreamProject { exprs: [l_suppkey, sum($expr1), Vnode(l_suppkey) as _vnode] } + └─StreamShare { id: 9 } + └─StreamProject { exprs: [l_suppkey, sum($expr1)] } + └─StreamHashAgg [append_only] { group_key: [l_suppkey], aggs: [sum($expr1), count] } + └─StreamExchange { dist: HashShard(l_suppkey) } + └─StreamProject { exprs: [l_suppkey, (l_extendedprice * (1:Decimal - l_discount)) as $expr1, _row_id] } + └─StreamFilter { predicate: (l_shipdate >= '1993-01-01':Date) AND (l_shipdate < '1993-04-01 00:00:00':Timestamp) } + └─StreamRowIdGen { row_id_index: 19 } + └─StreamSourceScan { 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, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } +- id: tpch_q16 + before: + - create_tables + sql: | + select + p_brand, + p_type, + p_size, + count(distinct ps_suppkey) as supplier_cnt + from + partsupp, + part + where + p_partkey = ps_partkey + and p_brand <> 'Brand#45' + and p_type not like 'SMALL PLATED%' + and p_size in (19, 17, 16, 23, 10, 4, 38, 11) + and ps_suppkey not in ( + select + s_suppkey + from + supplier + where + s_comment like '%Customer%Complaints%' + ) + group by + p_brand, + p_type, + p_size + order by + supplier_cnt desc, + p_brand, + p_type, + p_size + LIMIT 1; + stream_plan: |- + StreamMaterialize { columns: [p_brand, p_type, p_size, supplier_cnt], stream_key: [], pk_columns: [supplier_cnt, p_brand, p_type, p_size], pk_conflict: NoCheck } + └─StreamProject { exprs: [p_brand, p_type, p_size, count(distinct ps_suppkey)] } + └─StreamTopN { order: [count(distinct ps_suppkey) DESC, p_brand ASC, p_type ASC, p_size ASC], limit: 1, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [count(distinct ps_suppkey) DESC, p_brand ASC, p_type ASC, p_size ASC], limit: 1, offset: 0, group_key: [$expr1] } + └─StreamProject { exprs: [p_brand, p_type, p_size, count(distinct ps_suppkey), Vnode(p_brand, p_type, p_size) as $expr1] } + └─StreamHashAgg { group_key: [p_brand, p_type, p_size], aggs: [count(distinct ps_suppkey), count] } + └─StreamExchange { dist: HashShard(p_brand, p_type, p_size) } + └─StreamHashJoin { type: LeftAnti, predicate: ps_suppkey = s_suppkey, output: [ps_suppkey, p_brand, p_type, p_size, _row_id, _row_id, ps_partkey] } + ├─StreamExchange { dist: HashShard(ps_suppkey) } + │ └─StreamHashJoin [append_only] { type: Inner, predicate: ps_partkey = p_partkey, output: [ps_suppkey, p_brand, p_type, p_size, _row_id, ps_partkey, _row_id] } + │ ├─StreamExchange { dist: HashShard(ps_partkey) } + │ │ └─StreamRowIdGen { row_id_index: 8 } + │ │ └─StreamSourceScan { columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ └─StreamExchange { dist: HashShard(p_partkey) } + │ └─StreamFilter { predicate: (p_brand <> 'Brand#45':Varchar) AND Not(Like(p_type, 'SMALL PLATED%':Varchar)) AND In(p_size, 19:Int32, 17:Int32, 16:Int32, 23:Int32, 10:Int32, 4:Int32, 38:Int32, 11:Int32) } + │ └─StreamRowIdGen { row_id_index: 12 } + │ └─StreamSourceScan { columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(s_suppkey) } + └─StreamProject { exprs: [s_suppkey, _row_id] } + └─StreamFilter { predicate: Like(s_comment, '%Customer%Complaints%':Varchar) } + └─StreamRowIdGen { row_id_index: 10 } + └─StreamSourceScan { columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } +- id: tpch_q17 + before: + - create_tables + sql: | + select + sum(l_extendedprice) / 7.0 as avg_yearly + from + lineitem, + part + where + p_partkey = l_partkey + and p_brand = 'Brand#13' + and p_container = 'JUMBO PKG' + and l_quantity < ( + select + 0.2 * avg(l_quantity) + from + lineitem + where + l_partkey = p_partkey + ); + stream_plan: |- + StreamMaterialize { columns: [avg_yearly], stream_key: [], pk_columns: [], pk_conflict: NoCheck } + └─StreamProject { exprs: [(sum(sum(l_extendedprice)) / 7.0:Decimal) as $expr2] } + └─StreamSimpleAgg { aggs: [sum(sum(l_extendedprice)), count] } + └─StreamExchange { dist: Single } + └─StreamStatelessSimpleAgg { aggs: [sum(l_extendedprice)] } + └─StreamProject { exprs: [l_extendedprice, _row_id, _row_id, l_partkey, p_partkey, l_partkey] } + └─StreamFilter { predicate: (l_quantity < $expr1) } + └─StreamHashJoin { type: Inner, predicate: p_partkey = l_partkey, output: all } + ├─StreamExchange { dist: HashShard(p_partkey) } + │ └─StreamHashJoin [append_only] { type: Inner, predicate: l_partkey = p_partkey, output: [l_quantity, l_extendedprice, p_partkey, _row_id, l_partkey, _row_id] } + │ ├─StreamExchange { dist: HashShard(l_partkey) } + │ │ └─StreamShare { id: 3 } + │ │ └─StreamProject { exprs: [l_partkey, l_quantity, l_extendedprice, _row_id] } + │ │ └─StreamRowIdGen { row_id_index: 19 } + │ │ └─StreamSourceScan { 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, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ └─StreamExchange { dist: HashShard(p_partkey) } + │ └─StreamFilter { predicate: (p_brand = 'Brand#13':Varchar) AND (p_container = 'JUMBO PKG':Varchar) } + │ └─StreamRowIdGen { row_id_index: 12 } + │ └─StreamSourceScan { columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamProject { exprs: [(0.2:Decimal * (sum(l_quantity) / count(l_quantity)::Decimal)) as $expr1, l_partkey] } + └─StreamHashAgg [append_only] { group_key: [l_partkey], aggs: [sum(l_quantity), count(l_quantity), count] } + └─StreamExchange { dist: HashShard(l_partkey) } + └─StreamShare { id: 3 } + └─StreamProject { exprs: [l_partkey, l_quantity, l_extendedprice, _row_id] } + └─StreamRowIdGen { row_id_index: 19 } + └─StreamSourceScan { 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, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } +- id: tpch_q18 + before: + - create_tables + sql: | + select + c_name, + c_custkey, + o_orderkey, + o_orderdate, + o_totalprice, + sum(l_quantity) quantity + from + customer, + orders, + lineitem + where + o_orderkey in ( + select + l_orderkey + from + lineitem + group by + l_orderkey + having + sum(l_quantity) > 1 + ) + and c_custkey = o_custkey + and o_orderkey = l_orderkey + group by + c_name, + c_custkey, + o_orderkey, + o_orderdate, + o_totalprice + order by + o_totalprice desc, + o_orderdate + LIMIT 100; + stream_plan: |- + StreamMaterialize { columns: [c_name, c_custkey, o_orderkey, o_orderdate, o_totalprice, quantity], stream_key: [c_custkey, c_name, o_orderkey, o_totalprice, o_orderdate], pk_columns: [o_totalprice, o_orderdate, c_custkey, c_name, o_orderkey], pk_conflict: NoCheck } + └─StreamProject { exprs: [c_name, c_custkey, o_orderkey, o_orderdate, o_totalprice, sum(l_quantity)] } + └─StreamTopN { order: [o_totalprice DESC, o_orderdate ASC], limit: 100, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [o_totalprice DESC, o_orderdate ASC], limit: 100, offset: 0, group_key: [$expr1] } + └─StreamProject { exprs: [c_name, c_custkey, o_orderkey, o_orderdate, o_totalprice, sum(l_quantity), Vnode(o_orderkey) as $expr1] } + └─StreamHashAgg { group_key: [c_custkey, c_name, o_orderkey, o_totalprice, o_orderdate], aggs: [sum(l_quantity), count] } + └─StreamHashJoin { type: LeftSemi, predicate: o_orderkey = l_orderkey, output: all } + ├─StreamHashJoin [append_only] { type: Inner, predicate: o_orderkey = l_orderkey, output: [c_custkey, c_name, o_orderkey, o_totalprice, o_orderdate, l_quantity, _row_id, _row_id, _row_id] } + │ ├─StreamExchange { dist: HashShard(o_orderkey) } + │ │ └─StreamHashJoin [append_only] { type: Inner, predicate: c_custkey = o_custkey, output: [c_custkey, c_name, o_orderkey, o_totalprice, o_orderdate, _row_id, _row_id] } + │ │ ├─StreamExchange { dist: HashShard(c_custkey) } + │ │ │ └─StreamRowIdGen { row_id_index: 11 } + │ │ │ └─StreamSourceScan { columns: [c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ │ └─StreamExchange { dist: HashShard(o_custkey) } + │ │ └─StreamRowIdGen { row_id_index: 12 } + │ │ └─StreamSourceScan { columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ └─StreamExchange { dist: HashShard(l_orderkey) } + │ └─StreamShare { id: 9 } + │ └─StreamProject { exprs: [l_orderkey, l_quantity, _row_id] } + │ └─StreamRowIdGen { row_id_index: 19 } + │ └─StreamSourceScan { 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, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamProject { exprs: [l_orderkey] } + └─StreamFilter { predicate: (sum(l_quantity) > 1:Decimal) } + └─StreamProject { exprs: [l_orderkey, sum(l_quantity)] } + └─StreamHashAgg [append_only] { group_key: [l_orderkey], aggs: [sum(l_quantity), count] } + └─StreamExchange { dist: HashShard(l_orderkey) } + └─StreamShare { id: 9 } + └─StreamProject { exprs: [l_orderkey, l_quantity, _row_id] } + └─StreamRowIdGen { row_id_index: 19 } + └─StreamSourceScan { 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, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } +- id: tpch_q19 + before: + - create_tables + sql: | + select + sum(l_extendedprice* (1 - l_discount)) as revenue + from + lineitem, + part + where + ( + p_partkey = l_partkey + and p_brand = 'Brand#52' + and p_container in ('SM CASE', 'SM BOX', 'SM PACK', 'SM PKG') + and l_quantity >= 1 and l_quantity <= 11 + and p_size between 1 and 5 + and l_shipmode in ('AIR', 'AIR REG') + and l_shipinstruct = 'DELIVER IN PERSON' + ) + or + ( + p_partkey = l_partkey + and p_brand = 'Brand#24' + and p_container in ('MED BAG', 'MED BOX', 'MED PKG', 'MED PACK') + and l_quantity >= 30 and l_quantity <= 40 + and p_size between 1 and 10 + and l_shipmode in ('AIR', 'AIR REG') + and l_shipinstruct = 'DELIVER IN PERSON' + ) + or + ( + p_partkey = l_partkey + and p_brand = 'Brand#32' + and p_container in ('LG CASE', 'LG BOX', 'LG PACK', 'LG PKG') + and l_quantity >= 10 and l_quantity <= 20 + and p_size between 1 and 15 + and l_shipmode in ('AIR', 'AIR REG') + and l_shipinstruct = 'DELIVER IN PERSON' + ); + stream_plan: |- + StreamMaterialize { columns: [revenue], stream_key: [], pk_columns: [], pk_conflict: NoCheck } + └─StreamProject { exprs: [sum(sum($expr1))] } + └─StreamSimpleAgg [append_only] { aggs: [sum(sum($expr1)), count] } + └─StreamExchange { dist: Single } + └─StreamStatelessSimpleAgg { aggs: [sum($expr1)] } + └─StreamProject { exprs: [(l_extendedprice * (1:Decimal - l_discount)) as $expr1, _row_id, _row_id, l_partkey] } + └─StreamFilter { predicate: ((((((p_brand = 'Brand#52':Varchar) AND In(p_container, 'SM CASE':Varchar, 'SM BOX':Varchar, 'SM PACK':Varchar, 'SM PKG':Varchar)) AND ((l_quantity >= 1:Decimal) AND (l_quantity <= 11:Decimal))) AND (p_size <= 5:Int32)) OR ((((p_brand = 'Brand#24':Varchar) AND In(p_container, 'MED BAG':Varchar, 'MED BOX':Varchar, 'MED PKG':Varchar, 'MED PACK':Varchar)) AND ((l_quantity >= 30:Decimal) AND (l_quantity <= 40:Decimal))) AND (p_size <= 10:Int32))) OR ((((p_brand = 'Brand#32':Varchar) AND In(p_container, 'LG CASE':Varchar, 'LG BOX':Varchar, 'LG PACK':Varchar, 'LG PKG':Varchar)) AND ((l_quantity >= 10:Decimal) AND (l_quantity <= 20:Decimal))) AND (p_size <= 15:Int32))) } + └─StreamHashJoin [append_only] { type: Inner, predicate: l_partkey = p_partkey, output: all } + ├─StreamExchange { dist: HashShard(l_partkey) } + │ └─StreamFilter { predicate: In(l_shipmode, 'AIR':Varchar, 'AIR REG':Varchar) AND (l_shipinstruct = 'DELIVER IN PERSON':Varchar) } + │ └─StreamRowIdGen { row_id_index: 19 } + │ └─StreamSourceScan { 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, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(p_partkey) } + └─StreamFilter { predicate: (p_size >= 1:Int32) } + └─StreamRowIdGen { row_id_index: 12 } + └─StreamSourceScan { columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } +- id: tpch_q20 + before: + - create_tables + sql: "select\n\ts_name,\n\ts_address\nfrom\n\tsupplier,\n\tnation\nwhere\n\ts_suppkey in (\n\t\tselect\n\t\t\tps_suppkey\n\t\tfrom\n\t\t\tpartsupp,\n\t\t\t(\n\t\t\t\tselect\n\t\t\t\t\tl_partkey agg_partkey,\n\t\t\t\t\tl_suppkey agg_suppkey,\n\t\t\t\t\t0.5 * sum(l_quantity) AS agg_quantity\n\t\t\t\tfrom\n\t\t\t\t\tlineitem\n\t\t\t\twhere\n\t\t\t\t\tl_shipdate >= date '1994-01-01'\n\t\t\t\t\tand l_shipdate < date '1994-01-01' + interval '1' year\n\t\t\t\tgroup by\n\t\t\t\t\tl_partkey,\n\t\t\t\t\tl_suppkey\n\t\t\t) agg_lineitem\n\t\twhere\n\t\t\tagg_partkey = ps_partkey\n\t\t\tand agg_suppkey = ps_suppkey\n\t\t\tand ps_partkey in (\n\t\t\t\tselect\n\t\t\t\t\tp_partkey\n\t\t\t\tfrom\n\t\t\t\t\tpart\n\t\t\t\twhere\n\t\t\t\t\tp_name like 'forest%'\n\t\t\t)\n\t\t\tand ps_availqty > agg_quantity\n\t)\n\tand s_nationkey = n_nationkey\n\tand n_name = 'KENYA'\norder by\n\ts_name\nLIMIT 1;\n" + stream_plan: |- + StreamMaterialize { columns: [s_name, s_address, _row_id(hidden), _row_id#1(hidden), s_nationkey(hidden), s_suppkey(hidden)], stream_key: [], pk_columns: [s_name], pk_conflict: NoCheck } + └─StreamProject { exprs: [s_name, s_address, _row_id, _row_id, s_nationkey, s_suppkey] } + └─StreamTopN { order: [s_name ASC], limit: 1, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [s_name ASC], limit: 1, offset: 0, group_key: [_vnode] } + └─StreamProject { exprs: [s_name, s_address, _row_id, _row_id, s_nationkey, s_suppkey, Vnode(s_suppkey) as _vnode] } + └─StreamHashJoin { type: LeftSemi, predicate: s_suppkey = ps_suppkey, output: [s_name, s_address, _row_id, _row_id, s_nationkey, s_suppkey] } + ├─StreamExchange { dist: HashShard(s_suppkey) } + │ └─StreamHashJoin [append_only] { type: Inner, predicate: s_nationkey = n_nationkey, output: [s_suppkey, s_name, s_address, _row_id, s_nationkey, _row_id] } + │ ├─StreamExchange { dist: HashShard(s_nationkey) } + │ │ └─StreamRowIdGen { row_id_index: 10 } + │ │ └─StreamSourceScan { columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ └─StreamExchange { dist: HashShard(n_nationkey) } + │ └─StreamFilter { predicate: (n_name = 'KENYA':Varchar) } + │ └─StreamRowIdGen { row_id_index: 7 } + │ └─StreamSourceScan { columns: [n_nationkey, n_name, n_regionkey, n_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(ps_suppkey) } + └─StreamHashJoin { type: LeftSemi, predicate: ps_partkey = p_partkey, output: [ps_suppkey, _row_id, ps_partkey] } + ├─StreamExchange { dist: HashShard(ps_partkey) } + │ └─StreamProject { exprs: [ps_partkey, ps_suppkey, _row_id, l_partkey, l_suppkey] } + │ └─StreamFilter { predicate: ($expr1 > $expr2) } + │ └─StreamHashJoin { type: Inner, predicate: ps_partkey = l_partkey AND ps_suppkey = l_suppkey, output: all } + │ ├─StreamExchange { dist: HashShard(ps_partkey, ps_suppkey) } + │ │ └─StreamProject { exprs: [ps_partkey, ps_suppkey, ps_availqty::Decimal as $expr1, _row_id] } + │ │ └─StreamRowIdGen { row_id_index: 8 } + │ │ └─StreamSourceScan { columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ └─StreamProject { exprs: [l_partkey, l_suppkey, (0.5:Decimal * sum(l_quantity)) as $expr2] } + │ └─StreamHashAgg [append_only] { group_key: [l_partkey, l_suppkey], aggs: [sum(l_quantity), count] } + │ └─StreamExchange { dist: HashShard(l_partkey, l_suppkey) } + │ └─StreamFilter { predicate: (l_shipdate >= '1994-01-01':Date) AND (l_shipdate < '1995-01-01 00:00:00':Timestamp) } + │ └─StreamRowIdGen { row_id_index: 19 } + │ └─StreamSourceScan { 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, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(p_partkey) } + └─StreamProject { exprs: [p_partkey, _row_id] } + └─StreamFilter { predicate: Like(p_name, 'forest%':Varchar) } + └─StreamRowIdGen { row_id_index: 12 } + └─StreamSourceScan { columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } +- id: tpch_q21 + before: + - create_tables + sql: | + select + s_name, + count(*) as numwait + from + supplier, + lineitem l1, + orders, + nation + where + s_suppkey = l1.l_suppkey + and o_orderkey = l1.l_orderkey + and o_orderstatus = 'F' + and l1.l_receiptdate > l1.l_commitdate + and exists ( + select + * + from + lineitem l2 + where + l2.l_orderkey = l1.l_orderkey + and l2.l_suppkey <> l1.l_suppkey + ) + and not exists ( + select + * + from + lineitem l3 + where + l3.l_orderkey = l1.l_orderkey + and l3.l_suppkey <> l1.l_suppkey + and l3.l_receiptdate > l3.l_commitdate + ) + and s_nationkey = n_nationkey + and n_name = 'GERMANY' + group by + s_name + order by + numwait desc, + s_name + LIMIT 100; + stream_plan: |- + StreamMaterialize { columns: [s_name, numwait], stream_key: [s_name], pk_columns: [numwait, s_name], pk_conflict: NoCheck } + └─StreamProject { exprs: [s_name, count] } + └─StreamTopN { order: [count DESC, s_name ASC], limit: 100, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [count DESC, s_name ASC], limit: 100, offset: 0, group_key: [_vnode] } + └─StreamProject { exprs: [s_name, count, Vnode(s_name) as _vnode] } + └─StreamHashAgg { group_key: [s_name], aggs: [count] } + └─StreamExchange { dist: HashShard(s_name) } + └─StreamHashJoin { type: LeftAnti, predicate: l_orderkey = l_orderkey AND (l_suppkey <> l_suppkey), output: [s_name, _row_id, _row_id, n_nationkey, _row_id, _row_id, o_orderkey, s_suppkey, l_orderkey] } + ├─StreamHashJoin { type: LeftSemi, predicate: l_orderkey = l_orderkey AND (l_suppkey <> l_suppkey), output: [s_name, l_orderkey, l_suppkey, _row_id, _row_id, n_nationkey, _row_id, _row_id, o_orderkey, s_suppkey] } + │ ├─StreamExchange { dist: HashShard(l_orderkey) } + │ │ └─StreamHashJoin [append_only] { type: Inner, predicate: s_suppkey = l_suppkey, output: [s_name, l_orderkey, l_suppkey, _row_id, _row_id, n_nationkey, s_suppkey, _row_id, _row_id, o_orderkey] } + │ │ ├─StreamExchange { dist: HashShard(s_suppkey) } + │ │ │ └─StreamHashJoin [append_only] { type: Inner, predicate: n_nationkey = s_nationkey, output: [s_suppkey, s_name, _row_id, n_nationkey, _row_id] } + │ │ │ ├─StreamExchange { dist: HashShard(n_nationkey) } + │ │ │ │ └─StreamFilter { predicate: (n_name = 'GERMANY':Varchar) } + │ │ │ │ └─StreamRowIdGen { row_id_index: 7 } + │ │ │ │ └─StreamSourceScan { columns: [n_nationkey, n_name, n_regionkey, n_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ │ │ └─StreamExchange { dist: HashShard(s_nationkey) } + │ │ │ └─StreamRowIdGen { row_id_index: 10 } + │ │ │ └─StreamSourceScan { columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ │ └─StreamExchange { dist: HashShard(l_suppkey) } + │ │ └─StreamHashJoin [append_only] { type: Inner, predicate: o_orderkey = l_orderkey, output: [l_orderkey, l_suppkey, _row_id, o_orderkey, _row_id] } + │ │ ├─StreamExchange { dist: HashShard(o_orderkey) } + │ │ │ └─StreamFilter { predicate: (o_orderstatus = 'F':Varchar) } + │ │ │ └─StreamRowIdGen { row_id_index: 12 } + │ │ │ └─StreamSourceScan { columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ │ └─StreamExchange { dist: HashShard(l_orderkey) } + │ │ └─StreamFilter { predicate: (l_receiptdate > l_commitdate) } + │ │ └─StreamShare { id: 13 } + │ │ └─StreamProject { exprs: [l_orderkey, l_suppkey, l_commitdate, l_receiptdate, _row_id] } + │ │ └─StreamRowIdGen { row_id_index: 19 } + │ │ └─StreamSourceScan { 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, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ └─StreamExchange { dist: HashShard(l_orderkey) } + │ └─StreamProject { exprs: [l_orderkey, l_suppkey, _row_id] } + │ └─StreamShare { id: 13 } + │ └─StreamProject { exprs: [l_orderkey, l_suppkey, l_commitdate, l_receiptdate, _row_id] } + │ └─StreamRowIdGen { row_id_index: 19 } + │ └─StreamSourceScan { 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, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(l_orderkey) } + └─StreamProject { exprs: [l_orderkey, l_suppkey, _row_id] } + └─StreamFilter { predicate: (l_receiptdate > l_commitdate) } + └─StreamShare { id: 13 } + └─StreamProject { exprs: [l_orderkey, l_suppkey, l_commitdate, l_receiptdate, _row_id] } + └─StreamRowIdGen { row_id_index: 19 } + └─StreamSourceScan { 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, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } +- id: tpch_q22 + before: + - create_tables + sql: | + select + cntrycode, + count(*) as numcust, + sum(c_acctbal) as totacctbal + from + ( + select + substring(c_phone from 1 for 2) as cntrycode, + c_acctbal + from + customer + where + substring(c_phone from 1 for 2) in + ('30', '24', '31', '38', '25', '34', '37') + and c_acctbal > ( + select + avg(c_acctbal) + from + customer + where + c_acctbal > 0.00::numeric + and substring(c_phone from 1 for 2) in + ('30', '24', '31', '38', '25', '34', '37') + ) + and not exists ( + select + * + from + orders + where + o_custkey = c_custkey + ) + ) as custsale + group by + cntrycode + order by + cntrycode + LIMIT 1; + stream_plan: |- + StreamMaterialize { columns: [cntrycode, numcust, totacctbal], stream_key: [], pk_columns: [cntrycode], pk_conflict: NoCheck } + └─StreamProject { exprs: [$expr2, count, sum(c_acctbal)] } + └─StreamTopN { order: [$expr2 ASC], limit: 1, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [$expr2 ASC], limit: 1, offset: 0, group_key: [_vnode] } + └─StreamProject { exprs: [$expr2, count, sum(c_acctbal), Vnode($expr2) as _vnode] } + └─StreamHashAgg { group_key: [$expr2], aggs: [count, sum(c_acctbal)] } + └─StreamExchange { dist: HashShard($expr2) } + └─StreamProject { exprs: [Substr(c_phone, 1:Int32, 2:Int32) as $expr2, c_acctbal, _row_id, c_custkey] } + └─StreamDynamicFilter { predicate: (c_acctbal > $expr1), output: [c_phone, c_acctbal, _row_id, c_custkey] } + ├─StreamHashJoin { type: LeftAnti, predicate: c_custkey = o_custkey, output: [c_phone, c_acctbal, _row_id, c_custkey] } + │ ├─StreamExchange { dist: HashShard(c_custkey) } + │ │ └─StreamFilter { predicate: In(Substr(c_phone, 1:Int32, 2:Int32), '30':Varchar, '24':Varchar, '31':Varchar, '38':Varchar, '25':Varchar, '34':Varchar, '37':Varchar) } + │ │ └─StreamShare { id: 4 } + │ │ └─StreamProject { exprs: [c_custkey, c_phone, c_acctbal, _row_id] } + │ │ └─StreamFilter { predicate: In(Substr(c_phone, 1:Int32, 2:Int32), '30':Varchar, '24':Varchar, '31':Varchar, '38':Varchar, '25':Varchar, '34':Varchar, '37':Varchar) } + │ │ └─StreamRowIdGen { row_id_index: 11 } + │ │ └─StreamSourceScan { columns: [c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ └─StreamExchange { dist: HashShard(o_custkey) } + │ └─StreamProject { exprs: [o_custkey, _row_id] } + │ └─StreamRowIdGen { row_id_index: 12 } + │ └─StreamSourceScan { columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: Broadcast } + └─StreamProject { exprs: [(sum(sum(c_acctbal)) / sum0(count(c_acctbal))::Decimal) as $expr1] } + └─StreamSimpleAgg [append_only] { aggs: [sum(sum(c_acctbal)), sum0(count(c_acctbal)), count] } + └─StreamExchange { dist: Single } + └─StreamStatelessSimpleAgg { aggs: [sum(c_acctbal), count(c_acctbal)] } + └─StreamFilter { predicate: (c_acctbal > 0.00:Decimal) AND In(Substr(c_phone, 1:Int32, 2:Int32), '30':Varchar, '24':Varchar, '31':Varchar, '38':Varchar, '25':Varchar, '34':Varchar, '37':Varchar) } + └─StreamShare { id: 4 } + └─StreamProject { exprs: [c_custkey, c_phone, c_acctbal, _row_id] } + └─StreamFilter { predicate: In(Substr(c_phone, 1:Int32, 2:Int32), '30':Varchar, '24':Varchar, '31':Varchar, '38':Varchar, '25':Varchar, '34':Varchar, '37':Varchar) } + └─StreamRowIdGen { row_id_index: 11 } + └─StreamSourceScan { columns: [c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] }