diff --git a/Cargo.lock b/Cargo.lock index 73fb557..3e9182f 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -2085,7 +2085,7 @@ dependencies = [ [[package]] name = "raysql" -version = "0.3.0" +version = "0.4.0" dependencies = [ "datafusion", "datafusion-proto", diff --git a/Cargo.toml b/Cargo.toml index 6c2f47b..f14d2dd 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -4,7 +4,7 @@ description = "RaySQL: DataFusion on Ray" homepage = "https://github.com/andygrove/ray-sql" repository = "https://github.com/andygrove/ray-sql" authors = ["Andy Grove "] -version = "0.3.0" +version = "0.4.0" edition = "2021" readme = "README.md" license = "Apache-2.0" diff --git a/README.md b/README.md index d490ae9..ad176bf 100644 --- a/README.md +++ b/README.md @@ -5,13 +5,14 @@ This is a personal research project to evaluate performing distributed SQL queri ## Goals -- Demonstrate how easily new systems can be built on top of DataFusion -- Drive requirements for DataFusion's Python bindings -- Create content for an interesting blog post or conference talk +- Demonstrate how easily new systems can be built on top of DataFusion. See the [design documentation](./docs/README.md) + to understand how RaySQL works. +- Drive requirements for DataFusion's [Python bindings](https://github.com/apache/arrow-datafusion-python). +- Create content for an interesting blog post or conference talk. ## Non Goals -- Build and support a production system +- Build and support a production system. ## Example @@ -28,10 +29,14 @@ ray.init() # create some remote Workers workers = [Worker.remote() for i in range(2)] -# create context and plan a query -ctx = RaySqlContext(workers) -ctx.register_csv('tips', 'tips.csv', True) -result_set = ctx.sql('select sex, smoker, avg(tip/total_bill) as tip_pct from tips group by sex, smoker') +# create a remote context and register a table +ctx = RaySqlContext.remote(workers) +ray.get(ctx.register_csv.remote('tips', 'tips.csv', True)) + +# Parquet is also supported +# ctx.register_parquet('tips', 'tips.parquet') + +result_set = ray.get(ctx.sql.remote('select sex, smoker, avg(tip/total_bill) as tip_pct from tips group by sex, smoker')) print(result_set) ``` @@ -58,7 +63,7 @@ Note that query 15 is excluded from both results since RaySQL does not support D ### Overall Time -RaySQL is ~30% faster overall for this scale factor and environment. +RaySQL is ~65% faster overall for this scale factor and environment. ![SQLBench-H Total](./docs/sqlbench-h-total.png) diff --git a/docs/README.md b/docs/README.md index c3eb716..9eefa1a 100644 --- a/docs/README.md +++ b/docs/README.md @@ -1,8 +1,260 @@ -# Notes +# RaySQL Design Documentation -## Shuffle +RaySQL is a distributed SQL query engine that is powered by DataFusion. -- Each query stage has input and output partition count -- There is one task per input stage -- Each task can produce multiple output partitions -- Example: 4 input partitions, 4 output partitions (with different partitioning scheme) results in 16 shuffle files (4 x 4) \ No newline at end of file +DataFusion provides a high-performance query engine that is already partition-aware, with partitions being executed +in parallel in separate threads. RaySQL provides a distributed query planner that translates a DataFusion physical +plan into a distributed plan. + +Let's walk through an example to see how that works. We'll use [SQLBench-H](https://github.com/sql-benchmarks/sqlbench-h) +query 3 for the example. This is an aggregate query with a three-way join. + +_SQLBench-H Query 3_ + +```sql +-- SQLBench-H query 3 derived from TPC-H query 3 under the terms of the TPC Fair Use Policy. +-- TPC-H queries are Copyright 1993-2022 Transaction Processing Performance Council. +select + l_orderkey, + sum(l_extendedprice * (1 - l_discount)) as revenue, + o_orderdate, + o_shippriority +from + customer, + orders, + lineitem +where + c_mktsegment = 'HOUSEHOLD' + and c_custkey = o_custkey + and l_orderkey = o_orderkey + and o_orderdate < date '1995-03-21' + and l_shipdate > date '1995-03-21' +group by + l_orderkey, + o_orderdate, + o_shippriority +order by + revenue desc, + o_orderdate limit 10; +``` + +## DataFusion's Logical Plan + +DataFusion produces the following optimized _logical_ query plan. Note that this plan does not have the +concept of partitions yet. + +```text +Limit: skip=0, fetch=10 + Sort: revenue DESC NULLS FIRST, orders.o_orderdate ASC NULLS LAST, fetch=10 + Projection: lineitem.l_orderkey, SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount) AS revenue, orders.o_orderdate, orders.o_shippriority + Aggregate: groupBy=[[lineitem.l_orderkey, orders.o_orderdate, orders.o_shippriority]], aggr=[[SUM(CAST(lineitem.l_extendedprice AS Decimal128(35, 4)) * CAST(Decimal128(Some(100),23,2) - CAST(lineitem.l_discount AS Decimal128(23, 2)) AS Decimal128(35, 4))) AS SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)]] + Inner Join: orders.o_orderkey = lineitem.l_orderkey + Inner Join: customer.c_custkey = orders.o_custkey + Filter: customer.c_mktsegment = Utf8("BUILDING") + TableScan: customer projection=[c_custkey, c_mktsegment], partial_filters=[customer.c_mktsegment = Utf8("BUILDING")] + Filter: orders.o_orderdate < Date32("9204") + TableScan: orders projection=[o_orderkey, o_custkey, o_orderdate, o_shippriority], partial_filters=[orders.o_orderdate < Date32("9204")] + Filter: lineitem.l_shipdate > Date32("9204") + TableScan: lineitem projection=[l_orderkey, l_extendedprice, l_discount, l_shipdate], partial_filters=[lineitem.l_shipdate > Date32("9204")] +``` + +## DataFusion's Physical Plan + +DataFusion's physical plan lists all the files to be queried, and they are organized into partitions to allow for +parallel execution within a single process. In this example, the level of concurrency was configured to be four, so +we see `partitions={4 groups: [[ ... ]]` in the leaf `ParquetExec` nodes, with the filenames listed in four groups. + +_DataFusion will soon support parallel execution for single Parquet files but for now the parallelism is based on +splitting the available files into separate groups, so RaySQL will not yet scale well for single-file inputs._ + +Here is the full physical plan for query 3. + +```text +GlobalLimitExec: skip=0, fetch=10 + SortPreservingMergeExec: [revenue@1 DESC,o_orderdate@2 ASC NULLS LAST] + SortExec: [revenue@1 DESC,o_orderdate@2 ASC NULLS LAST] + ProjectionExec: expr=[l_orderkey@0 as l_orderkey, SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@3 as revenue, o_orderdate@1 as o_orderdate, o_shippriority@2 as o_shippriority] + AggregateExec: mode=FinalPartitioned, gby=[l_orderkey@0 as l_orderkey, o_orderdate@1 as o_orderdate, o_shippriority@2 as o_shippriority], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] + CoalesceBatchesExec: target_batch_size=8192 + RepartitionExec: partitioning=Hash([Column { name: "l_orderkey", index: 0 }, Column { name: "o_orderdate", index: 1 }, Column { name: "o_shippriority", index: 2 }], 4), input_partitions=4 + AggregateExec: mode=Partial, gby=[l_orderkey@6 as l_orderkey, o_orderdate@4 as o_orderdate, o_shippriority@5 as o_shippriority], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] + CoalesceBatchesExec: target_batch_size=8192 + HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "o_orderkey", index: 2 }, Column { name: "l_orderkey", index: 0 })] + CoalesceBatchesExec: target_batch_size=8192 + RepartitionExec: partitioning=Hash([Column { name: "o_orderkey", index: 2 }], 4), input_partitions=4 + CoalesceBatchesExec: target_batch_size=8192 + HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "c_custkey", index: 0 }, Column { name: "o_custkey", index: 1 })] + CoalesceBatchesExec: target_batch_size=8192 + RepartitionExec: partitioning=Hash([Column { name: "c_custkey", index: 0 }], 4), input_partitions=4 + CoalesceBatchesExec: target_batch_size=8192 + FilterExec: c_mktsegment@1 = BUILDING + ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-3.parquet]]}, predicate=c_mktsegment = Utf8("BUILDING"), pruning_predicate=c_mktsegment_min@0 <= BUILDING AND BUILDING <= c_mktsegment_max@1, projection=[c_custkey, c_mktsegment] + CoalesceBatchesExec: target_batch_size=8192 + RepartitionExec: partitioning=Hash([Column { name: "o_custkey", index: 1 }], 4), input_partitions=4 + CoalesceBatchesExec: target_batch_size=8192 + FilterExec: o_orderdate@2 < 9204 + ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-3.parquet]]}, predicate=o_orderdate < Date32("9204"), pruning_predicate=o_orderdate_min@0 < 9204, projection=[o_orderkey, o_custkey, o_orderdate, o_shippriority] + CoalesceBatchesExec: target_batch_size=8192 + RepartitionExec: partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 4), input_partitions=4 + CoalesceBatchesExec: target_batch_size=8192 + FilterExec: l_shipdate@3 > 9204 + ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-3.parquet]]}, predicate=l_shipdate > Date32("9204"), pruning_predicate=l_shipdate_max@0 > 9204, projection=[l_orderkey, l_extendedprice, l_discount, l_shipdate] +``` + +## Partitioning & Distribution + +The partitioning scheme changes throughout the plan and this is the most important concept to +understand in order to understand RaySQL's design. Changes in partitioning are implemented by the `RepartitionExec` +operator in DataFusion and are happen in the following scenarios. + +### Joins + +The first join to happen is between `customer` and `orders`. The join condition is +`customers.c_custkey = orders.o_custkey`. To perform this join in parallel we first +need to repartition the data on the join keys so that the data for customers with `c_custkey = 1` +and the data for orders with `o_custkey = 1` can be sent to the same thread or node for +processing. This allows the join to happen in parallel. This is known as a hash-partitioned +join and is implemented by the `HashJoinExec` operator in DataFusion. + +We can see that `DataFusion` has inserted `RepartitionExec` operators around both inputs to the join. + +### Aggregates + +There are multiple approaches to distributed aggregate queries. Here are two popular approaches: + +- Perform aggregates in parallel on each partition, where the resulting data for each partition could contain + duplicate grouping keys and then perform a final aggregate on the intermediate aggregates to remove the duplicates. +- Partition the input data by the grouping keys so that the aggregates from each partition can simply be merged to + produce the final result. + +For this example query, the data is already partitioned on the aggregate's grouping key so the latter approach is used. + +### Sort + +Sort also has multiple approaches. + +- The input partitions can be collapsed down to a single partition and then sorted +- Partitions can be sorted in parallel and then merged using a sort-preserving merge + +DataFusion and RaySQL currently the first approach, but there is a DataFusion PR open for implementing the second. + +### Limit + +- The input partitions can be collapsed down to a single partition and then have the limit applied +- The limit can be pushed down to each partition as well + +## Query Stages + +The first two query stages to be executed will read the `customer` and `order` parquet files and reparition them by the join keys `c_custkey` and `o_custkey`. + +```text +Query Stage #0: +ShuffleWriterExec(stage_id=0, output_partitioning=Hash([Column { name: "c_custkey", index: 0 }], 4)) + CoalesceBatchesExec: target_batch_size=8192 + FilterExec: c_mktsegment@1 = BUILDING + ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-3.parquet]]}, predicate=c_mktsegment = Utf8("BUILDING"), pruning_predicate=c_mktsegment_min@0 <= BUILDING AND BUILDING <= c_mktsegment_max@1, projection=[c_custkey, c_mktsegment] + +Query Stage #1: +ShuffleWriterExec(stage_id=1, output_partitioning=Hash([Column { name: "o_custkey", index: 1 }], 4)) + CoalesceBatchesExec: target_batch_size=8192 + FilterExec: o_orderdate@2 < 9204 + ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-3.parquet]]}, predicate=o_orderdate < Date32("9204"), pruning_predicate=o_orderdate_min@0 < 9204, projection=[o_orderkey, o_custkey, o_orderdate, o_shippriority] +``` + +Now that these inputs are partitioned by join key, we can execute the join itself. The two inputs to the join are instances of `ShuffleReaderExec` that +read the shuffle files produced by the previous stages. The result of the join is repartitioned by `o_order_key` in preparation for the next join. + +```text +Query Stage #2: +ShuffleWriterExec(stage_id=2, output_partitioning=Hash([Column { name: "o_orderkey", index: 2 }], 4)) + CoalesceBatchesExec: target_batch_size=8192 + HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "c_custkey", index: 0 }, Column { name: "o_custkey", index: 1 })] + CoalesceBatchesExec: target_batch_size=8192 + ShuffleReaderExec(stage_id=0, input_partitions=4) + CoalesceBatchesExec: target_batch_size=8192 + ShuffleReaderExec(stage_id=1, input_partitions=4) +``` + +We continue to prepare for the next join with `lineitem` by repartitioning the parquet files by the join key. + +```text +Query Stage #3: +ShuffleWriterExec(stage_id=3, output_partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 4)) + CoalesceBatchesExec: target_batch_size=8192 + FilterExec: l_shipdate@3 > 9204 + ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-3.parquet]]}, predicate=l_shipdate > Date32("9204"), pruning_predicate=l_shipdate_max@0 > 9204, projection=[l_orderkey, l_extendedprice, l_discount, l_shipdate] +``` + +With the inputs repartitioned by the join keys, we can execute a query stage to perform the join. However, in this case, we also perform an aggregate in the +same stage without any additional repartitioning. This is possible because the aggregate grouping includes `l_orderkey` which we already partitioned on. This +means that we can perform this aggregate in parallel and guarantee that there will be no duplicates in the output of each aggregate. Pretty neat! + +The output of this shuffle is partitioned by `l_orderkey`, `o_orderdate`, and `o_shippriority` in preparation for the `ORDER BY` part of the query. + +```text +Query Stage #4: +ShuffleWriterExec(stage_id=4, output_partitioning=Hash([Column { name: "l_orderkey", index: 0 }, Column { name: "o_orderdate", index: 1 }, Column { name: "o_shippriority", index: 2 }], 4)) + AggregateExec: mode=Partial, gby=[l_orderkey@6 as l_orderkey, o_orderdate@4 as o_orderdate, o_shippriority@5 as o_shippriority], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] + CoalesceBatchesExec: target_batch_size=8192 + HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "o_orderkey", index: 2 }, Column { name: "l_orderkey", index: 0 })] + CoalesceBatchesExec: target_batch_size=8192 + ShuffleReaderExec(stage_id=2, input_partitions=4) + CoalesceBatchesExec: target_batch_size=8192 + ShuffleReaderExec(stage_id=3, input_partitions=4) +``` + +Now we perform a final aggregate (which is maybe redundant?) and then sort the results in parallel across the +partitions. + +```text +Query Stage #5: +ShuffleWriterExec(stage_id=5, output_partitioning=Hash([Column { name: "l_orderkey", index: 0 }, Column { name: "o_orderdate", index: 2 }, Column { name: "o_shippriority", index: 3 }], 4)) + SortExec: [revenue@1 DESC,o_orderdate@2 ASC NULLS LAST] + ProjectionExec: expr=[l_orderkey@0 as l_orderkey, SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@3 as revenue, o_orderdate@1 as o_orderdate, o_shippriority@2 as o_shippriority] + AggregateExec: mode=FinalPartitioned, gby=[l_orderkey@0 as l_orderkey, o_orderdate@1 as o_orderdate, o_shippriority@2 as o_shippriority], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] + CoalesceBatchesExec: target_batch_size=8192 + ShuffleReaderExec(stage_id=4, input_partitions=4) +``` + +Finally, we have a query stage that reads the sorted results and merges them into a single partition (preserving the +sort order) and applies the `LIMIT` clause. + +```text +Query Stage #6: +GlobalLimitExec: skip=0, fetch=10 + SortPreservingMergeExec: [revenue@1 DESC,o_orderdate@2 ASC NULLS LAST] + ShuffleReaderExec(stage_id=5, input_partitions=4) +``` + +## Distributed Scheduling + +In the previous section we walked through the plan as if we were executing one query stage at a time. However, some +of these query stages could execute in parallel (assuming enough resource is available). + +For example, we could execute query stage 3 to repartition the `lineitem` input without waiting for query stages 0, 1, +and 2 to finish. + +More generally, we can typically execute all remaining leaf nodes of the plan concurrently. + +The `execute_query_stage` method in `context.py` is a remote method that recursively walks the query plan and executes +child plans, building up a DAG of futures. + +## Distributed Shuffle + +The output of each query stage needs to be persisted somewhere so that the next query stage can read it. Currently, +RaySQL is just writing the output to disk in Arrow IPC format, and this means that RaySQL is not truly distributed +yet because it requires a shared file system. It would be better to use the Ray object store instead, as +proposed [here](https://github.com/andygrove/ray-sql/issues/22). + +DataFusion's `RepartitionExec` uses threads and channels within a single process and is not suitable for a +distributed query engine, so RaySQL rewrites the physical plan and replaces the `RepartionExec` with a pair of +operators to perform a "shuffle". These are the `ShuffleWriterExec` and `ShuffleReaderExec`. + +### Shuffle Writes + +`ShuffleWriterExec` reads input partitions and repartitions them, using the same `BatchPartitioner` that DataFusion +uses, then writes the output to disk in Arrow IPC format. + +### Shuffle Reads + +`ShuffleReaderExec` reads the shuffle files written by the `ShuffleWriterExec`. diff --git a/docs/sqlbench-h-per-query.png b/docs/sqlbench-h-per-query.png index d2b75d7..16b4e21 100644 Binary files a/docs/sqlbench-h-per-query.png and b/docs/sqlbench-h-per-query.png differ diff --git a/docs/sqlbench-h-total.png b/docs/sqlbench-h-total.png index 02bbf35..5e62f37 100644 Binary files a/docs/sqlbench-h-total.png and b/docs/sqlbench-h-total.png differ diff --git a/examples/tips.py b/examples/tips.py index 7f03227..95faff0 100644 --- a/examples/tips.py +++ b/examples/tips.py @@ -17,4 +17,3 @@ result_set = ray.get(ctx.sql.remote('select sex, smoker, avg(tip/total_bill) as tip_pct from tips group by sex, smoker')) print(result_set) -# print(ray.get(result_set)) diff --git a/raysql/context.py b/raysql/context.py index abc3e23..9417375 100644 --- a/raysql/context.py +++ b/raysql/context.py @@ -15,18 +15,18 @@ def execute_query_stage(query_stages, stage_id, workers): # if the query stage has a single output partition then we need to execute for the output # partition, otherwise we need to execute in parallel for each input partition - if stage.get_output_partition_count == 1: - partition_count = 1 - else: - partition_count = stage.get_input_partition_count() + concurrency = stage.get_input_partition_count() + if stage.get_output_partition_count() == 1: + # reduce stage + concurrency = 1 - print("Scheduling query stage #{} with {} input partitions and {} output partitions".format(stage.id(), partition_count, stage.get_output_partition_count())) + print("Scheduling query stage #{} with {} input partitions and {} output partitions".format(stage.id(), stage.get_input_partition_count(), stage.get_output_partition_count())) plan_bytes = ray.put(serialize_execution_plan(stage.get_execution_plan())) # round-robin allocation across workers futures = [] - for part in range(partition_count): + for part in range(concurrency): worker_index = part % len(workers) futures.append(workers[worker_index].execute_query_partition.remote(plan_bytes, part)) diff --git a/src/context.rs b/src/context.rs index 42d90a0..0200892 100644 --- a/src/context.rs +++ b/src/context.rs @@ -5,6 +5,8 @@ use datafusion::arrow::record_batch::RecordBatch; use datafusion::arrow::util::pretty::pretty_format_batches; use datafusion::error::Result; use datafusion::execution::context::TaskContext; +use datafusion::execution::disk_manager::DiskManagerConfig; +use datafusion::execution::memory_pool::FairSpillPool; use datafusion::execution::runtime_env::RuntimeEnv; use datafusion::physical_plan::displayable; use datafusion::prelude::*; @@ -28,17 +30,22 @@ pub struct PyContext { #[pymethods] impl PyContext { #[new] - pub fn new(target_partitions: usize) -> Self { + pub fn new(target_partitions: usize) -> Result { let config = SessionConfig::default() .with_target_partitions(target_partitions) - .with_batch_size(16*1024) + .with_batch_size(16 * 1024) .with_repartition_aggregations(true) .with_repartition_windows(true) .with_repartition_joins(true) .with_parquet_pruning(true); - Self { - ctx: SessionContext::with_config(config), - } + + let mem_pool_size = 1024 * 1024 * 1024; + let runtime_config = datafusion::execution::runtime_env::RuntimeConfig::new() + .with_memory_pool(Arc::new(FairSpillPool::new(mem_pool_size))) + .with_disk_manager(DiskManagerConfig::new_specified(vec!["/tmp".into()])); + let runtime = Arc::new(RuntimeEnv::new(runtime_config)?); + let ctx = SessionContext::with_config_rt(config, runtime); + Ok(Self { ctx }) } pub fn register_csv( @@ -78,7 +85,6 @@ impl PyContext { Ok(PyExecutionGraph::new(graph)) } - /// Execute a partition of a query plan. This will typically be executing a shuffle write and write the results to disk pub fn execute_partition(&self, plan: PyExecutionPlan, part: usize) -> PyResultSet { let batches = self diff --git a/src/lib.rs b/src/lib.rs index 21a008a..5018b08 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -3,14 +3,14 @@ extern crate core; use pyo3::prelude::*; mod proto; +use crate::context::{deserialize_execution_plan, serialize_execution_plan}; pub use proto::generated::protobuf; -use crate::context::{serialize_execution_plan, deserialize_execution_plan}; pub mod context; pub mod planner; +pub mod query_stage; pub mod shuffle; pub mod utils; -pub mod query_stage; /// A Python module implemented in Rust. #[pymodule] diff --git a/src/planner.rs b/src/planner.rs index 6a2b2e5..65dd777 100644 --- a/src/planner.rs +++ b/src/planner.rs @@ -1,11 +1,11 @@ -use crate::query_stage::QueryStage; use crate::query_stage::PyQueryStage; +use crate::query_stage::QueryStage; use crate::shuffle::{ShuffleReaderExec, ShuffleWriterExec}; use datafusion::error::Result; use datafusion::physical_plan::coalesce_partitions::CoalescePartitionsExec; use datafusion::physical_plan::repartition::RepartitionExec; use datafusion::physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; -use datafusion::physical_plan::Partitioning; +use datafusion::physical_plan::{displayable, Partitioning}; use datafusion::physical_plan::{with_new_children_if_necessary, ExecutionPlan}; use log::debug; use pyo3::prelude::*; @@ -27,13 +27,14 @@ impl PyExecutionGraph { #[pymethods] impl PyExecutionGraph { - /// Get a list of stages sorted by id pub fn get_query_stages(&self) -> Vec { let mut stages = vec![]; let max_id = self.graph.get_final_query_stage().id; for id in 0..=max_id { - stages.push(PyQueryStage::from_rust(self.graph.query_stages.get(&id).unwrap().clone())); + stages.push(PyQueryStage::from_rust( + self.graph.query_stages.get(&id).unwrap().clone(), + )); } stages } @@ -95,8 +96,6 @@ impl ExecutionGraph { } } - - pub fn make_execution_graph(plan: Arc) -> Result { let mut graph = ExecutionGraph::new(); let root = generate_query_stages(plan, &mut graph)?; @@ -118,31 +117,50 @@ fn generate_query_stages( .collect::>>()?; let plan = with_new_children_if_necessary(plan, new_children)?; - if let Some(repart) = plan.as_any().downcast_ref::() { + debug!("plan = {}", displayable(plan.as_ref()).one_line()); + debug!("output_part = {:?}", plan.output_partitioning()); + + let new_plan = if let Some(repart) = plan.as_any().downcast_ref::() { match repart.partitioning() { &Partitioning::UnknownPartitioning(_) | &Partitioning::RoundRobinBatch(_) => { // just remove these Ok(repart.children()[0].clone()) } - partitioning_scheme => { - create_shuffle_exchange(plan.as_ref(), graph, partitioning_scheme.clone()) - } + partitioning_scheme => create_shuffle_exchange( + plan.children()[0].clone(), + graph, + partitioning_scheme.clone(), + ), } } else if plan .as_any() .downcast_ref::() .is_some() { - create_shuffle_exchange(plan.as_ref(), graph, Partitioning::UnknownPartitioning(1)) + let coalesce_input = plan.children()[0].clone(); + let partitioning_scheme = coalesce_input.output_partitioning(); + let new_input = create_shuffle_exchange(coalesce_input, graph, partitioning_scheme)?; + with_new_children_if_necessary(plan, vec![new_input]) } else if plan .as_any() .downcast_ref::() .is_some() { - create_shuffle_exchange(plan.as_ref(), graph, Partitioning::UnknownPartitioning(1)) + let partitioned_sort_plan = plan.children()[0].clone(); + let partitioning_scheme = partitioned_sort_plan.output_partitioning(); + let new_input = create_shuffle_exchange(partitioned_sort_plan, graph, partitioning_scheme)?; + with_new_children_if_necessary(plan, vec![new_input]) } else { Ok(plan) - } + }?; + + debug!("new_plan = {}", displayable(new_plan.as_ref()).one_line()); + debug!( + "new_output_part = {:?}\n\n-------------------------\n\n", + new_plan.output_partitioning() + ); + + Ok(new_plan) } /// Create a shuffle exchange. @@ -150,7 +168,7 @@ fn generate_query_stages( /// The plan is wrapped in a ShuffleWriteExec and added as a new query plan in the execution graph /// and a ShuffleReaderExec is returned to replace the plan. fn create_shuffle_exchange( - plan: &dyn ExecutionPlan, + plan: Arc, graph: &mut ExecutionGraph, partitioning_scheme: Partitioning, ) -> Result> { @@ -160,13 +178,19 @@ fn create_shuffle_exchange( // create temp dir for stage shuffle files let temp_dir = create_temp_dir(stage_id)?; - let shuffle_writer_input = plan.children()[0].clone(); + let shuffle_writer_input = plan.clone(); let shuffle_writer = ShuffleWriterExec::new( stage_id, shuffle_writer_input, partitioning_scheme.clone(), &temp_dir, ); + + debug!( + "Created shuffle writer with output partitioning {:?}", + shuffle_writer.output_partitioning() + ); + let stage_id = graph.add_query_stage(stage_id, Arc::new(shuffle_writer)); // replace the plan with a shuffle reader Ok(Arc::new(ShuffleReaderExec::new( @@ -345,7 +369,9 @@ mod test { for id in 0..=graph.get_final_query_stage().id { let query_stage = graph.query_stages.get(&id).unwrap(); output.push_str(&format!( - "Query Stage #{id}:\n{}\n", + "Query Stage #{id} ({} -> {}):\n{}\n", + query_stage.get_input_partition_count(), + query_stage.get_output_partition_count(), displayable(query_stage.plan.as_ref()).indent() )); } diff --git a/src/query_stage.rs b/src/query_stage.rs index 11bdcb8..3ab87a2 100644 --- a/src/query_stage.rs +++ b/src/query_stage.rs @@ -1,11 +1,11 @@ use crate::shuffle::{ShuffleCodec, ShuffleReaderExec}; +use datafusion::error::Result; use datafusion::physical_plan::ExecutionPlan; +use datafusion::prelude::SessionContext; +use datafusion_proto::bytes::physical_plan_from_bytes_with_extension_codec; use datafusion_python::physical_plan::PyExecutionPlan; use pyo3::prelude::*; use std::sync::Arc; -use datafusion::prelude::SessionContext; -use datafusion::error::Result; -use datafusion_proto::bytes::physical_plan_from_bytes_with_extension_codec; #[pyclass(name = "QueryStage", module = "raysql", subclass)] pub struct PyQueryStage { @@ -26,10 +26,7 @@ impl PyQueryStage { let codec = ShuffleCodec {}; let plan = physical_plan_from_bytes_with_extension_codec(&bytes, &ctx, &codec)?; Ok(PyQueryStage { - stage: Arc::new(QueryStage { - id, - plan - }) + stage: Arc::new(QueryStage { id, plan }), }) } @@ -74,7 +71,11 @@ impl QueryStage { /// Get the input partition count. This is the same as the number of concurrent tasks /// when we schedule this query stage for execution pub fn get_input_partition_count(&self) -> usize { - collect_input_partition_count(self.plan.as_ref()) + self.plan.children()[0].output_partitioning().partition_count() + } + + pub fn get_output_partition_count(&self) -> usize { + self.plan.output_partitioning().partition_count() } } @@ -87,13 +88,3 @@ fn collect_child_stage_ids(plan: &dyn ExecutionPlan, ids: &mut Vec) { } } } - -fn collect_input_partition_count(plan: &dyn ExecutionPlan) -> usize { - if plan.children().is_empty() { - plan.output_partitioning().partition_count() - } else { - // invariants: - // - all inputs must have the same partition count - collect_input_partition_count(plan.children()[0].as_ref()) - } -} \ No newline at end of file diff --git a/src/shuffle/codec.rs b/src/shuffle/codec.rs index 42508b2..464607b 100644 --- a/src/shuffle/codec.rs +++ b/src/shuffle/codec.rs @@ -109,9 +109,9 @@ fn encode_partitioning_scheme(partitioning: &Partitioning) -> Result, DataFusionError>>()?, partition_count: *partition_count as u64, }), - Partitioning::UnknownPartitioning(1) => Ok(protobuf::PhysicalHashRepartition { + Partitioning::UnknownPartitioning(n) => Ok(protobuf::PhysicalHashRepartition { hash_expr: vec![], - partition_count: 1, + partition_count: *n as u64, }), other => Err(DataFusionError::Plan(format!( "Unsupported shuffle partitioning scheme: {other:?}" diff --git a/src/shuffle/reader.rs b/src/shuffle/reader.rs index 9bdd287..7cec8d4 100644 --- a/src/shuffle/reader.rs +++ b/src/shuffle/reader.rs @@ -4,6 +4,7 @@ use datafusion::arrow::record_batch::RecordBatch; use datafusion::common::Statistics; use datafusion::error::DataFusionError; use datafusion::execution::context::TaskContext; +use datafusion::physical_expr::expressions::UnKnownColumn; use datafusion::physical_expr::PhysicalSortExpr; use datafusion::physical_plan::union::CombinedRecordBatchStream; use datafusion::physical_plan::{ @@ -38,6 +39,20 @@ impl ShuffleReaderExec { partitioning: Partitioning, shuffle_dir: &str, ) -> Self { + let partitioning = match partitioning { + Partitioning::Hash(expr, n) if expr.is_empty() => Partitioning::UnknownPartitioning(n), + Partitioning::Hash(expr, n) => { + // workaround for DataFusion bug https://github.com/apache/arrow-datafusion/issues/5184 + Partitioning::Hash( + expr.into_iter() + .filter(|e| e.as_any().downcast_ref::().is_none()) + .collect(), + n, + ) + }, + _ => partitioning, + }; + Self { stage_id, schema, @@ -88,7 +103,12 @@ impl ExecutionPlan for ShuffleReaderExec { let mut streams: Vec = vec![]; for entry in glob(&pattern).expect("Failed to read glob pattern") { let file = entry.unwrap(); - debug!("Shuffle reader reading from {}", file.display()); + debug!( + "ShuffleReaderExec partition {} reading from stage {} file {}", + partition, + self.stage_id, + file.display() + ); let reader = FileReader::try_new(File::open(&file)?, None)?; let stream = LocalShuffleStream::new(reader); if self.schema != stream.schema() { @@ -107,9 +127,8 @@ impl ExecutionPlan for ShuffleReaderExec { fn fmt_as(&self, _t: DisplayFormatType, f: &mut Formatter) -> std::fmt::Result { write!( f, - "ShuffleReaderExec(stage_id={}, input_partitions={})", - self.stage_id, - self.partitioning.partition_count() + "ShuffleReaderExec(stage_id={}, input_partitioning={:?})", + self.stage_id, self.partitioning ) } diff --git a/src/shuffle/writer.rs b/src/shuffle/writer.rs index 615d5ca..93ef619 100644 --- a/src/shuffle/writer.rs +++ b/src/shuffle/writer.rs @@ -5,6 +5,7 @@ use datafusion::arrow::record_batch::RecordBatch; use datafusion::arrow::util::pretty::pretty_format_batches; use datafusion::common::{Result, Statistics}; use datafusion::execution::context::TaskContext; +use datafusion::physical_expr::expressions::UnKnownColumn; use datafusion::physical_expr::PhysicalSortExpr; use datafusion::physical_plan::common::{batch_byte_size, IPCWriter}; use datafusion::physical_plan::memory::MemoryStream; @@ -45,6 +46,20 @@ impl ShuffleWriterExec { partitioning: Partitioning, shuffle_dir: &str, ) -> Self { + let partitioning = match partitioning { + Partitioning::Hash(expr, n) if expr.is_empty() => Partitioning::UnknownPartitioning(n), + Partitioning::Hash(expr, n) => { + // workaround for DataFusion bug https://github.com/apache/arrow-datafusion/issues/5184 + Partitioning::Hash( + expr.into_iter() + .filter(|e| e.as_any().downcast_ref::().is_none()) + .collect(), + n, + ) + }, + _ => partitioning, + }; + Self { stage_id, plan, @@ -97,7 +112,10 @@ impl ExecutionPlan for ShuffleWriterExec { input_partition: usize, context: Arc, ) -> Result { - debug!("ShuffleWriteExec::execute(input_partition={input_partition})"); + debug!( + "ShuffleWriterExec[stage={}].execute(input_partition={input_partition})", + self.stage_id + ); let mut stream = self.plan.execute(input_partition, context)?; let write_time = @@ -111,78 +129,90 @@ impl ExecutionPlan for ShuffleWriterExec { let shuffle_dir = self.shuffle_dir.clone(); let results = async move { - if partition_count == 1 { - // stream the results from the query - let file = format!("/{shuffle_dir}/shuffle_{stage_id}_{input_partition}_0.arrow",); - debug!("Executing query and writing results to {file}"); - let stats = write_stream_to_disk(&mut stream, &file, &write_time).await?; - debug!( - "Query completed. Shuffle write time: {}. Rows: {}.", - write_time, stats.num_rows - ); - } else { - // we won't necessary produce output for every possible partition, so we - // create writers on demand - let mut writers: Vec> = vec![]; - for _ in 0..partition_count { - writers.push(None); + match &partitioning { + Partitioning::RoundRobinBatch(_) => { + unimplemented!() + } + Partitioning::UnknownPartitioning(_) => { + // stream the results from the query, preserving the input partitioning + let file = + format!("/{shuffle_dir}/shuffle_{stage_id}_{input_partition}_0.arrow"); + debug!("Executing query and writing results to {file}"); + let stats = write_stream_to_disk(&mut stream, &file, &write_time).await?; + debug!( + "Query completed. Shuffle write time: {}. Rows: {}.", + write_time, stats.num_rows + ); } + Partitioning::Hash(_, _) => { + // we won't necessary produce output for every possible partition, so we + // create writers on demand + let mut writers: Vec> = vec![]; + for _ in 0..partition_count { + writers.push(None); + } - let mut partitioner = BatchPartitioner::try_new(partitioning, repart_time.clone())?; + let mut partitioner = + BatchPartitioner::try_new(partitioning, repart_time.clone())?; - let mut rows = 0; + let mut rows = 0; - while let Some(result) = stream.next().await { - let input_batch = result?; - rows += input_batch.num_rows(); + while let Some(result) = stream.next().await { + let input_batch = result?; + rows += input_batch.num_rows(); - debug!( - "ShuffleWriterExec writing batch:\n{}", - pretty_format_batches(&[input_batch.clone()])? - ); + debug!( + "ShuffleWriterExec[stage={}] writing batch:\n{}", + stage_id, + pretty_format_batches(&[input_batch.clone()])? + ); - //write_metrics.input_rows.add(input_batch.num_rows()); + //write_metrics.input_rows.add(input_batch.num_rows()); - partitioner.partition(input_batch, |output_partition, output_batch| { - match &mut writers[output_partition] { - Some(w) => { - w.write(&output_batch)?; - } - None => { - let path = format!( - "/{shuffle_dir}/shuffle_{stage_id}_{input_partition}_{output_partition}.arrow", - ); - let path = Path::new(&path); - debug!("Writing results to {:?}", path); + partitioner.partition(input_batch, |output_partition, output_batch| { + match &mut writers[output_partition] { + Some(w) => { + w.write(&output_batch)?; + } + None => { + let path = format!( + "/{shuffle_dir}/shuffle_{stage_id}_{input_partition}_{output_partition}.arrow", + ); + let path = Path::new(&path); + debug!("ShuffleWriterExec[stage={}] Writing results to {:?}", stage_id, path); - let mut writer = IPCWriter::new(path, stream.schema().as_ref())?; + let mut writer = IPCWriter::new(path, stream.schema().as_ref())?; - writer.write(&output_batch)?; - writers[output_partition] = Some(writer); + writer.write(&output_batch)?; + writers[output_partition] = Some(writer); + } } - } - Ok(()) - })?; - } + Ok(()) + })?; + } - for (i, w) in writers.iter_mut().enumerate() { - match w { - Some(w) => { - w.finish()?; - debug!( - "Finished writing shuffle partition {} at {:?}. Batches: {}. Rows: {}. Bytes: {}.", - i, - w.path(), - w.num_batches, - w.num_rows, - w.num_bytes - ); + for (i, w) in writers.iter_mut().enumerate() { + match w { + Some(w) => { + w.finish()?; + debug!( + "ShuffleWriterExec[stage={}] Finished writing shuffle partition {} at {:?}. Batches: {}. Rows: {}. Bytes: {}.", + stage_id, + i, + w.path(), + w.num_batches, + w.num_rows, + w.num_bytes + ); + } + None => {} } - None => {} } + debug!( + "ShuffleWriterExec[stage={}] Finished processing stream with {rows} rows", + stage_id + ); } - - debug!("finished processing stream with {rows} rows"); } // create a dummy batch to return - later this could be metadata about the diff --git a/testdata/expected-plans/q1.txt b/testdata/expected-plans/q1.txt index fbd08a1..7b0f3db 100644 --- a/testdata/expected-plans/q1.txt +++ b/testdata/expected-plans/q1.txt @@ -26,7 +26,7 @@ SortExec: [l_returnflag@0 ASC NULLS LAST,l_linestatus@1 ASC NULLS LAST] RaySQL Plan =========== -Query Stage #0: +Query Stage #0 (4 -> 4): ShuffleWriterExec(stage_id=0, output_partitioning=Hash([Column { name: "l_returnflag", index: 0 }, Column { name: "l_linestatus", index: 1 }], 4)) AggregateExec: mode=Partial, gby=[l_returnflag@5 as l_returnflag, l_linestatus@6 as l_linestatus], aggr=[SUM(lineitem.l_quantity), SUM(lineitem.l_extendedprice), SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount), SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount * Int64(1) + lineitem.l_tax), AVG(lineitem.l_quantity), AVG(lineitem.l_extendedprice), AVG(lineitem.l_discount), COUNT(UInt8(1))] ProjectionExec: expr=[CAST(l_extendedprice@1 AS Decimal128(35, 4)) * CAST(Some(100),23,2 - CAST(l_discount@2 AS Decimal128(23, 2)) AS Decimal128(35, 4)) as CAST(lineitem.l_extendedprice AS Decimal128(35, 4)) * CAST(Decimal128(Some(100),23,2) - CAST(lineitem.l_discount AS Decimal128(23, 2)) AS Decimal128(35, 4))CAST(Decimal128(Some(100),23,2) - CAST(lineitem.l_discount AS Decimal128(23, 2)) AS Decimal128(35, 4))Decimal128(Some(100),23,2) - CAST(lineitem.l_discount AS Decimal128(23, 2))CAST(lineitem.l_discount AS Decimal128(23, 2))lineitem.l_discountDecimal128(Some(100),23,2)CAST(lineitem.l_extendedprice AS Decimal128(35, 4))lineitem.l_extendedprice, l_quantity@0 as l_quantity, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, l_tax@3 as l_tax, l_returnflag@4 as l_returnflag, l_linestatus@5 as l_linestatus] @@ -34,14 +34,15 @@ ShuffleWriterExec(stage_id=0, output_partitioning=Hash([Column { name: "l_return FilterExec: l_shipdate@6 <= 10493 ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-3.parquet]]}, predicate=l_shipdate <= Date32("10493"), pruning_predicate=l_shipdate_min@0 <= 10493, projection=[l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate] -Query Stage #1: -ShuffleWriterExec(stage_id=1, output_partitioning=UnknownPartitioning(1)) +Query Stage #1 (4 -> 4): +ShuffleWriterExec(stage_id=1, output_partitioning=Hash([Column { name: "l_returnflag", index: 0 }, Column { name: "l_linestatus", index: 1 }], 4)) ProjectionExec: expr=[l_returnflag@0 as l_returnflag, l_linestatus@1 as l_linestatus, SUM(lineitem.l_quantity)@2 as sum_qty, SUM(lineitem.l_extendedprice)@3 as sum_base_price, SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@4 as sum_disc_price, SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount * Int64(1) + lineitem.l_tax)@5 as sum_charge, AVG(lineitem.l_quantity)@6 as avg_qty, AVG(lineitem.l_extendedprice)@7 as avg_price, AVG(lineitem.l_discount)@8 as avg_disc, COUNT(UInt8(1))@9 as count_order] AggregateExec: mode=FinalPartitioned, gby=[l_returnflag@0 as l_returnflag, l_linestatus@1 as l_linestatus], aggr=[SUM(lineitem.l_quantity), SUM(lineitem.l_extendedprice), SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount), SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount * Int64(1) + lineitem.l_tax), AVG(lineitem.l_quantity), AVG(lineitem.l_extendedprice), AVG(lineitem.l_discount), COUNT(UInt8(1))] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=0, input_partitions=4) + ShuffleReaderExec(stage_id=0, input_partitioning=Hash([Column { name: "l_returnflag", index: 0 }, Column { name: "l_linestatus", index: 1 }], 4)) -Query Stage #2: +Query Stage #2 (1 -> 1): SortExec: [l_returnflag@0 ASC NULLS LAST,l_linestatus@1 ASC NULLS LAST] - ShuffleReaderExec(stage_id=1, input_partitions=1) + CoalescePartitionsExec + ShuffleReaderExec(stage_id=1, input_partitioning=Hash([Column { name: "l_returnflag", index: 0 }, Column { name: "l_linestatus", index: 1 }], 4)) diff --git a/testdata/expected-plans/q10.txt b/testdata/expected-plans/q10.txt index c16708f..c4c3a7c 100644 --- a/testdata/expected-plans/q10.txt +++ b/testdata/expected-plans/q10.txt @@ -56,63 +56,64 @@ GlobalLimitExec: skip=0, fetch=20 RaySQL Plan =========== -Query Stage #0: +Query Stage #0 (4 -> 4): ShuffleWriterExec(stage_id=0, output_partitioning=Hash([Column { name: "c_custkey", index: 0 }], 4)) ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-3.parquet]]}, projection=[c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_comment] -Query Stage #1: +Query Stage #1 (4 -> 4): ShuffleWriterExec(stage_id=1, output_partitioning=Hash([Column { name: "o_custkey", index: 1 }], 4)) CoalesceBatchesExec: target_batch_size=8192 FilterExec: o_orderdate@2 >= 8582 AND o_orderdate@2 < 8674 ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-3.parquet]]}, predicate=o_orderdate >= Date32("8582") AND o_orderdate < Date32("8674"), pruning_predicate=o_orderdate_max@0 >= 8582 AND o_orderdate_min@1 < 8674, projection=[o_orderkey, o_custkey, o_orderdate] -Query Stage #2: +Query Stage #2 (4 -> 4): ShuffleWriterExec(stage_id=2, output_partitioning=Hash([Column { name: "o_orderkey", index: 7 }], 4)) CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "c_custkey", index: 0 }, Column { name: "o_custkey", index: 1 })] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=0, input_partitions=4) + ShuffleReaderExec(stage_id=0, input_partitioning=Hash([Column { name: "c_custkey", index: 0 }], 4)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=1, input_partitions=4) + ShuffleReaderExec(stage_id=1, input_partitioning=Hash([Column { name: "o_custkey", index: 1 }], 4)) -Query Stage #3: +Query Stage #3 (4 -> 4): ShuffleWriterExec(stage_id=3, output_partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 4)) CoalesceBatchesExec: target_batch_size=8192 FilterExec: l_returnflag@3 = R ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-3.parquet]]}, predicate=l_returnflag = Utf8("R"), pruning_predicate=l_returnflag_min@0 <= R AND R <= l_returnflag_max@1, projection=[l_orderkey, l_extendedprice, l_discount, l_returnflag] -Query Stage #4: +Query Stage #4 (4 -> 4): ShuffleWriterExec(stage_id=4, output_partitioning=Hash([Column { name: "c_nationkey", index: 3 }], 4)) CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "o_orderkey", index: 7 }, Column { name: "l_orderkey", index: 0 })] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=2, input_partitions=4) + ShuffleReaderExec(stage_id=2, input_partitioning=Hash([Column { name: "o_orderkey", index: 7 }], 4)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=3, input_partitions=4) + ShuffleReaderExec(stage_id=3, input_partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 4)) -Query Stage #5: +Query Stage #5 (1 -> 4): ShuffleWriterExec(stage_id=5, output_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 4)) ParquetExec: limit=None, partitions={1 group: [[mnt/bigdata/tpch/sf10-parquet/nation.parquet/part-0.parquet]]}, projection=[n_nationkey, n_name] -Query Stage #6: +Query Stage #6 (4 -> 4): ShuffleWriterExec(stage_id=6, output_partitioning=Hash([Column { name: "c_custkey", index: 0 }, Column { name: "c_name", index: 1 }, Column { name: "c_acctbal", index: 2 }, Column { name: "c_phone", index: 3 }, Column { name: "n_name", index: 4 }, Column { name: "c_address", index: 5 }, Column { name: "c_comment", index: 6 }], 4)) AggregateExec: mode=Partial, gby=[c_custkey@0 as c_custkey, c_name@1 as c_name, c_acctbal@5 as c_acctbal, c_phone@4 as c_phone, n_name@15 as n_name, c_address@2 as c_address, c_comment@6 as c_comment], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "c_nationkey", index: 3 }, Column { name: "n_nationkey", index: 0 })] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=4, input_partitions=4) + ShuffleReaderExec(stage_id=4, input_partitioning=Hash([Column { name: "c_nationkey", index: 3 }], 4)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=5, input_partitions=4) + ShuffleReaderExec(stage_id=5, input_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 4)) -Query Stage #7: -ShuffleWriterExec(stage_id=7, output_partitioning=UnknownPartitioning(1)) +Query Stage #7 (4 -> 4): +ShuffleWriterExec(stage_id=7, output_partitioning=Hash([Column { name: "c_custkey", index: 0 }, Column { name: "c_name", index: 1 }, Column { name: "c_acctbal", index: 3 }, Column { name: "c_phone", index: 6 }, Column { name: "n_name", index: 4 }, Column { name: "c_address", index: 5 }, Column { name: "c_comment", index: 7 }], 4)) SortExec: [revenue@2 DESC] ProjectionExec: expr=[c_custkey@0 as c_custkey, c_name@1 as c_name, SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@7 as revenue, c_acctbal@2 as c_acctbal, n_name@4 as n_name, c_address@5 as c_address, c_phone@3 as c_phone, c_comment@6 as c_comment] AggregateExec: mode=FinalPartitioned, gby=[c_custkey@0 as c_custkey, c_name@1 as c_name, c_acctbal@2 as c_acctbal, c_phone@3 as c_phone, n_name@4 as n_name, c_address@5 as c_address, c_comment@6 as c_comment], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=6, input_partitions=4) + ShuffleReaderExec(stage_id=6, input_partitioning=Hash([Column { name: "c_custkey", index: 0 }, Column { name: "c_name", index: 1 }, Column { name: "c_acctbal", index: 2 }, Column { name: "c_phone", index: 3 }, Column { name: "n_name", index: 4 }, Column { name: "c_address", index: 5 }, Column { name: "c_comment", index: 6 }], 4)) -Query Stage #8: +Query Stage #8 (1 -> 1): GlobalLimitExec: skip=0, fetch=20 - ShuffleReaderExec(stage_id=7, input_partitions=1) + SortPreservingMergeExec: [revenue@2 DESC] + ShuffleReaderExec(stage_id=7, input_partitioning=Hash([Column { name: "c_custkey", index: 0 }, Column { name: "c_name", index: 1 }, Column { name: "c_acctbal", index: 3 }, Column { name: "c_phone", index: 6 }, Column { name: "n_name", index: 4 }, Column { name: "c_address", index: 5 }, Column { name: "c_comment", index: 7 }], 4)) diff --git a/testdata/expected-plans/q11.txt b/testdata/expected-plans/q11.txt index baf3df2..8155849 100644 --- a/testdata/expected-plans/q11.txt +++ b/testdata/expected-plans/q11.txt @@ -81,88 +81,90 @@ SortExec: [value@1 DESC] RaySQL Plan =========== -Query Stage #0: +Query Stage #0 (4 -> 4): ShuffleWriterExec(stage_id=0, output_partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 4)) ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-3.parquet]]}, projection=[s_suppkey, s_nationkey] -Query Stage #1: +Query Stage #1 (4 -> 4): ShuffleWriterExec(stage_id=1, output_partitioning=Hash([Column { name: "ps_suppkey", index: 1 }], 4)) ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-3.parquet]]}, projection=[ps_partkey, ps_suppkey, ps_availqty, ps_supplycost] -Query Stage #2: +Query Stage #2 (4 -> 4): ShuffleWriterExec(stage_id=2, output_partitioning=Hash([Column { name: "s_nationkey", index: 5 }], 4)) ProjectionExec: expr=[ps_partkey@2 as ps_partkey, ps_suppkey@3 as ps_suppkey, ps_availqty@4 as ps_availqty, ps_supplycost@5 as ps_supplycost, s_suppkey@0 as s_suppkey, s_nationkey@1 as s_nationkey] CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "s_suppkey", index: 0 }, Column { name: "ps_suppkey", index: 1 })] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=0, input_partitions=4) + ShuffleReaderExec(stage_id=0, input_partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 4)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=1, input_partitions=4) + ShuffleReaderExec(stage_id=1, input_partitioning=Hash([Column { name: "ps_suppkey", index: 1 }], 4)) -Query Stage #3: +Query Stage #3 (1 -> 4): ShuffleWriterExec(stage_id=3, output_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 4)) CoalesceBatchesExec: target_batch_size=8192 FilterExec: n_name@1 = ALGERIA ParquetExec: limit=None, partitions={1 group: [[mnt/bigdata/tpch/sf10-parquet/nation.parquet/part-0.parquet]]}, predicate=n_name = Utf8("ALGERIA"), pruning_predicate=n_name_min@0 <= ALGERIA AND ALGERIA <= n_name_max@1, projection=[n_nationkey, n_name] -Query Stage #4: +Query Stage #4 (4 -> 4): ShuffleWriterExec(stage_id=4, output_partitioning=Hash([Column { name: "ps_partkey", index: 0 }], 4)) AggregateExec: mode=Partial, gby=[ps_partkey@0 as ps_partkey], aggr=[SUM(partsupp.ps_supplycost * partsupp.ps_availqty)] CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "s_nationkey", index: 5 }, Column { name: "n_nationkey", index: 0 })] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=2, input_partitions=4) + ShuffleReaderExec(stage_id=2, input_partitioning=Hash([Column { name: "s_nationkey", index: 5 }], 4)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=3, input_partitions=4) + ShuffleReaderExec(stage_id=3, input_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 4)) -Query Stage #5: -ShuffleWriterExec(stage_id=5, output_partitioning=UnknownPartitioning(1)) +Query Stage #5 (4 -> 4): +ShuffleWriterExec(stage_id=5, output_partitioning=Hash([Column { name: "ps_partkey", index: 0 }], 4)) AggregateExec: mode=FinalPartitioned, gby=[ps_partkey@0 as ps_partkey], aggr=[SUM(partsupp.ps_supplycost * partsupp.ps_availqty)] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=4, input_partitions=4) + ShuffleReaderExec(stage_id=4, input_partitioning=Hash([Column { name: "ps_partkey", index: 0 }], 4)) -Query Stage #6: +Query Stage #6 (4 -> 4): ShuffleWriterExec(stage_id=6, output_partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 4)) ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-3.parquet]]}, projection=[s_suppkey, s_nationkey] -Query Stage #7: +Query Stage #7 (4 -> 4): ShuffleWriterExec(stage_id=7, output_partitioning=Hash([Column { name: "ps_suppkey", index: 0 }], 4)) ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-3.parquet]]}, projection=[ps_suppkey, ps_availqty, ps_supplycost] -Query Stage #8: +Query Stage #8 (4 -> 4): ShuffleWriterExec(stage_id=8, output_partitioning=Hash([Column { name: "s_nationkey", index: 4 }], 4)) ProjectionExec: expr=[ps_suppkey@2 as ps_suppkey, ps_availqty@3 as ps_availqty, ps_supplycost@4 as ps_supplycost, s_suppkey@0 as s_suppkey, s_nationkey@1 as s_nationkey] CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "s_suppkey", index: 0 }, Column { name: "ps_suppkey", index: 0 })] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=6, input_partitions=4) + ShuffleReaderExec(stage_id=6, input_partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 4)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=7, input_partitions=4) + ShuffleReaderExec(stage_id=7, input_partitioning=Hash([Column { name: "ps_suppkey", index: 0 }], 4)) -Query Stage #9: +Query Stage #9 (1 -> 4): ShuffleWriterExec(stage_id=9, output_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 4)) CoalesceBatchesExec: target_batch_size=8192 FilterExec: n_name@1 = ALGERIA ParquetExec: limit=None, partitions={1 group: [[mnt/bigdata/tpch/sf10-parquet/nation.parquet/part-0.parquet]]}, predicate=n_name = Utf8("ALGERIA"), pruning_predicate=n_name_min@0 <= ALGERIA AND ALGERIA <= n_name_max@1, projection=[n_nationkey, n_name] -Query Stage #10: -ShuffleWriterExec(stage_id=10, output_partitioning=UnknownPartitioning(1)) +Query Stage #10 (4 -> 4): +ShuffleWriterExec(stage_id=10, output_partitioning=Hash([], 4)) AggregateExec: mode=Partial, gby=[], aggr=[SUM(partsupp.ps_supplycost * partsupp.ps_availqty)] CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "s_nationkey", index: 4 }, Column { name: "n_nationkey", index: 0 })] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=8, input_partitions=4) + ShuffleReaderExec(stage_id=8, input_partitioning=Hash([Column { name: "s_nationkey", index: 4 }], 4)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=9, input_partitions=4) + ShuffleReaderExec(stage_id=9, input_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 4)) -Query Stage #11: +Query Stage #11 (1 -> 1): SortExec: [value@1 DESC] ProjectionExec: expr=[ps_partkey@0 as ps_partkey, SUM(partsupp.ps_supplycost * partsupp.ps_availqty)@1 as value] CoalesceBatchesExec: target_batch_size=8192 FilterExec: CAST(SUM(partsupp.ps_supplycost * partsupp.ps_availqty)@1 AS Decimal128(38, 15)) > CAST(__value@2 AS Decimal128(38, 15)) CrossJoinExec - ShuffleReaderExec(stage_id=5, input_partitions=1) + CoalescePartitionsExec + ShuffleReaderExec(stage_id=5, input_partitioning=Hash([Column { name: "ps_partkey", index: 0 }], 4)) ProjectionExec: expr=[CAST(SUM(partsupp.ps_supplycost * partsupp.ps_availqty)@0 AS Float64) * 0.0001 as __value] AggregateExec: mode=Final, gby=[], aggr=[SUM(partsupp.ps_supplycost * partsupp.ps_availqty)] - ShuffleReaderExec(stage_id=10, input_partitions=1) + CoalescePartitionsExec + ShuffleReaderExec(stage_id=10, input_partitioning=Hash([], 4)) diff --git a/testdata/expected-plans/q12.txt b/testdata/expected-plans/q12.txt index c9fe0bc..3270d3b 100644 --- a/testdata/expected-plans/q12.txt +++ b/testdata/expected-plans/q12.txt @@ -33,34 +33,35 @@ SortExec: [l_shipmode@0 ASC NULLS LAST] RaySQL Plan =========== -Query Stage #0: +Query Stage #0 (4 -> 4): ShuffleWriterExec(stage_id=0, output_partitioning=Hash([Column { name: "o_orderkey", index: 0 }], 4)) ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-3.parquet]]}, projection=[o_orderkey, o_orderpriority] -Query Stage #1: +Query Stage #1 (4 -> 4): ShuffleWriterExec(stage_id=1, output_partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 4)) CoalesceBatchesExec: target_batch_size=8192 FilterExec: l_shipmode@4 = SHIP OR l_shipmode@4 = FOB AND l_commitdate@2 < l_receiptdate@3 AND l_shipdate@1 < l_commitdate@2 AND l_receiptdate@3 >= 9131 AND l_receiptdate@3 < 9496 ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-3.parquet]]}, predicate=(l_shipmode = Utf8("SHIP") OR l_shipmode = Utf8("FOB")) AND l_commitdate < l_receiptdate AND l_shipdate < l_commitdate AND l_receiptdate >= Date32("9131") AND l_receiptdate < Date32("9496"), pruning_predicate=l_shipmode_min@0 <= SHIP AND SHIP <= l_shipmode_max@1 OR l_shipmode_min@0 <= FOB AND FOB <= l_shipmode_max@1 AND l_receiptdate_max@2 >= 9131 AND l_receiptdate_min@3 < 9496, projection=[l_orderkey, l_shipdate, l_commitdate, l_receiptdate, l_shipmode] -Query Stage #2: +Query Stage #2 (4 -> 4): ShuffleWriterExec(stage_id=2, output_partitioning=Hash([Column { name: "l_shipmode", index: 0 }], 4)) AggregateExec: mode=Partial, gby=[l_shipmode@6 as l_shipmode], aggr=[SUM(CASE WHEN orders.o_orderpriority = Utf8("1-URGENT") OR orders.o_orderpriority = Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END), SUM(CASE WHEN orders.o_orderpriority != Utf8("1-URGENT") AND orders.o_orderpriority != Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END)] CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "o_orderkey", index: 0 }, Column { name: "l_orderkey", index: 0 })] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=0, input_partitions=4) + ShuffleReaderExec(stage_id=0, input_partitioning=Hash([Column { name: "o_orderkey", index: 0 }], 4)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=1, input_partitions=4) + ShuffleReaderExec(stage_id=1, input_partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 4)) -Query Stage #3: -ShuffleWriterExec(stage_id=3, output_partitioning=UnknownPartitioning(1)) +Query Stage #3 (4 -> 4): +ShuffleWriterExec(stage_id=3, output_partitioning=Hash([Column { name: "l_shipmode", index: 0 }], 4)) ProjectionExec: expr=[l_shipmode@0 as l_shipmode, SUM(CASE WHEN orders.o_orderpriority = Utf8("1-URGENT") OR orders.o_orderpriority = Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END)@1 as high_line_count, SUM(CASE WHEN orders.o_orderpriority != Utf8("1-URGENT") AND orders.o_orderpriority != Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END)@2 as low_line_count] AggregateExec: mode=FinalPartitioned, gby=[l_shipmode@0 as l_shipmode], aggr=[SUM(CASE WHEN orders.o_orderpriority = Utf8("1-URGENT") OR orders.o_orderpriority = Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END), SUM(CASE WHEN orders.o_orderpriority != Utf8("1-URGENT") AND orders.o_orderpriority != Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END)] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=2, input_partitions=4) + ShuffleReaderExec(stage_id=2, input_partitioning=Hash([Column { name: "l_shipmode", index: 0 }], 4)) -Query Stage #4: +Query Stage #4 (1 -> 1): SortExec: [l_shipmode@0 ASC NULLS LAST] - ShuffleReaderExec(stage_id=3, input_partitions=1) + CoalescePartitionsExec + ShuffleReaderExec(stage_id=3, input_partitioning=Hash([Column { name: "l_shipmode", index: 0 }], 4)) diff --git a/testdata/expected-plans/q13.txt b/testdata/expected-plans/q13.txt index 0e38652..2b0236b 100644 --- a/testdata/expected-plans/q13.txt +++ b/testdata/expected-plans/q13.txt @@ -41,17 +41,17 @@ SortExec: [custdist@1 DESC,c_count@0 DESC] RaySQL Plan =========== -Query Stage #0: +Query Stage #0 (4 -> 4): ShuffleWriterExec(stage_id=0, output_partitioning=Hash([Column { name: "c_custkey", index: 0 }], 4)) ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-3.parquet]]}, projection=[c_custkey] -Query Stage #1: +Query Stage #1 (4 -> 4): ShuffleWriterExec(stage_id=1, output_partitioning=Hash([Column { name: "o_custkey", index: 1 }], 4)) CoalesceBatchesExec: target_batch_size=8192 FilterExec: o_comment@2 NOT LIKE %express%requests% ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-3.parquet]]}, predicate=o_comment NOT LIKE Utf8("%express%requests%"), projection=[o_orderkey, o_custkey, o_comment] -Query Stage #2: +Query Stage #2 (4 -> 4): ShuffleWriterExec(stage_id=2, output_partitioning=Hash([Column { name: "c_count", index: 0 }], 4)) AggregateExec: mode=Partial, gby=[c_count@0 as c_count], aggr=[COUNT(UInt8(1))] ProjectionExec: expr=[COUNT(orders.o_orderkey)@0 as c_count] @@ -61,18 +61,19 @@ ShuffleWriterExec(stage_id=2, output_partitioning=Hash([Column { name: "c_count" CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=Left, on=[(Column { name: "c_custkey", index: 0 }, Column { name: "o_custkey", index: 1 })] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=0, input_partitions=4) + ShuffleReaderExec(stage_id=0, input_partitioning=Hash([Column { name: "c_custkey", index: 0 }], 4)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=1, input_partitions=4) + ShuffleReaderExec(stage_id=1, input_partitioning=Hash([Column { name: "o_custkey", index: 1 }], 4)) -Query Stage #3: -ShuffleWriterExec(stage_id=3, output_partitioning=UnknownPartitioning(1)) +Query Stage #3 (4 -> 4): +ShuffleWriterExec(stage_id=3, output_partitioning=Hash([Column { name: "c_count", index: 0 }], 4)) ProjectionExec: expr=[c_count@0 as c_count, COUNT(UInt8(1))@1 as custdist] AggregateExec: mode=FinalPartitioned, gby=[c_count@0 as c_count], aggr=[COUNT(UInt8(1))] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=2, input_partitions=4) + ShuffleReaderExec(stage_id=2, input_partitioning=Hash([Column { name: "c_count", index: 0 }], 4)) -Query Stage #4: +Query Stage #4 (1 -> 1): SortExec: [custdist@1 DESC,c_count@0 DESC] - ShuffleReaderExec(stage_id=3, input_partitions=1) + CoalescePartitionsExec + ShuffleReaderExec(stage_id=3, input_partitioning=Hash([Column { name: "c_count", index: 0 }], 4)) diff --git a/testdata/expected-plans/q14.txt b/testdata/expected-plans/q14.txt index 7add99a..88c3890 100644 --- a/testdata/expected-plans/q14.txt +++ b/testdata/expected-plans/q14.txt @@ -31,29 +31,30 @@ ProjectionExec: expr=[100 * CAST(SUM(CASE WHEN part.p_type LIKE Utf8("PROMO%") RaySQL Plan =========== -Query Stage #0: +Query Stage #0 (4 -> 4): ShuffleWriterExec(stage_id=0, output_partitioning=Hash([Column { name: "l_partkey", index: 0 }], 4)) CoalesceBatchesExec: target_batch_size=8192 FilterExec: l_shipdate@3 >= 9162 AND l_shipdate@3 < 9190 ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-3.parquet]]}, predicate=l_shipdate >= Date32("9162") AND l_shipdate < Date32("9190"), pruning_predicate=l_shipdate_max@0 >= 9162 AND l_shipdate_min@1 < 9190, projection=[l_partkey, l_extendedprice, l_discount, l_shipdate] -Query Stage #1: +Query Stage #1 (4 -> 4): ShuffleWriterExec(stage_id=1, output_partitioning=Hash([Column { name: "p_partkey", index: 0 }], 4)) ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/part.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/part.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/part.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/part.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-3.parquet]]}, projection=[p_partkey, p_type] -Query Stage #2: -ShuffleWriterExec(stage_id=2, output_partitioning=UnknownPartitioning(1)) +Query Stage #2 (4 -> 4): +ShuffleWriterExec(stage_id=2, output_partitioning=Hash([], 4)) AggregateExec: mode=Partial, gby=[], aggr=[SUM(CASE WHEN part.p_type LIKE Utf8("PROMO%") THEN lineitem.l_extendedprice * Int64(1) - lineitem.l_discount ELSE Int64(0) END), SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] ProjectionExec: expr=[CAST(l_extendedprice@1 AS Decimal128(35, 4)) * CAST(Some(100),23,2 - CAST(l_discount@2 AS Decimal128(23, 2)) AS Decimal128(35, 4)) as CAST(lineitem.l_extendedprice AS Decimal128(35, 4)) * CAST(Decimal128(Some(100),23,2) - CAST(lineitem.l_discount AS Decimal128(23, 2)) AS Decimal128(35, 4))CAST(Decimal128(Some(100),23,2) - CAST(lineitem.l_discount AS Decimal128(23, 2)) AS Decimal128(35, 4))Decimal128(Some(100),23,2) - CAST(lineitem.l_discount AS Decimal128(23, 2))CAST(lineitem.l_discount AS Decimal128(23, 2))lineitem.l_discountDecimal128(Some(100),23,2)CAST(lineitem.l_extendedprice AS Decimal128(35, 4))lineitem.l_extendedprice, p_type@5 as p_type] CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "l_partkey", index: 0 }, Column { name: "p_partkey", index: 0 })] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=0, input_partitions=4) + ShuffleReaderExec(stage_id=0, input_partitioning=Hash([Column { name: "l_partkey", index: 0 }], 4)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=1, input_partitions=4) + ShuffleReaderExec(stage_id=1, input_partitioning=Hash([Column { name: "p_partkey", index: 0 }], 4)) -Query Stage #3: +Query Stage #3 (1 -> 1): ProjectionExec: expr=[100 * CAST(SUM(CASE WHEN part.p_type LIKE Utf8("PROMO%") THEN lineitem.l_extendedprice * Int64(1) - lineitem.l_discount ELSE Int64(0) END)@0 AS Float64) / CAST(SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 AS Float64) as promo_revenue] AggregateExec: mode=Final, gby=[], aggr=[SUM(CASE WHEN part.p_type LIKE Utf8("PROMO%") THEN lineitem.l_extendedprice * Int64(1) - lineitem.l_discount ELSE Int64(0) END), SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] - ShuffleReaderExec(stage_id=2, input_partitions=1) + CoalescePartitionsExec + ShuffleReaderExec(stage_id=2, input_partitioning=Hash([], 4)) diff --git a/testdata/expected-plans/q16.txt b/testdata/expected-plans/q16.txt index 75a8de8..a1179c0 100644 --- a/testdata/expected-plans/q16.txt +++ b/testdata/expected-plans/q16.txt @@ -55,58 +55,59 @@ SortExec: [supplier_cnt@3 DESC,p_brand@0 ASC NULLS LAST,p_type@1 ASC NULLS LAST, RaySQL Plan =========== -Query Stage #0: +Query Stage #0 (4 -> 4): ShuffleWriterExec(stage_id=0, output_partitioning=Hash([Column { name: "ps_partkey", index: 0 }], 4)) ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-3.parquet]]}, projection=[ps_partkey, ps_suppkey] -Query Stage #1: +Query Stage #1 (4 -> 4): ShuffleWriterExec(stage_id=1, output_partitioning=Hash([Column { name: "p_partkey", index: 0 }], 4)) CoalesceBatchesExec: target_batch_size=8192 FilterExec: p_brand@1 != Brand#14 AND p_type@2 NOT LIKE SMALL PLATED% AND Use p_size@3 IN (SET) ([Literal { value: Int32(14) }, Literal { value: Int32(6) }, Literal { value: Int32(5) }, Literal { value: Int32(31) }, Literal { value: Int32(49) }, Literal { value: Int32(15) }, Literal { value: Int32(41) }, Literal { value: Int32(47) }]) ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/part.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/part.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/part.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/part.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-3.parquet]]}, predicate=p_brand != Utf8("Brand#14") AND p_type NOT LIKE Utf8("SMALL PLATED%") AND p_size IN ([Int32(14), Int32(6), Int32(5), Int32(31), Int32(49), Int32(15), Int32(41), Int32(47)]), pruning_predicate=p_brand_min@0 != Brand#14 OR Brand#14 != p_brand_max@1 AND p_size_min@2 <= 14 AND 14 <= p_size_max@3 OR p_size_min@2 <= 6 AND 6 <= p_size_max@3 OR p_size_min@2 <= 5 AND 5 <= p_size_max@3 OR p_size_min@2 <= 31 AND 31 <= p_size_max@3 OR p_size_min@2 <= 49 AND 49 <= p_size_max@3 OR p_size_min@2 <= 15 AND 15 <= p_size_max@3 OR p_size_min@2 <= 41 AND 41 <= p_size_max@3 OR p_size_min@2 <= 47 AND 47 <= p_size_max@3, projection=[p_partkey, p_brand, p_type, p_size] -Query Stage #2: +Query Stage #2 (4 -> 4): ShuffleWriterExec(stage_id=2, output_partitioning=Hash([Column { name: "ps_suppkey", index: 1 }], 4)) CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "ps_partkey", index: 0 }, Column { name: "p_partkey", index: 0 })] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=0, input_partitions=4) + ShuffleReaderExec(stage_id=0, input_partitioning=Hash([Column { name: "ps_partkey", index: 0 }], 4)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=1, input_partitions=4) + ShuffleReaderExec(stage_id=1, input_partitioning=Hash([Column { name: "p_partkey", index: 0 }], 4)) -Query Stage #3: +Query Stage #3 (4 -> 4): ShuffleWriterExec(stage_id=3, output_partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 4)) ProjectionExec: expr=[s_suppkey@0 as s_suppkey] CoalesceBatchesExec: target_batch_size=8192 FilterExec: s_comment@1 LIKE %Customer%Complaints% ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-3.parquet]]}, predicate=s_comment LIKE Utf8("%Customer%Complaints%"), projection=[s_suppkey, s_comment] -Query Stage #4: +Query Stage #4 (4 -> 4): ShuffleWriterExec(stage_id=4, output_partitioning=Hash([Column { name: "group_alias_0", index: 0 }, Column { name: "group_alias_1", index: 1 }, Column { name: "group_alias_2", index: 2 }, Column { name: "alias1", index: 3 }], 4)) AggregateExec: mode=Partial, gby=[p_brand@3 as group_alias_0, p_type@4 as group_alias_1, p_size@5 as group_alias_2, ps_suppkey@1 as alias1], aggr=[] CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(Column { name: "ps_suppkey", index: 1 }, Column { name: "s_suppkey", index: 0 })] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=2, input_partitions=4) + ShuffleReaderExec(stage_id=2, input_partitioning=Hash([Column { name: "ps_suppkey", index: 1 }], 4)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=3, input_partitions=4) + ShuffleReaderExec(stage_id=3, input_partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 4)) -Query Stage #5: +Query Stage #5 (4 -> 4): ShuffleWriterExec(stage_id=5, output_partitioning=Hash([Column { name: "group_alias_0", index: 0 }, Column { name: "group_alias_1", index: 1 }, Column { name: "group_alias_2", index: 2 }], 4)) AggregateExec: mode=Partial, gby=[group_alias_0@0 as group_alias_0, group_alias_1@1 as group_alias_1, group_alias_2@2 as group_alias_2], aggr=[COUNT(alias1)] AggregateExec: mode=FinalPartitioned, gby=[group_alias_0@0 as group_alias_0, group_alias_1@1 as group_alias_1, group_alias_2@2 as group_alias_2, alias1@3 as alias1], aggr=[] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=4, input_partitions=4) + ShuffleReaderExec(stage_id=4, input_partitioning=Hash([Column { name: "group_alias_0", index: 0 }, Column { name: "group_alias_1", index: 1 }, Column { name: "group_alias_2", index: 2 }, Column { name: "alias1", index: 3 }], 4)) -Query Stage #6: -ShuffleWriterExec(stage_id=6, output_partitioning=UnknownPartitioning(1)) +Query Stage #6 (4 -> 4): +ShuffleWriterExec(stage_id=6, output_partitioning=Hash([Column { name: "part.p_brand", index: 0 }, Column { name: "part.p_type", index: 1 }, Column { name: "part.p_size", index: 2 }], 4)) ProjectionExec: expr=[p_brand@0 as part.p_brand, p_type@1 as part.p_type, p_size@2 as part.p_size, COUNT(DISTINCT partsupp.ps_suppkey)@3 as supplier_cnt] ProjectionExec: expr=[group_alias_0@0 as part.p_brand, group_alias_1@1 as part.p_type, group_alias_2@2 as part.p_size, COUNT(alias1)@3 as COUNT(DISTINCT partsupp.ps_suppkey)] AggregateExec: mode=FinalPartitioned, gby=[group_alias_0@0 as group_alias_0, group_alias_1@1 as group_alias_1, group_alias_2@2 as group_alias_2], aggr=[COUNT(alias1)] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=5, input_partitions=4) + ShuffleReaderExec(stage_id=5, input_partitioning=Hash([Column { name: "group_alias_0", index: 0 }, Column { name: "group_alias_1", index: 1 }, Column { name: "group_alias_2", index: 2 }], 4)) -Query Stage #7: +Query Stage #7 (1 -> 1): SortExec: [supplier_cnt@3 DESC,p_brand@0 ASC NULLS LAST,p_type@1 ASC NULLS LAST,p_size@2 ASC NULLS LAST] - ShuffleReaderExec(stage_id=6, input_partitions=1) + CoalescePartitionsExec + ShuffleReaderExec(stage_id=6, input_partitioning=Hash([Column { name: "part.p_brand", index: 0 }, Column { name: "part.p_type", index: 1 }, Column { name: "part.p_size", index: 2 }], 4)) diff --git a/testdata/expected-plans/q17.txt b/testdata/expected-plans/q17.txt index 9709bc3..b6613cd 100644 --- a/testdata/expected-plans/q17.txt +++ b/testdata/expected-plans/q17.txt @@ -49,51 +49,52 @@ ProjectionExec: expr=[CAST(SUM(lineitem.l_extendedprice)@0 AS Float64) / 7 as av RaySQL Plan =========== -Query Stage #0: +Query Stage #0 (4 -> 4): ShuffleWriterExec(stage_id=0, output_partitioning=Hash([Column { name: "l_partkey", index: 0 }], 4)) ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-3.parquet]]}, projection=[l_partkey, l_quantity, l_extendedprice] -Query Stage #1: +Query Stage #1 (4 -> 4): ShuffleWriterExec(stage_id=1, output_partitioning=Hash([Column { name: "p_partkey", index: 0 }], 4)) CoalesceBatchesExec: target_batch_size=8192 FilterExec: p_brand@1 = Brand#42 AND p_container@2 = LG BAG ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/part.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/part.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/part.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/part.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-3.parquet]]}, predicate=p_brand = Utf8("Brand#42") AND p_container = Utf8("LG BAG"), pruning_predicate=p_brand_min@0 <= Brand#42 AND Brand#42 <= p_brand_max@1 AND p_container_min@2 <= LG BAG AND LG BAG <= p_container_max@3, projection=[p_partkey, p_brand, p_container] -Query Stage #2: +Query Stage #2 (4 -> 4): ShuffleWriterExec(stage_id=2, output_partitioning=Hash([Column { name: "p_partkey", index: 3 }, Column { name: "l_partkey", index: 0 }], 4)) CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "l_partkey", index: 0 }, Column { name: "p_partkey", index: 0 })] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=0, input_partitions=4) + ShuffleReaderExec(stage_id=0, input_partitioning=Hash([Column { name: "l_partkey", index: 0 }], 4)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=1, input_partitions=4) + ShuffleReaderExec(stage_id=1, input_partitioning=Hash([Column { name: "p_partkey", index: 0 }], 4)) -Query Stage #3: +Query Stage #3 (4 -> 4): ShuffleWriterExec(stage_id=3, output_partitioning=Hash([Column { name: "l_partkey", index: 0 }], 4)) AggregateExec: mode=Partial, gby=[l_partkey@0 as l_partkey], aggr=[AVG(lineitem.l_quantity)] ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-3.parquet]]}, projection=[l_partkey, l_quantity] -Query Stage #4: +Query Stage #4 (4 -> 4): ShuffleWriterExec(stage_id=4, output_partitioning=Hash([Column { name: "l_partkey", index: 0 }, Column { name: "l_partkey", index: 0 }], 4)) ProjectionExec: expr=[l_partkey@0 as l_partkey, 0.2 * CAST(AVG(lineitem.l_quantity)@1 AS Float64) as __value] AggregateExec: mode=FinalPartitioned, gby=[l_partkey@0 as l_partkey], aggr=[AVG(lineitem.l_quantity)] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=3, input_partitions=4) + ShuffleReaderExec(stage_id=3, input_partitioning=Hash([Column { name: "l_partkey", index: 0 }], 4)) -Query Stage #5: -ShuffleWriterExec(stage_id=5, output_partitioning=UnknownPartitioning(1)) +Query Stage #5 (4 -> 4): +ShuffleWriterExec(stage_id=5, output_partitioning=Hash([], 4)) AggregateExec: mode=Partial, gby=[], aggr=[SUM(lineitem.l_extendedprice)] CoalesceBatchesExec: target_batch_size=8192 FilterExec: CAST(l_quantity@1 AS Decimal128(30, 15)) < CAST(__value@7 AS Decimal128(30, 15)) CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "p_partkey", index: 3 }, Column { name: "l_partkey", index: 0 }), (Column { name: "l_partkey", index: 0 }, Column { name: "l_partkey", index: 0 })] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=2, input_partitions=4) + ShuffleReaderExec(stage_id=2, input_partitioning=Hash([Column { name: "p_partkey", index: 3 }, Column { name: "l_partkey", index: 0 }], 4)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=4, input_partitions=4) + ShuffleReaderExec(stage_id=4, input_partitioning=Hash([Column { name: "l_partkey", index: 0 }, Column { name: "l_partkey", index: 0 }], 4)) -Query Stage #6: +Query Stage #6 (1 -> 1): ProjectionExec: expr=[CAST(SUM(lineitem.l_extendedprice)@0 AS Float64) / 7 as avg_yearly] AggregateExec: mode=Final, gby=[], aggr=[SUM(lineitem.l_extendedprice)] - ShuffleReaderExec(stage_id=5, input_partitions=1) + CoalescePartitionsExec + ShuffleReaderExec(stage_id=5, input_partitioning=Hash([], 4)) diff --git a/testdata/expected-plans/q18.txt b/testdata/expected-plans/q18.txt index abe4b14..129ce60 100644 --- a/testdata/expected-plans/q18.txt +++ b/testdata/expected-plans/q18.txt @@ -57,33 +57,33 @@ GlobalLimitExec: skip=0, fetch=100 RaySQL Plan =========== -Query Stage #0: +Query Stage #0 (4 -> 4): ShuffleWriterExec(stage_id=0, output_partitioning=Hash([Column { name: "c_custkey", index: 0 }], 4)) ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-3.parquet]]}, projection=[c_custkey, c_name] -Query Stage #1: +Query Stage #1 (4 -> 4): ShuffleWriterExec(stage_id=1, output_partitioning=Hash([Column { name: "o_custkey", index: 1 }], 4)) ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-3.parquet]]}, projection=[o_orderkey, o_custkey, o_totalprice, o_orderdate] -Query Stage #2: +Query Stage #2 (4 -> 4): ShuffleWriterExec(stage_id=2, output_partitioning=Hash([Column { name: "o_orderkey", index: 2 }], 4)) CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "c_custkey", index: 0 }, Column { name: "o_custkey", index: 1 })] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=0, input_partitions=4) + ShuffleReaderExec(stage_id=0, input_partitioning=Hash([Column { name: "c_custkey", index: 0 }], 4)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=1, input_partitions=4) + ShuffleReaderExec(stage_id=1, input_partitioning=Hash([Column { name: "o_custkey", index: 1 }], 4)) -Query Stage #3: +Query Stage #3 (4 -> 4): ShuffleWriterExec(stage_id=3, output_partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 4)) ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-3.parquet]]}, projection=[l_orderkey, l_quantity] -Query Stage #4: +Query Stage #4 (4 -> 4): ShuffleWriterExec(stage_id=4, output_partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 4)) AggregateExec: mode=Partial, gby=[l_orderkey@0 as l_orderkey], aggr=[SUM(lineitem.l_quantity)] ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-3.parquet]]}, projection=[l_orderkey, l_quantity] -Query Stage #5: +Query Stage #5 (4 -> 4): ShuffleWriterExec(stage_id=5, output_partitioning=Hash([Column { name: "c_name", index: 0 }, Column { name: "c_custkey", index: 1 }, Column { name: "o_orderkey", index: 2 }, Column { name: "o_orderdate", index: 3 }, Column { name: "o_totalprice", index: 4 }], 4)) AggregateExec: mode=Partial, gby=[c_name@1 as c_name, c_custkey@0 as c_custkey, o_orderkey@2 as o_orderkey, o_orderdate@5 as o_orderdate, o_totalprice@4 as o_totalprice], aggr=[SUM(lineitem.l_quantity)] CoalesceBatchesExec: target_batch_size=8192 @@ -91,25 +91,26 @@ ShuffleWriterExec(stage_id=5, output_partitioning=Hash([Column { name: "c_name", CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "o_orderkey", index: 2 }, Column { name: "l_orderkey", index: 0 })] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=2, input_partitions=4) + ShuffleReaderExec(stage_id=2, input_partitioning=Hash([Column { name: "o_orderkey", index: 2 }], 4)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=3, input_partitions=4) + ShuffleReaderExec(stage_id=3, input_partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 4)) ProjectionExec: expr=[l_orderkey@0 as l_orderkey] CoalesceBatchesExec: target_batch_size=8192 FilterExec: SUM(lineitem.l_quantity)@1 > Some(31300),21,2 AggregateExec: mode=FinalPartitioned, gby=[l_orderkey@0 as l_orderkey], aggr=[SUM(lineitem.l_quantity)] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=4, input_partitions=4) + ShuffleReaderExec(stage_id=4, input_partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 4)) -Query Stage #6: -ShuffleWriterExec(stage_id=6, output_partitioning=UnknownPartitioning(1)) +Query Stage #6 (4 -> 4): +ShuffleWriterExec(stage_id=6, output_partitioning=Hash([Column { name: "c_name", index: 0 }, Column { name: "c_custkey", index: 1 }, Column { name: "o_orderkey", index: 2 }, Column { name: "o_orderdate", index: 3 }, Column { name: "o_totalprice", index: 4 }], 4)) SortExec: [o_totalprice@4 DESC,o_orderdate@3 ASC NULLS LAST] ProjectionExec: expr=[c_name@0 as c_name, c_custkey@1 as c_custkey, o_orderkey@2 as o_orderkey, o_orderdate@3 as o_orderdate, o_totalprice@4 as o_totalprice, SUM(lineitem.l_quantity)@5 as SUM(lineitem.l_quantity)] AggregateExec: mode=FinalPartitioned, gby=[c_name@0 as c_name, c_custkey@1 as c_custkey, o_orderkey@2 as o_orderkey, o_orderdate@3 as o_orderdate, o_totalprice@4 as o_totalprice], aggr=[SUM(lineitem.l_quantity)] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=5, input_partitions=4) + ShuffleReaderExec(stage_id=5, input_partitioning=Hash([Column { name: "c_name", index: 0 }, Column { name: "c_custkey", index: 1 }, Column { name: "o_orderkey", index: 2 }, Column { name: "o_orderdate", index: 3 }, Column { name: "o_totalprice", index: 4 }], 4)) -Query Stage #7: +Query Stage #7 (1 -> 1): GlobalLimitExec: skip=0, fetch=100 - ShuffleReaderExec(stage_id=6, input_partitions=1) + SortPreservingMergeExec: [o_totalprice@4 DESC,o_orderdate@3 ASC NULLS LAST] + ShuffleReaderExec(stage_id=6, input_partitioning=Hash([Column { name: "c_name", index: 0 }, Column { name: "c_custkey", index: 1 }, Column { name: "o_orderkey", index: 2 }, Column { name: "o_orderdate", index: 3 }, Column { name: "o_totalprice", index: 4 }], 4)) diff --git a/testdata/expected-plans/q19.txt b/testdata/expected-plans/q19.txt index 296d5ac..c294653 100644 --- a/testdata/expected-plans/q19.txt +++ b/testdata/expected-plans/q19.txt @@ -35,32 +35,33 @@ ProjectionExec: expr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_disco RaySQL Plan =========== -Query Stage #0: +Query Stage #0 (4 -> 4): ShuffleWriterExec(stage_id=0, output_partitioning=Hash([Column { name: "l_partkey", index: 0 }], 4)) CoalesceBatchesExec: target_batch_size=8192 FilterExec: l_quantity@1 >= Some(800),11,2 AND l_quantity@1 <= Some(1800),11,2 OR l_quantity@1 >= Some(2000),11,2 AND l_quantity@1 <= Some(3000),11,2 OR l_quantity@1 >= Some(3000),11,2 AND l_quantity@1 <= Some(4000),11,2 AND l_shipmode@5 = AIR REG OR l_shipmode@5 = AIR AND l_shipinstruct@4 = DELIVER IN PERSON ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-3.parquet]]}, predicate=(l_shipmode = Utf8("AIR REG") OR l_shipmode = Utf8("AIR")) AND l_shipinstruct = Utf8("DELIVER IN PERSON") AND (l_quantity >= Decimal128(Some(800),11,2) AND l_quantity <= Decimal128(Some(1800),11,2) OR l_quantity >= Decimal128(Some(2000),11,2) AND l_quantity <= Decimal128(Some(3000),11,2) OR l_quantity >= Decimal128(Some(3000),11,2) AND l_quantity <= Decimal128(Some(4000),11,2)), pruning_predicate=l_shipmode_min@0 <= AIR REG AND AIR REG <= l_shipmode_max@1 OR l_shipmode_min@0 <= AIR AND AIR <= l_shipmode_max@1 AND l_shipinstruct_min@2 <= DELIVER IN PERSON AND DELIVER IN PERSON <= l_shipinstruct_max@3 AND l_quantity_max@4 >= Some(800),11,2 AND l_quantity_min@5 <= Some(1800),11,2 OR l_quantity_max@4 >= Some(2000),11,2 AND l_quantity_min@5 <= Some(3000),11,2 OR l_quantity_max@4 >= Some(3000),11,2 AND l_quantity_min@5 <= Some(4000),11,2, projection=[l_partkey, l_quantity, l_extendedprice, l_discount, l_shipinstruct, l_shipmode] -Query Stage #1: +Query Stage #1 (4 -> 4): ShuffleWriterExec(stage_id=1, output_partitioning=Hash([Column { name: "p_partkey", index: 0 }], 4)) CoalesceBatchesExec: target_batch_size=8192 FilterExec: p_brand@1 = Brand#21 AND Use p_container@3 IN (SET) ([Literal { value: Utf8("SM CASE") }, Literal { value: Utf8("SM BOX") }, Literal { value: Utf8("SM PACK") }, Literal { value: Utf8("SM PKG") }]) AND p_size@2 <= 5 OR p_brand@1 = Brand#13 AND Use p_container@3 IN (SET) ([Literal { value: Utf8("MED BAG") }, Literal { value: Utf8("MED BOX") }, Literal { value: Utf8("MED PKG") }, Literal { value: Utf8("MED PACK") }]) AND p_size@2 <= 10 OR p_brand@1 = Brand#52 AND Use p_container@3 IN (SET) ([Literal { value: Utf8("LG CASE") }, Literal { value: Utf8("LG BOX") }, Literal { value: Utf8("LG PACK") }, Literal { value: Utf8("LG PKG") }]) AND p_size@2 <= 15 AND p_size@2 >= 1 ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/part.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/part.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/part.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/part.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-3.parquet]]}, predicate=p_size >= Int32(1) AND (p_brand = Utf8("Brand#21") AND p_container IN ([Utf8("SM CASE"), Utf8("SM BOX"), Utf8("SM PACK"), Utf8("SM PKG")]) AND p_size <= Int32(5) OR p_brand = Utf8("Brand#13") AND p_container IN ([Utf8("MED BAG"), Utf8("MED BOX"), Utf8("MED PKG"), Utf8("MED PACK")]) AND p_size <= Int32(10) OR p_brand = Utf8("Brand#52") AND p_container IN ([Utf8("LG CASE"), Utf8("LG BOX"), Utf8("LG PACK"), Utf8("LG PKG")]) AND p_size <= Int32(15)), pruning_predicate=p_size_max@0 >= 1 AND p_brand_min@1 <= Brand#21 AND Brand#21 <= p_brand_max@2 AND p_container_min@3 <= SM CASE AND SM CASE <= p_container_max@4 OR p_container_min@3 <= SM BOX AND SM BOX <= p_container_max@4 OR p_container_min@3 <= SM PACK AND SM PACK <= p_container_max@4 OR p_container_min@3 <= SM PKG AND SM PKG <= p_container_max@4 AND p_size_min@5 <= 5 OR p_brand_min@1 <= Brand#13 AND Brand#13 <= p_brand_max@2 AND p_container_min@3 <= MED BAG AND MED BAG <= p_container_max@4 OR p_container_min@3 <= MED BOX AND MED BOX <= p_container_max@4 OR p_container_min@3 <= MED PKG AND MED PKG <= p_container_max@4 OR p_container_min@3 <= MED PACK AND MED PACK <= p_container_max@4 AND p_size_min@5 <= 10 OR p_brand_min@1 <= Brand#52 AND Brand#52 <= p_brand_max@2 AND p_container_min@3 <= LG CASE AND LG CASE <= p_container_max@4 OR p_container_min@3 <= LG BOX AND LG BOX <= p_container_max@4 OR p_container_min@3 <= LG PACK AND LG PACK <= p_container_max@4 OR p_container_min@3 <= LG PKG AND LG PKG <= p_container_max@4 AND p_size_min@5 <= 15, projection=[p_partkey, p_brand, p_size, p_container] -Query Stage #2: -ShuffleWriterExec(stage_id=2, output_partitioning=UnknownPartitioning(1)) +Query Stage #2 (4 -> 4): +ShuffleWriterExec(stage_id=2, output_partitioning=Hash([], 4)) AggregateExec: mode=Partial, gby=[], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] CoalesceBatchesExec: target_batch_size=8192 FilterExec: p_brand@7 = Brand#21 AND Use p_container@9 IN (SET) ([Literal { value: Utf8("SM CASE") }, Literal { value: Utf8("SM BOX") }, Literal { value: Utf8("SM PACK") }, Literal { value: Utf8("SM PKG") }]) AND l_quantity@1 >= Some(800),11,2 AND l_quantity@1 <= Some(1800),11,2 AND p_size@8 <= 5 OR p_brand@7 = Brand#13 AND Use p_container@9 IN (SET) ([Literal { value: Utf8("MED BAG") }, Literal { value: Utf8("MED BOX") }, Literal { value: Utf8("MED PKG") }, Literal { value: Utf8("MED PACK") }]) AND l_quantity@1 >= Some(2000),11,2 AND l_quantity@1 <= Some(3000),11,2 AND p_size@8 <= 10 OR p_brand@7 = Brand#52 AND Use p_container@9 IN (SET) ([Literal { value: Utf8("LG CASE") }, Literal { value: Utf8("LG BOX") }, Literal { value: Utf8("LG PACK") }, Literal { value: Utf8("LG PKG") }]) AND l_quantity@1 >= Some(3000),11,2 AND l_quantity@1 <= Some(4000),11,2 AND p_size@8 <= 15 CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "l_partkey", index: 0 }, Column { name: "p_partkey", index: 0 })] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=0, input_partitions=4) + ShuffleReaderExec(stage_id=0, input_partitioning=Hash([Column { name: "l_partkey", index: 0 }], 4)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=1, input_partitions=4) + ShuffleReaderExec(stage_id=1, input_partitioning=Hash([Column { name: "p_partkey", index: 0 }], 4)) -Query Stage #3: +Query Stage #3 (1 -> 1): ProjectionExec: expr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@0 as revenue] AggregateExec: mode=Final, gby=[], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] - ShuffleReaderExec(stage_id=2, input_partitions=1) + CoalescePartitionsExec + ShuffleReaderExec(stage_id=2, input_partitioning=Hash([], 4)) diff --git a/testdata/expected-plans/q2.txt b/testdata/expected-plans/q2.txt index 524ab8e..5af73bd 100644 --- a/testdata/expected-plans/q2.txt +++ b/testdata/expected-plans/q2.txt @@ -113,134 +113,135 @@ GlobalLimitExec: skip=0, fetch=100 RaySQL Plan =========== -Query Stage #0: +Query Stage #0 (4 -> 4): ShuffleWriterExec(stage_id=0, output_partitioning=Hash([Column { name: "p_partkey", index: 0 }], 4)) CoalesceBatchesExec: target_batch_size=8192 FilterExec: p_size@3 = 48 AND p_type@2 LIKE %TIN ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/part.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/part.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/part.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/part.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-3.parquet]]}, predicate=p_size = Int32(48) AND p_type LIKE Utf8("%TIN"), pruning_predicate=p_size_min@0 <= 48 AND 48 <= p_size_max@1, projection=[p_partkey, p_mfgr, p_type, p_size] -Query Stage #1: +Query Stage #1 (4 -> 4): ShuffleWriterExec(stage_id=1, output_partitioning=Hash([Column { name: "ps_partkey", index: 0 }], 4)) ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-3.parquet]]}, projection=[ps_partkey, ps_suppkey, ps_supplycost] -Query Stage #2: +Query Stage #2 (4 -> 4): ShuffleWriterExec(stage_id=2, output_partitioning=Hash([Column { name: "ps_suppkey", index: 5 }], 4)) CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "p_partkey", index: 0 }, Column { name: "ps_partkey", index: 0 })] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=0, input_partitions=4) + ShuffleReaderExec(stage_id=0, input_partitioning=Hash([Column { name: "p_partkey", index: 0 }], 4)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=1, input_partitions=4) + ShuffleReaderExec(stage_id=1, input_partitioning=Hash([Column { name: "ps_partkey", index: 0 }], 4)) -Query Stage #3: +Query Stage #3 (4 -> 4): ShuffleWriterExec(stage_id=3, output_partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 4)) ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-3.parquet]]}, projection=[s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment] -Query Stage #4: +Query Stage #4 (4 -> 4): ShuffleWriterExec(stage_id=4, output_partitioning=Hash([Column { name: "s_nationkey", index: 10 }], 4)) CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "ps_suppkey", index: 5 }, Column { name: "s_suppkey", index: 0 })] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=2, input_partitions=4) + ShuffleReaderExec(stage_id=2, input_partitioning=Hash([Column { name: "ps_suppkey", index: 5 }], 4)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=3, input_partitions=4) + ShuffleReaderExec(stage_id=3, input_partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 4)) -Query Stage #5: +Query Stage #5 (1 -> 4): ShuffleWriterExec(stage_id=5, output_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 4)) ParquetExec: limit=None, partitions={1 group: [[mnt/bigdata/tpch/sf10-parquet/nation.parquet/part-0.parquet]]}, projection=[n_nationkey, n_name, n_regionkey] -Query Stage #6: +Query Stage #6 (4 -> 4): ShuffleWriterExec(stage_id=6, output_partitioning=Hash([Column { name: "n_regionkey", index: 16 }], 4)) CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "s_nationkey", index: 10 }, Column { name: "n_nationkey", index: 0 })] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=4, input_partitions=4) + ShuffleReaderExec(stage_id=4, input_partitioning=Hash([Column { name: "s_nationkey", index: 10 }], 4)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=5, input_partitions=4) + ShuffleReaderExec(stage_id=5, input_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 4)) -Query Stage #7: +Query Stage #7 (1 -> 4): ShuffleWriterExec(stage_id=7, output_partitioning=Hash([Column { name: "r_regionkey", index: 0 }], 4)) CoalesceBatchesExec: target_batch_size=8192 FilterExec: r_name@1 = ASIA ParquetExec: limit=None, partitions={1 group: [[mnt/bigdata/tpch/sf10-parquet/region.parquet/part-0.parquet]]}, predicate=r_name = Utf8("ASIA"), pruning_predicate=r_name_min@0 <= ASIA AND ASIA <= r_name_max@1, projection=[r_regionkey, r_name] -Query Stage #8: +Query Stage #8 (4 -> 4): ShuffleWriterExec(stage_id=8, output_partitioning=Hash([Column { name: "p_partkey", index: 0 }, Column { name: "ps_supplycost", index: 6 }], 4)) CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "n_regionkey", index: 16 }, Column { name: "r_regionkey", index: 0 })] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=6, input_partitions=4) + ShuffleReaderExec(stage_id=6, input_partitioning=Hash([Column { name: "n_regionkey", index: 16 }], 4)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=7, input_partitions=4) + ShuffleReaderExec(stage_id=7, input_partitioning=Hash([Column { name: "r_regionkey", index: 0 }], 4)) -Query Stage #9: +Query Stage #9 (1 -> 4): ShuffleWriterExec(stage_id=9, output_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 4)) ParquetExec: limit=None, partitions={1 group: [[mnt/bigdata/tpch/sf10-parquet/nation.parquet/part-0.parquet]]}, projection=[n_nationkey, n_regionkey] -Query Stage #10: +Query Stage #10 (4 -> 4): ShuffleWriterExec(stage_id=10, output_partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 4)) ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-3.parquet]]}, projection=[s_suppkey, s_nationkey] -Query Stage #11: +Query Stage #11 (4 -> 4): ShuffleWriterExec(stage_id=11, output_partitioning=Hash([Column { name: "ps_suppkey", index: 1 }], 4)) ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-3.parquet]]}, projection=[ps_partkey, ps_suppkey, ps_supplycost] -Query Stage #12: +Query Stage #12 (4 -> 4): ShuffleWriterExec(stage_id=12, output_partitioning=Hash([Column { name: "s_nationkey", index: 4 }], 4)) ProjectionExec: expr=[ps_partkey@2 as ps_partkey, ps_suppkey@3 as ps_suppkey, ps_supplycost@4 as ps_supplycost, s_suppkey@0 as s_suppkey, s_nationkey@1 as s_nationkey] CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "s_suppkey", index: 0 }, Column { name: "ps_suppkey", index: 1 })] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=10, input_partitions=4) + ShuffleReaderExec(stage_id=10, input_partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 4)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=11, input_partitions=4) + ShuffleReaderExec(stage_id=11, input_partitioning=Hash([Column { name: "ps_suppkey", index: 1 }], 4)) -Query Stage #13: +Query Stage #13 (4 -> 4): ShuffleWriterExec(stage_id=13, output_partitioning=Hash([Column { name: "n_regionkey", index: 6 }], 4)) ProjectionExec: expr=[ps_partkey@2 as ps_partkey, ps_suppkey@3 as ps_suppkey, ps_supplycost@4 as ps_supplycost, s_suppkey@5 as s_suppkey, s_nationkey@6 as s_nationkey, n_nationkey@0 as n_nationkey, n_regionkey@1 as n_regionkey] CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "n_nationkey", index: 0 }, Column { name: "s_nationkey", index: 4 })] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=9, input_partitions=4) + ShuffleReaderExec(stage_id=9, input_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 4)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=12, input_partitions=4) + ShuffleReaderExec(stage_id=12, input_partitioning=Hash([Column { name: "s_nationkey", index: 4 }], 4)) -Query Stage #14: +Query Stage #14 (1 -> 4): ShuffleWriterExec(stage_id=14, output_partitioning=Hash([Column { name: "r_regionkey", index: 0 }], 4)) CoalesceBatchesExec: target_batch_size=8192 FilterExec: r_name@1 = ASIA ParquetExec: limit=None, partitions={1 group: [[mnt/bigdata/tpch/sf10-parquet/region.parquet/part-0.parquet]]}, predicate=r_name = Utf8("ASIA"), pruning_predicate=r_name_min@0 <= ASIA AND ASIA <= r_name_max@1, projection=[r_regionkey, r_name] -Query Stage #15: +Query Stage #15 (4 -> 4): ShuffleWriterExec(stage_id=15, output_partitioning=Hash([Column { name: "ps_partkey", index: 0 }], 4)) AggregateExec: mode=Partial, gby=[ps_partkey@0 as ps_partkey], aggr=[MIN(partsupp.ps_supplycost)] CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "n_regionkey", index: 6 }, Column { name: "r_regionkey", index: 0 })] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=13, input_partitions=4) + ShuffleReaderExec(stage_id=13, input_partitioning=Hash([Column { name: "n_regionkey", index: 6 }], 4)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=14, input_partitions=4) + ShuffleReaderExec(stage_id=14, input_partitioning=Hash([Column { name: "r_regionkey", index: 0 }], 4)) -Query Stage #16: +Query Stage #16 (4 -> 4): ShuffleWriterExec(stage_id=16, output_partitioning=Hash([Column { name: "ps_partkey", index: 0 }, Column { name: "__value", index: 1 }], 4)) ProjectionExec: expr=[ps_partkey@0 as ps_partkey, MIN(partsupp.ps_supplycost)@1 as __value] AggregateExec: mode=FinalPartitioned, gby=[ps_partkey@0 as ps_partkey], aggr=[MIN(partsupp.ps_supplycost)] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=15, input_partitions=4) + ShuffleReaderExec(stage_id=15, input_partitioning=Hash([Column { name: "ps_partkey", index: 0 }], 4)) -Query Stage #17: -ShuffleWriterExec(stage_id=17, output_partitioning=UnknownPartitioning(1)) +Query Stage #17 (4 -> 4): +ShuffleWriterExec(stage_id=17, output_partitioning=Hash([Column { name: "p_partkey", index: 3 }], 4)) SortExec: [s_acctbal@0 DESC,n_name@2 ASC NULLS LAST,s_name@1 ASC NULLS LAST,p_partkey@3 ASC NULLS LAST] ProjectionExec: expr=[s_acctbal@5 as s_acctbal, s_name@2 as s_name, n_name@7 as n_name, p_partkey@0 as p_partkey, p_mfgr@1 as p_mfgr, s_address@3 as s_address, s_phone@4 as s_phone, s_comment@6 as s_comment] ProjectionExec: expr=[p_partkey@0 as p_partkey, p_mfgr@1 as p_mfgr, s_name@8 as s_name, s_address@9 as s_address, s_phone@11 as s_phone, s_acctbal@12 as s_acctbal, s_comment@13 as s_comment, n_name@15 as n_name] CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "p_partkey", index: 0 }, Column { name: "ps_partkey", index: 0 }), (Column { name: "ps_supplycost", index: 6 }, Column { name: "__value", index: 1 })] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=8, input_partitions=4) + ShuffleReaderExec(stage_id=8, input_partitioning=Hash([Column { name: "p_partkey", index: 0 }, Column { name: "ps_supplycost", index: 6 }], 4)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=16, input_partitions=4) + ShuffleReaderExec(stage_id=16, input_partitioning=Hash([Column { name: "ps_partkey", index: 0 }, Column { name: "__value", index: 1 }], 4)) -Query Stage #18: +Query Stage #18 (1 -> 1): GlobalLimitExec: skip=0, fetch=100 - ShuffleReaderExec(stage_id=17, input_partitions=1) + SortPreservingMergeExec: [s_acctbal@0 DESC,n_name@2 ASC NULLS LAST,s_name@1 ASC NULLS LAST,p_partkey@3 ASC NULLS LAST] + ShuffleReaderExec(stage_id=17, input_partitioning=Hash([Column { name: "p_partkey", index: 3 }], 4)) diff --git a/testdata/expected-plans/q20.txt b/testdata/expected-plans/q20.txt index 5db5f5c..a93076b 100644 --- a/testdata/expected-plans/q20.txt +++ b/testdata/expected-plans/q20.txt @@ -77,53 +77,53 @@ SortExec: [s_name@0 ASC NULLS LAST] RaySQL Plan =========== -Query Stage #0: +Query Stage #0 (4 -> 4): ShuffleWriterExec(stage_id=0, output_partitioning=Hash([Column { name: "s_nationkey", index: 3 }], 4)) ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-3.parquet]]}, projection=[s_suppkey, s_name, s_address, s_nationkey] -Query Stage #1: +Query Stage #1 (1 -> 4): ShuffleWriterExec(stage_id=1, output_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 4)) CoalesceBatchesExec: target_batch_size=8192 FilterExec: n_name@1 = KENYA ParquetExec: limit=None, partitions={1 group: [[mnt/bigdata/tpch/sf10-parquet/nation.parquet/part-0.parquet]]}, predicate=n_name = Utf8("KENYA"), pruning_predicate=n_name_min@0 <= KENYA AND KENYA <= n_name_max@1, projection=[n_nationkey, n_name] -Query Stage #2: +Query Stage #2 (4 -> 4): ShuffleWriterExec(stage_id=2, output_partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 4)) CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "s_nationkey", index: 3 }, Column { name: "n_nationkey", index: 0 })] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=0, input_partitions=4) + ShuffleReaderExec(stage_id=0, input_partitioning=Hash([Column { name: "s_nationkey", index: 3 }], 4)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=1, input_partitions=4) + ShuffleReaderExec(stage_id=1, input_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 4)) -Query Stage #3: +Query Stage #3 (4 -> 4): ShuffleWriterExec(stage_id=3, output_partitioning=Hash([Column { name: "ps_partkey", index: 0 }], 4)) ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-3.parquet]]}, projection=[ps_partkey, ps_suppkey, ps_availqty] -Query Stage #4: +Query Stage #4 (4 -> 4): ShuffleWriterExec(stage_id=4, output_partitioning=Hash([Column { name: "p_partkey", index: 0 }], 4)) ProjectionExec: expr=[p_partkey@0 as p_partkey] CoalesceBatchesExec: target_batch_size=8192 FilterExec: p_name@1 LIKE blanched% ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/part.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/part.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/part.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/part.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-3.parquet]]}, predicate=p_name LIKE Utf8("blanched%"), projection=[p_partkey, p_name] -Query Stage #5: +Query Stage #5 (4 -> 4): ShuffleWriterExec(stage_id=5, output_partitioning=Hash([Column { name: "ps_partkey", index: 0 }, Column { name: "ps_suppkey", index: 1 }], 4)) CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(Column { name: "ps_partkey", index: 0 }, Column { name: "p_partkey", index: 0 })] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=3, input_partitions=4) + ShuffleReaderExec(stage_id=3, input_partitioning=Hash([Column { name: "ps_partkey", index: 0 }], 4)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=4, input_partitions=4) + ShuffleReaderExec(stage_id=4, input_partitioning=Hash([Column { name: "p_partkey", index: 0 }], 4)) -Query Stage #6: +Query Stage #6 (4 -> 4): ShuffleWriterExec(stage_id=6, output_partitioning=Hash([Column { name: "l_partkey", index: 0 }, Column { name: "l_suppkey", index: 1 }], 4)) AggregateExec: mode=Partial, gby=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey], aggr=[SUM(lineitem.l_quantity)] CoalesceBatchesExec: target_batch_size=8192 FilterExec: l_shipdate@3 >= 8401 AND l_shipdate@3 < 8766 ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-3.parquet]]}, predicate=l_shipdate >= Date32("8401") AND l_shipdate < Date32("8766"), pruning_predicate=l_shipdate_max@0 >= 8401 AND l_shipdate_min@1 < 8766, projection=[l_partkey, l_suppkey, l_quantity, l_shipdate] -Query Stage #7: +Query Stage #7 (4 -> 4): ShuffleWriterExec(stage_id=7, output_partitioning=Hash([Column { name: "ps_suppkey", index: 0 }], 4)) ProjectionExec: expr=[ps_suppkey@1 as ps_suppkey] CoalesceBatchesExec: target_batch_size=8192 @@ -131,23 +131,24 @@ ShuffleWriterExec(stage_id=7, output_partitioning=Hash([Column { name: "ps_suppk CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "ps_partkey", index: 0 }, Column { name: "l_partkey", index: 0 }), (Column { name: "ps_suppkey", index: 1 }, Column { name: "l_suppkey", index: 1 })] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=5, input_partitions=4) + ShuffleReaderExec(stage_id=5, input_partitioning=Hash([Column { name: "ps_partkey", index: 0 }, Column { name: "ps_suppkey", index: 1 }], 4)) ProjectionExec: expr=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey, 0.5 * CAST(SUM(lineitem.l_quantity)@2 AS Float64) as __value] AggregateExec: mode=FinalPartitioned, gby=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey], aggr=[SUM(lineitem.l_quantity)] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=6, input_partitions=4) + ShuffleReaderExec(stage_id=6, input_partitioning=Hash([Column { name: "l_partkey", index: 0 }, Column { name: "l_suppkey", index: 1 }], 4)) -Query Stage #8: -ShuffleWriterExec(stage_id=8, output_partitioning=UnknownPartitioning(1)) +Query Stage #8 (4 -> 4): +ShuffleWriterExec(stage_id=8, output_partitioning=Hash([], 4)) ProjectionExec: expr=[s_name@1 as s_name, s_address@2 as s_address] CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(Column { name: "s_suppkey", index: 0 }, Column { name: "ps_suppkey", index: 0 })] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=2, input_partitions=4) + ShuffleReaderExec(stage_id=2, input_partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 4)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=7, input_partitions=4) + ShuffleReaderExec(stage_id=7, input_partitioning=Hash([Column { name: "ps_suppkey", index: 0 }], 4)) -Query Stage #9: +Query Stage #9 (1 -> 1): SortExec: [s_name@0 ASC NULLS LAST] - ShuffleReaderExec(stage_id=8, input_partitions=1) + CoalescePartitionsExec + ShuffleReaderExec(stage_id=8, input_partitioning=Hash([], 4)) diff --git a/testdata/expected-plans/q21.txt b/testdata/expected-plans/q21.txt index 9e36858..c19f7a2 100644 --- a/testdata/expected-plans/q21.txt +++ b/testdata/expected-plans/q21.txt @@ -82,66 +82,66 @@ GlobalLimitExec: skip=0, fetch=100 RaySQL Plan =========== -Query Stage #0: +Query Stage #0 (4 -> 4): ShuffleWriterExec(stage_id=0, output_partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 4)) ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-3.parquet]]}, projection=[s_suppkey, s_name, s_nationkey] -Query Stage #1: +Query Stage #1 (4 -> 4): ShuffleWriterExec(stage_id=1, output_partitioning=Hash([Column { name: "l_suppkey", index: 1 }], 4)) CoalesceBatchesExec: target_batch_size=8192 FilterExec: l_receiptdate@3 > l_commitdate@2 ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-3.parquet]]}, predicate=l_receiptdate > l_commitdate, projection=[l_orderkey, l_suppkey, l_commitdate, l_receiptdate] -Query Stage #2: +Query Stage #2 (4 -> 4): ShuffleWriterExec(stage_id=2, output_partitioning=Hash([Column { name: "l_orderkey", index: 3 }], 4)) CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "s_suppkey", index: 0 }, Column { name: "l_suppkey", index: 1 })] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=0, input_partitions=4) + ShuffleReaderExec(stage_id=0, input_partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 4)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=1, input_partitions=4) + ShuffleReaderExec(stage_id=1, input_partitioning=Hash([Column { name: "l_suppkey", index: 1 }], 4)) -Query Stage #3: +Query Stage #3 (4 -> 4): ShuffleWriterExec(stage_id=3, output_partitioning=Hash([Column { name: "o_orderkey", index: 0 }], 4)) CoalesceBatchesExec: target_batch_size=8192 FilterExec: o_orderstatus@1 = F ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-3.parquet]]}, predicate=o_orderstatus = Utf8("F"), pruning_predicate=o_orderstatus_min@0 <= F AND F <= o_orderstatus_max@1, projection=[o_orderkey, o_orderstatus] -Query Stage #4: +Query Stage #4 (4 -> 4): ShuffleWriterExec(stage_id=4, output_partitioning=Hash([Column { name: "s_nationkey", index: 2 }], 4)) CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "l_orderkey", index: 3 }, Column { name: "o_orderkey", index: 0 })] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=2, input_partitions=4) + ShuffleReaderExec(stage_id=2, input_partitioning=Hash([Column { name: "l_orderkey", index: 3 }], 4)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=3, input_partitions=4) + ShuffleReaderExec(stage_id=3, input_partitioning=Hash([Column { name: "o_orderkey", index: 0 }], 4)) -Query Stage #5: +Query Stage #5 (1 -> 4): ShuffleWriterExec(stage_id=5, output_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 4)) CoalesceBatchesExec: target_batch_size=8192 FilterExec: n_name@1 = ARGENTINA ParquetExec: limit=None, partitions={1 group: [[mnt/bigdata/tpch/sf10-parquet/nation.parquet/part-0.parquet]]}, predicate=n_name = Utf8("ARGENTINA"), pruning_predicate=n_name_min@0 <= ARGENTINA AND ARGENTINA <= n_name_max@1, projection=[n_nationkey, n_name] -Query Stage #6: +Query Stage #6 (4 -> 4): ShuffleWriterExec(stage_id=6, output_partitioning=Hash([Column { name: "l_orderkey", index: 3 }], 4)) CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "s_nationkey", index: 2 }, Column { name: "n_nationkey", index: 0 })] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=4, input_partitions=4) + ShuffleReaderExec(stage_id=4, input_partitioning=Hash([Column { name: "s_nationkey", index: 2 }], 4)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=5, input_partitions=4) + ShuffleReaderExec(stage_id=5, input_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 4)) -Query Stage #7: +Query Stage #7 (4 -> 4): ShuffleWriterExec(stage_id=7, output_partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 4)) ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-3.parquet]]}, projection=[l_orderkey, l_suppkey] -Query Stage #8: +Query Stage #8 (4 -> 4): ShuffleWriterExec(stage_id=8, output_partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 4)) CoalesceBatchesExec: target_batch_size=8192 FilterExec: l_receiptdate@3 > l_commitdate@2 ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-3.parquet]]}, predicate=l_receiptdate > l_commitdate, projection=[l_orderkey, l_suppkey, l_commitdate, l_receiptdate] -Query Stage #9: +Query Stage #9 (4 -> 4): ShuffleWriterExec(stage_id=9, output_partitioning=Hash([Column { name: "s_name", index: 0 }], 4)) AggregateExec: mode=Partial, gby=[s_name@1 as s_name], aggr=[COUNT(UInt8(1))] CoalesceBatchesExec: target_batch_size=8192 @@ -149,21 +149,22 @@ ShuffleWriterExec(stage_id=9, output_partitioning=Hash([Column { name: "s_name", CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(Column { name: "l_orderkey", index: 3 }, Column { name: "l_orderkey", index: 0 })], filter=BinaryExpr { left: Column { name: "l_suppkey", index: 1 }, op: NotEq, right: Column { name: "l_suppkey", index: 0 } } CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=6, input_partitions=4) + ShuffleReaderExec(stage_id=6, input_partitioning=Hash([Column { name: "l_orderkey", index: 3 }], 4)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=7, input_partitions=4) + ShuffleReaderExec(stage_id=7, input_partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 4)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=8, input_partitions=4) + ShuffleReaderExec(stage_id=8, input_partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 4)) -Query Stage #10: -ShuffleWriterExec(stage_id=10, output_partitioning=UnknownPartitioning(1)) +Query Stage #10 (4 -> 4): +ShuffleWriterExec(stage_id=10, output_partitioning=Hash([Column { name: "s_name", index: 0 }], 4)) SortExec: [numwait@1 DESC,s_name@0 ASC NULLS LAST] ProjectionExec: expr=[s_name@0 as s_name, COUNT(UInt8(1))@1 as numwait] AggregateExec: mode=FinalPartitioned, gby=[s_name@0 as s_name], aggr=[COUNT(UInt8(1))] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=9, input_partitions=4) + ShuffleReaderExec(stage_id=9, input_partitioning=Hash([Column { name: "s_name", index: 0 }], 4)) -Query Stage #11: +Query Stage #11 (1 -> 1): GlobalLimitExec: skip=0, fetch=100 - ShuffleReaderExec(stage_id=10, input_partitions=1) + SortPreservingMergeExec: [numwait@1 DESC,s_name@0 ASC NULLS LAST] + ShuffleReaderExec(stage_id=10, input_partitioning=Hash([Column { name: "s_name", index: 0 }], 4)) diff --git a/testdata/expected-plans/q22.txt b/testdata/expected-plans/q22.txt index 942fa53..f4787fe 100644 --- a/testdata/expected-plans/q22.txt +++ b/testdata/expected-plans/q22.txt @@ -54,52 +54,55 @@ SortExec: [cntrycode@0 ASC NULLS LAST] RaySQL Plan =========== -Query Stage #0: +Query Stage #0 (4 -> 4): ShuffleWriterExec(stage_id=0, output_partitioning=Hash([Column { name: "c_custkey", index: 0 }], 4)) CoalesceBatchesExec: target_batch_size=8192 FilterExec: Use substr(c_phone@1, 1, 2) IN (SET) ([Literal { value: Utf8("24") }, Literal { value: Utf8("34") }, Literal { value: Utf8("16") }, Literal { value: Utf8("30") }, Literal { value: Utf8("33") }, Literal { value: Utf8("14") }, Literal { value: Utf8("13") }]) ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-3.parquet]]}, predicate=substr(c_phone, Int64(1), Int64(2)) IN ([Utf8("24"), Utf8("34"), Utf8("16"), Utf8("30"), Utf8("33"), Utf8("14"), Utf8("13")]), projection=[c_custkey, c_phone, c_acctbal] -Query Stage #1: +Query Stage #1 (4 -> 4): ShuffleWriterExec(stage_id=1, output_partitioning=Hash([Column { name: "o_custkey", index: 0 }], 4)) ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-3.parquet]]}, projection=[o_custkey] -Query Stage #2: -ShuffleWriterExec(stage_id=2, output_partitioning=UnknownPartitioning(1)) +Query Stage #2 (4 -> 4): +ShuffleWriterExec(stage_id=2, output_partitioning=Hash([Column { name: "c_custkey", index: 0 }], 4)) CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(Column { name: "c_custkey", index: 0 }, Column { name: "o_custkey", index: 0 })] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=0, input_partitions=4) + ShuffleReaderExec(stage_id=0, input_partitioning=Hash([Column { name: "c_custkey", index: 0 }], 4)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=1, input_partitions=4) + ShuffleReaderExec(stage_id=1, input_partitioning=Hash([Column { name: "o_custkey", index: 0 }], 4)) -Query Stage #3: -ShuffleWriterExec(stage_id=3, output_partitioning=UnknownPartitioning(1)) +Query Stage #3 (4 -> 4): +ShuffleWriterExec(stage_id=3, output_partitioning=UnknownPartitioning(4)) AggregateExec: mode=Partial, gby=[], aggr=[AVG(customer.c_acctbal)] CoalesceBatchesExec: target_batch_size=8192 FilterExec: c_acctbal@1 > Some(0),11,2 AND Use substr(c_phone@0, 1, 2) IN (SET) ([Literal { value: Utf8("24") }, Literal { value: Utf8("34") }, Literal { value: Utf8("16") }, Literal { value: Utf8("30") }, Literal { value: Utf8("33") }, Literal { value: Utf8("14") }, Literal { value: Utf8("13") }]) ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-3.parquet]]}, predicate=c_acctbal > Decimal128(Some(0),11,2) AND substr(c_phone, Int64(1), Int64(2)) IN ([Utf8("24"), Utf8("34"), Utf8("16"), Utf8("30"), Utf8("33"), Utf8("14"), Utf8("13")]) AND c_acctbal > Decimal128(Some(0),11,2), pruning_predicate=c_acctbal_max@0 > Some(0),11,2 AND c_acctbal_max@0 > Some(0),11,2, projection=[c_phone, c_acctbal] -Query Stage #4: +Query Stage #4 (1 -> 4): ShuffleWriterExec(stage_id=4, output_partitioning=Hash([Column { name: "cntrycode", index: 0 }], 4)) AggregateExec: mode=Partial, gby=[cntrycode@0 as cntrycode], aggr=[COUNT(UInt8(1)), SUM(custsale.c_acctbal)] ProjectionExec: expr=[substr(c_phone@1, 1, 2) as cntrycode, c_acctbal@2 as c_acctbal] CoalesceBatchesExec: target_batch_size=8192 FilterExec: CAST(c_acctbal@2 AS Decimal128(15, 6)) > __value@3 CrossJoinExec - ShuffleReaderExec(stage_id=2, input_partitions=1) + CoalescePartitionsExec + ShuffleReaderExec(stage_id=2, input_partitioning=Hash([Column { name: "c_custkey", index: 0 }], 4)) ProjectionExec: expr=[AVG(customer.c_acctbal)@0 as __value] AggregateExec: mode=Final, gby=[], aggr=[AVG(customer.c_acctbal)] - ShuffleReaderExec(stage_id=3, input_partitions=1) + CoalescePartitionsExec + ShuffleReaderExec(stage_id=3, input_partitioning=UnknownPartitioning(4)) -Query Stage #5: -ShuffleWriterExec(stage_id=5, output_partitioning=UnknownPartitioning(1)) +Query Stage #5 (4 -> 4): +ShuffleWriterExec(stage_id=5, output_partitioning=Hash([Column { name: "cntrycode", index: 0 }], 4)) ProjectionExec: expr=[cntrycode@0 as cntrycode, COUNT(UInt8(1))@1 as numcust, SUM(custsale.c_acctbal)@2 as totacctbal] AggregateExec: mode=FinalPartitioned, gby=[cntrycode@0 as cntrycode], aggr=[COUNT(UInt8(1)), SUM(custsale.c_acctbal)] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=4, input_partitions=4) + ShuffleReaderExec(stage_id=4, input_partitioning=Hash([Column { name: "cntrycode", index: 0 }], 4)) -Query Stage #6: +Query Stage #6 (1 -> 1): SortExec: [cntrycode@0 ASC NULLS LAST] - ShuffleReaderExec(stage_id=5, input_partitions=1) + CoalescePartitionsExec + ShuffleReaderExec(stage_id=5, input_partitioning=Hash([Column { name: "cntrycode", index: 0 }], 4)) diff --git a/testdata/expected-plans/q3.txt b/testdata/expected-plans/q3.txt index f85ed5e..040c6f7 100644 --- a/testdata/expected-plans/q3.txt +++ b/testdata/expected-plans/q3.txt @@ -50,52 +50,53 @@ GlobalLimitExec: skip=0, fetch=10 RaySQL Plan =========== -Query Stage #0: +Query Stage #0 (4 -> 4): ShuffleWriterExec(stage_id=0, output_partitioning=Hash([Column { name: "c_custkey", index: 0 }], 4)) CoalesceBatchesExec: target_batch_size=8192 FilterExec: c_mktsegment@1 = BUILDING ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-3.parquet]]}, predicate=c_mktsegment = Utf8("BUILDING"), pruning_predicate=c_mktsegment_min@0 <= BUILDING AND BUILDING <= c_mktsegment_max@1, projection=[c_custkey, c_mktsegment] -Query Stage #1: +Query Stage #1 (4 -> 4): ShuffleWriterExec(stage_id=1, output_partitioning=Hash([Column { name: "o_custkey", index: 1 }], 4)) CoalesceBatchesExec: target_batch_size=8192 FilterExec: o_orderdate@2 < 9204 ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-3.parquet]]}, predicate=o_orderdate < Date32("9204"), pruning_predicate=o_orderdate_min@0 < 9204, projection=[o_orderkey, o_custkey, o_orderdate, o_shippriority] -Query Stage #2: +Query Stage #2 (4 -> 4): ShuffleWriterExec(stage_id=2, output_partitioning=Hash([Column { name: "o_orderkey", index: 2 }], 4)) CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "c_custkey", index: 0 }, Column { name: "o_custkey", index: 1 })] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=0, input_partitions=4) + ShuffleReaderExec(stage_id=0, input_partitioning=Hash([Column { name: "c_custkey", index: 0 }], 4)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=1, input_partitions=4) + ShuffleReaderExec(stage_id=1, input_partitioning=Hash([Column { name: "o_custkey", index: 1 }], 4)) -Query Stage #3: +Query Stage #3 (4 -> 4): ShuffleWriterExec(stage_id=3, output_partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 4)) CoalesceBatchesExec: target_batch_size=8192 FilterExec: l_shipdate@3 > 9204 ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-3.parquet]]}, predicate=l_shipdate > Date32("9204"), pruning_predicate=l_shipdate_max@0 > 9204, projection=[l_orderkey, l_extendedprice, l_discount, l_shipdate] -Query Stage #4: +Query Stage #4 (4 -> 4): ShuffleWriterExec(stage_id=4, output_partitioning=Hash([Column { name: "l_orderkey", index: 0 }, Column { name: "o_orderdate", index: 1 }, Column { name: "o_shippriority", index: 2 }], 4)) AggregateExec: mode=Partial, gby=[l_orderkey@6 as l_orderkey, o_orderdate@4 as o_orderdate, o_shippriority@5 as o_shippriority], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "o_orderkey", index: 2 }, Column { name: "l_orderkey", index: 0 })] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=2, input_partitions=4) + ShuffleReaderExec(stage_id=2, input_partitioning=Hash([Column { name: "o_orderkey", index: 2 }], 4)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=3, input_partitions=4) + ShuffleReaderExec(stage_id=3, input_partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 4)) -Query Stage #5: -ShuffleWriterExec(stage_id=5, output_partitioning=UnknownPartitioning(1)) +Query Stage #5 (4 -> 4): +ShuffleWriterExec(stage_id=5, output_partitioning=Hash([Column { name: "l_orderkey", index: 0 }, Column { name: "o_orderdate", index: 2 }, Column { name: "o_shippriority", index: 3 }], 4)) SortExec: [revenue@1 DESC,o_orderdate@2 ASC NULLS LAST] ProjectionExec: expr=[l_orderkey@0 as l_orderkey, SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@3 as revenue, o_orderdate@1 as o_orderdate, o_shippriority@2 as o_shippriority] AggregateExec: mode=FinalPartitioned, gby=[l_orderkey@0 as l_orderkey, o_orderdate@1 as o_orderdate, o_shippriority@2 as o_shippriority], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=4, input_partitions=4) + ShuffleReaderExec(stage_id=4, input_partitioning=Hash([Column { name: "l_orderkey", index: 0 }, Column { name: "o_orderdate", index: 1 }, Column { name: "o_shippriority", index: 2 }], 4)) -Query Stage #6: +Query Stage #6 (1 -> 1): GlobalLimitExec: skip=0, fetch=10 - ShuffleReaderExec(stage_id=5, input_partitions=1) + SortPreservingMergeExec: [revenue@1 DESC,o_orderdate@2 ASC NULLS LAST] + ShuffleReaderExec(stage_id=5, input_partitioning=Hash([Column { name: "l_orderkey", index: 0 }, Column { name: "o_orderdate", index: 2 }, Column { name: "o_shippriority", index: 3 }], 4)) diff --git a/testdata/expected-plans/q4.txt b/testdata/expected-plans/q4.txt index 5b22e28..edd8376 100644 --- a/testdata/expected-plans/q4.txt +++ b/testdata/expected-plans/q4.txt @@ -36,36 +36,37 @@ SortExec: [o_orderpriority@0 ASC NULLS LAST] RaySQL Plan =========== -Query Stage #0: +Query Stage #0 (4 -> 4): ShuffleWriterExec(stage_id=0, output_partitioning=Hash([Column { name: "o_orderkey", index: 0 }], 4)) CoalesceBatchesExec: target_batch_size=8192 FilterExec: o_orderdate@1 >= 9221 AND o_orderdate@1 < 9312 ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-3.parquet]]}, predicate=o_orderdate >= Date32("9221") AND o_orderdate < Date32("9312"), pruning_predicate=o_orderdate_max@0 >= 9221 AND o_orderdate_min@1 < 9312, projection=[o_orderkey, o_orderdate, o_orderpriority] -Query Stage #1: +Query Stage #1 (4 -> 4): ShuffleWriterExec(stage_id=1, output_partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 4)) CoalesceBatchesExec: target_batch_size=8192 FilterExec: l_commitdate@1 < l_receiptdate@2 ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-3.parquet]]}, predicate=l_commitdate < l_receiptdate, projection=[l_orderkey, l_commitdate, l_receiptdate] -Query Stage #2: +Query Stage #2 (4 -> 4): ShuffleWriterExec(stage_id=2, output_partitioning=Hash([Column { name: "o_orderpriority", index: 0 }], 4)) AggregateExec: mode=Partial, gby=[o_orderpriority@2 as o_orderpriority], aggr=[COUNT(UInt8(1))] CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(Column { name: "o_orderkey", index: 0 }, Column { name: "l_orderkey", index: 0 })] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=0, input_partitions=4) + ShuffleReaderExec(stage_id=0, input_partitioning=Hash([Column { name: "o_orderkey", index: 0 }], 4)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=1, input_partitions=4) + ShuffleReaderExec(stage_id=1, input_partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 4)) -Query Stage #3: -ShuffleWriterExec(stage_id=3, output_partitioning=UnknownPartitioning(1)) +Query Stage #3 (4 -> 4): +ShuffleWriterExec(stage_id=3, output_partitioning=Hash([Column { name: "o_orderpriority", index: 0 }], 4)) ProjectionExec: expr=[o_orderpriority@0 as o_orderpriority, COUNT(UInt8(1))@1 as order_count] AggregateExec: mode=FinalPartitioned, gby=[o_orderpriority@0 as o_orderpriority], aggr=[COUNT(UInt8(1))] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=2, input_partitions=4) + ShuffleReaderExec(stage_id=2, input_partitioning=Hash([Column { name: "o_orderpriority", index: 0 }], 4)) -Query Stage #4: +Query Stage #4 (1 -> 1): SortExec: [o_orderpriority@0 ASC NULLS LAST] - ShuffleReaderExec(stage_id=3, input_partitions=1) + CoalescePartitionsExec + ShuffleReaderExec(stage_id=3, input_partitioning=Hash([Column { name: "o_orderpriority", index: 0 }], 4)) diff --git a/testdata/expected-plans/q5.txt b/testdata/expected-plans/q5.txt index 2d814c6..d6aeedd 100644 --- a/testdata/expected-plans/q5.txt +++ b/testdata/expected-plans/q5.txt @@ -73,88 +73,89 @@ SortExec: [revenue@1 DESC] RaySQL Plan =========== -Query Stage #0: +Query Stage #0 (4 -> 4): ShuffleWriterExec(stage_id=0, output_partitioning=Hash([Column { name: "c_custkey", index: 0 }], 4)) ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-3.parquet]]}, projection=[c_custkey, c_nationkey] -Query Stage #1: +Query Stage #1 (4 -> 4): ShuffleWriterExec(stage_id=1, output_partitioning=Hash([Column { name: "o_custkey", index: 1 }], 4)) CoalesceBatchesExec: target_batch_size=8192 FilterExec: o_orderdate@2 >= 8766 AND o_orderdate@2 < 9131 ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-3.parquet]]}, predicate=o_orderdate >= Date32("8766") AND o_orderdate < Date32("9131"), pruning_predicate=o_orderdate_max@0 >= 8766 AND o_orderdate_min@1 < 9131, projection=[o_orderkey, o_custkey, o_orderdate] -Query Stage #2: +Query Stage #2 (4 -> 4): ShuffleWriterExec(stage_id=2, output_partitioning=Hash([Column { name: "o_orderkey", index: 2 }], 4)) CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "c_custkey", index: 0 }, Column { name: "o_custkey", index: 1 })] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=0, input_partitions=4) + ShuffleReaderExec(stage_id=0, input_partitioning=Hash([Column { name: "c_custkey", index: 0 }], 4)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=1, input_partitions=4) + ShuffleReaderExec(stage_id=1, input_partitioning=Hash([Column { name: "o_custkey", index: 1 }], 4)) -Query Stage #3: +Query Stage #3 (4 -> 4): ShuffleWriterExec(stage_id=3, output_partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 4)) ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-3.parquet]]}, projection=[l_orderkey, l_suppkey, l_extendedprice, l_discount] -Query Stage #4: +Query Stage #4 (4 -> 4): ShuffleWriterExec(stage_id=4, output_partitioning=Hash([Column { name: "l_suppkey", index: 6 }, Column { name: "c_nationkey", index: 1 }], 4)) CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "o_orderkey", index: 2 }, Column { name: "l_orderkey", index: 0 })] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=2, input_partitions=4) + ShuffleReaderExec(stage_id=2, input_partitioning=Hash([Column { name: "o_orderkey", index: 2 }], 4)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=3, input_partitions=4) + ShuffleReaderExec(stage_id=3, input_partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 4)) -Query Stage #5: +Query Stage #5 (4 -> 4): ShuffleWriterExec(stage_id=5, output_partitioning=Hash([Column { name: "s_suppkey", index: 0 }, Column { name: "s_nationkey", index: 1 }], 4)) ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-3.parquet]]}, projection=[s_suppkey, s_nationkey] -Query Stage #6: +Query Stage #6 (4 -> 4): ShuffleWriterExec(stage_id=6, output_partitioning=Hash([Column { name: "s_nationkey", index: 10 }], 4)) CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "l_suppkey", index: 6 }, Column { name: "s_suppkey", index: 0 }), (Column { name: "c_nationkey", index: 1 }, Column { name: "s_nationkey", index: 1 })] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=4, input_partitions=4) + ShuffleReaderExec(stage_id=4, input_partitioning=Hash([Column { name: "l_suppkey", index: 6 }, Column { name: "c_nationkey", index: 1 }], 4)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=5, input_partitions=4) + ShuffleReaderExec(stage_id=5, input_partitioning=Hash([Column { name: "s_suppkey", index: 0 }, Column { name: "s_nationkey", index: 1 }], 4)) -Query Stage #7: +Query Stage #7 (1 -> 4): ShuffleWriterExec(stage_id=7, output_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 4)) ParquetExec: limit=None, partitions={1 group: [[mnt/bigdata/tpch/sf10-parquet/nation.parquet/part-0.parquet]]}, projection=[n_nationkey, n_name, n_regionkey] -Query Stage #8: +Query Stage #8 (4 -> 4): ShuffleWriterExec(stage_id=8, output_partitioning=Hash([Column { name: "n_regionkey", index: 13 }], 4)) CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "s_nationkey", index: 10 }, Column { name: "n_nationkey", index: 0 })] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=6, input_partitions=4) + ShuffleReaderExec(stage_id=6, input_partitioning=Hash([Column { name: "s_nationkey", index: 10 }], 4)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=7, input_partitions=4) + ShuffleReaderExec(stage_id=7, input_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 4)) -Query Stage #9: +Query Stage #9 (1 -> 4): ShuffleWriterExec(stage_id=9, output_partitioning=Hash([Column { name: "r_regionkey", index: 0 }], 4)) CoalesceBatchesExec: target_batch_size=8192 FilterExec: r_name@1 = AFRICA ParquetExec: limit=None, partitions={1 group: [[mnt/bigdata/tpch/sf10-parquet/region.parquet/part-0.parquet]]}, predicate=r_name = Utf8("AFRICA"), pruning_predicate=r_name_min@0 <= AFRICA AND AFRICA <= r_name_max@1, projection=[r_regionkey, r_name] -Query Stage #10: +Query Stage #10 (4 -> 4): ShuffleWriterExec(stage_id=10, output_partitioning=Hash([Column { name: "n_name", index: 0 }], 4)) AggregateExec: mode=Partial, gby=[n_name@12 as n_name], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "n_regionkey", index: 13 }, Column { name: "r_regionkey", index: 0 })] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=8, input_partitions=4) + ShuffleReaderExec(stage_id=8, input_partitioning=Hash([Column { name: "n_regionkey", index: 13 }], 4)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=9, input_partitions=4) + ShuffleReaderExec(stage_id=9, input_partitioning=Hash([Column { name: "r_regionkey", index: 0 }], 4)) -Query Stage #11: -ShuffleWriterExec(stage_id=11, output_partitioning=UnknownPartitioning(1)) +Query Stage #11 (4 -> 4): +ShuffleWriterExec(stage_id=11, output_partitioning=Hash([Column { name: "n_name", index: 0 }], 4)) ProjectionExec: expr=[n_name@0 as n_name, SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 as revenue] AggregateExec: mode=FinalPartitioned, gby=[n_name@0 as n_name], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=10, input_partitions=4) + ShuffleReaderExec(stage_id=10, input_partitioning=Hash([Column { name: "n_name", index: 0 }], 4)) -Query Stage #12: +Query Stage #12 (1 -> 1): SortExec: [revenue@1 DESC] - ShuffleReaderExec(stage_id=11, input_partitions=1) + CoalescePartitionsExec + ShuffleReaderExec(stage_id=11, input_partitioning=Hash([Column { name: "n_name", index: 0 }], 4)) diff --git a/testdata/expected-plans/q6.txt b/testdata/expected-plans/q6.txt index 5ef8ae6..355399e 100644 --- a/testdata/expected-plans/q6.txt +++ b/testdata/expected-plans/q6.txt @@ -20,15 +20,16 @@ ProjectionExec: expr=[SUM(lineitem.l_extendedprice * lineitem.l_discount)@0 as r RaySQL Plan =========== -Query Stage #0: -ShuffleWriterExec(stage_id=0, output_partitioning=UnknownPartitioning(1)) +Query Stage #0 (4 -> 4): +ShuffleWriterExec(stage_id=0, output_partitioning=UnknownPartitioning(4)) AggregateExec: mode=Partial, gby=[], aggr=[SUM(lineitem.l_extendedprice * lineitem.l_discount)] CoalesceBatchesExec: target_batch_size=8192 FilterExec: l_shipdate@3 >= 8766 AND l_shipdate@3 < 9131 AND l_discount@2 >= Some(3),11,2 AND l_discount@2 <= Some(5),11,2 AND l_quantity@0 < Some(2400),11,2 ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-3.parquet]]}, predicate=l_shipdate >= Date32("8766") AND l_shipdate < Date32("9131") AND l_discount >= Decimal128(Some(3),11,2) AND l_discount <= Decimal128(Some(5),11,2) AND l_quantity < Decimal128(Some(2400),11,2), pruning_predicate=l_shipdate_max@0 >= 8766 AND l_shipdate_min@1 < 9131 AND l_discount_max@2 >= Some(3),11,2 AND l_discount_min@3 <= Some(5),11,2 AND l_quantity_min@4 < Some(2400),11,2, projection=[l_quantity, l_extendedprice, l_discount, l_shipdate] -Query Stage #1: +Query Stage #1 (1 -> 1): ProjectionExec: expr=[SUM(lineitem.l_extendedprice * lineitem.l_discount)@0 as revenue] AggregateExec: mode=Final, gby=[], aggr=[SUM(lineitem.l_extendedprice * lineitem.l_discount)] - ShuffleReaderExec(stage_id=0, input_partitions=1) + CoalescePartitionsExec + ShuffleReaderExec(stage_id=0, input_partitioning=UnknownPartitioning(4)) diff --git a/testdata/expected-plans/q7.txt b/testdata/expected-plans/q7.txt index 9e4449c..42419e6 100644 --- a/testdata/expected-plans/q7.txt +++ b/testdata/expected-plans/q7.txt @@ -85,73 +85,73 @@ SortExec: [supp_nation@0 ASC NULLS LAST,cust_nation@1 ASC NULLS LAST,l_year@2 AS RaySQL Plan =========== -Query Stage #0: +Query Stage #0 (4 -> 4): ShuffleWriterExec(stage_id=0, output_partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 4)) ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-3.parquet]]}, projection=[s_suppkey, s_nationkey] -Query Stage #1: +Query Stage #1 (4 -> 4): ShuffleWriterExec(stage_id=1, output_partitioning=Hash([Column { name: "l_suppkey", index: 1 }], 4)) CoalesceBatchesExec: target_batch_size=8192 FilterExec: l_shipdate@4 >= 9131 AND l_shipdate@4 <= 9861 ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-3.parquet]]}, predicate=l_shipdate >= Date32("9131") AND l_shipdate <= Date32("9861"), pruning_predicate=l_shipdate_max@0 >= 9131 AND l_shipdate_min@1 <= 9861, projection=[l_orderkey, l_suppkey, l_extendedprice, l_discount, l_shipdate] -Query Stage #2: +Query Stage #2 (4 -> 4): ShuffleWriterExec(stage_id=2, output_partitioning=Hash([Column { name: "l_orderkey", index: 2 }], 4)) CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "s_suppkey", index: 0 }, Column { name: "l_suppkey", index: 1 })] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=0, input_partitions=4) + ShuffleReaderExec(stage_id=0, input_partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 4)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=1, input_partitions=4) + ShuffleReaderExec(stage_id=1, input_partitioning=Hash([Column { name: "l_suppkey", index: 1 }], 4)) -Query Stage #3: +Query Stage #3 (4 -> 4): ShuffleWriterExec(stage_id=3, output_partitioning=Hash([Column { name: "o_orderkey", index: 0 }], 4)) ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-3.parquet]]}, projection=[o_orderkey, o_custkey] -Query Stage #4: +Query Stage #4 (4 -> 4): ShuffleWriterExec(stage_id=4, output_partitioning=Hash([Column { name: "o_custkey", index: 8 }], 4)) CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "l_orderkey", index: 2 }, Column { name: "o_orderkey", index: 0 })] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=2, input_partitions=4) + ShuffleReaderExec(stage_id=2, input_partitioning=Hash([Column { name: "l_orderkey", index: 2 }], 4)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=3, input_partitions=4) + ShuffleReaderExec(stage_id=3, input_partitioning=Hash([Column { name: "o_orderkey", index: 0 }], 4)) -Query Stage #5: +Query Stage #5 (4 -> 4): ShuffleWriterExec(stage_id=5, output_partitioning=Hash([Column { name: "c_custkey", index: 0 }], 4)) ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-3.parquet]]}, projection=[c_custkey, c_nationkey] -Query Stage #6: +Query Stage #6 (4 -> 4): ShuffleWriterExec(stage_id=6, output_partitioning=Hash([Column { name: "s_nationkey", index: 1 }], 4)) CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "o_custkey", index: 8 }, Column { name: "c_custkey", index: 0 })] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=4, input_partitions=4) + ShuffleReaderExec(stage_id=4, input_partitioning=Hash([Column { name: "o_custkey", index: 8 }], 4)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=5, input_partitions=4) + ShuffleReaderExec(stage_id=5, input_partitioning=Hash([Column { name: "c_custkey", index: 0 }], 4)) -Query Stage #7: +Query Stage #7 (1 -> 4): ShuffleWriterExec(stage_id=7, output_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 4)) CoalesceBatchesExec: target_batch_size=8192 FilterExec: n_name@1 = GERMANY OR n_name@1 = IRAQ ParquetExec: limit=None, partitions={1 group: [[mnt/bigdata/tpch/sf10-parquet/nation.parquet/part-0.parquet]]}, predicate=n_name = Utf8("GERMANY") OR n_name = Utf8("IRAQ"), pruning_predicate=n_name_min@0 <= GERMANY AND GERMANY <= n_name_max@1 OR n_name_min@0 <= IRAQ AND IRAQ <= n_name_max@1, projection=[n_nationkey, n_name] -Query Stage #8: +Query Stage #8 (4 -> 4): ShuffleWriterExec(stage_id=8, output_partitioning=Hash([Column { name: "c_nationkey", index: 10 }], 4)) CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "s_nationkey", index: 1 }, Column { name: "n_nationkey", index: 0 })] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=6, input_partitions=4) + ShuffleReaderExec(stage_id=6, input_partitioning=Hash([Column { name: "s_nationkey", index: 1 }], 4)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=7, input_partitions=4) + ShuffleReaderExec(stage_id=7, input_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 4)) -Query Stage #9: +Query Stage #9 (1 -> 4): ShuffleWriterExec(stage_id=9, output_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 4)) CoalesceBatchesExec: target_batch_size=8192 FilterExec: n_name@1 = IRAQ OR n_name@1 = GERMANY ParquetExec: limit=None, partitions={1 group: [[mnt/bigdata/tpch/sf10-parquet/nation.parquet/part-0.parquet]]}, predicate=n_name = Utf8("IRAQ") OR n_name = Utf8("GERMANY"), pruning_predicate=n_name_min@0 <= IRAQ AND IRAQ <= n_name_max@1 OR n_name_min@0 <= GERMANY AND GERMANY <= n_name_max@1, projection=[n_nationkey, n_name] -Query Stage #10: +Query Stage #10 (4 -> 4): ShuffleWriterExec(stage_id=10, output_partitioning=Hash([Column { name: "supp_nation", index: 0 }, Column { name: "cust_nation", index: 1 }, Column { name: "l_year", index: 2 }], 4)) AggregateExec: mode=Partial, gby=[supp_nation@0 as supp_nation, cust_nation@1 as cust_nation, l_year@2 as l_year], aggr=[SUM(shipping.volume)] ProjectionExec: expr=[n_name@12 as supp_nation, n_name@14 as cust_nation, datepart(YEAR, l_shipdate@6) as l_year, CAST(l_extendedprice@4 AS Decimal128(35, 4)) * CAST(Some(100),23,2 - CAST(l_discount@5 AS Decimal128(23, 2)) AS Decimal128(35, 4)) as volume] @@ -160,18 +160,19 @@ ShuffleWriterExec(stage_id=10, output_partitioning=Hash([Column { name: "supp_na CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "c_nationkey", index: 10 }, Column { name: "n_nationkey", index: 0 })] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=8, input_partitions=4) + ShuffleReaderExec(stage_id=8, input_partitioning=Hash([Column { name: "c_nationkey", index: 10 }], 4)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=9, input_partitions=4) + ShuffleReaderExec(stage_id=9, input_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 4)) -Query Stage #11: -ShuffleWriterExec(stage_id=11, output_partitioning=UnknownPartitioning(1)) +Query Stage #11 (4 -> 4): +ShuffleWriterExec(stage_id=11, output_partitioning=Hash([Column { name: "supp_nation", index: 0 }, Column { name: "cust_nation", index: 1 }, Column { name: "l_year", index: 2 }], 4)) ProjectionExec: expr=[supp_nation@0 as supp_nation, cust_nation@1 as cust_nation, l_year@2 as l_year, SUM(shipping.volume)@3 as revenue] AggregateExec: mode=FinalPartitioned, gby=[supp_nation@0 as supp_nation, cust_nation@1 as cust_nation, l_year@2 as l_year], aggr=[SUM(shipping.volume)] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=10, input_partitions=4) + ShuffleReaderExec(stage_id=10, input_partitioning=Hash([Column { name: "supp_nation", index: 0 }, Column { name: "cust_nation", index: 1 }, Column { name: "l_year", index: 2 }], 4)) -Query Stage #12: +Query Stage #12 (1 -> 1): SortExec: [supp_nation@0 ASC NULLS LAST,cust_nation@1 ASC NULLS LAST,l_year@2 ASC NULLS LAST] - ShuffleReaderExec(stage_id=11, input_partitions=1) + CoalescePartitionsExec + ShuffleReaderExec(stage_id=11, input_partitioning=Hash([Column { name: "supp_nation", index: 0 }, Column { name: "cust_nation", index: 1 }, Column { name: "l_year", index: 2 }], 4)) diff --git a/testdata/expected-plans/q8.txt b/testdata/expected-plans/q8.txt index ff3fe7c..e38534c 100644 --- a/testdata/expected-plans/q8.txt +++ b/testdata/expected-plans/q8.txt @@ -101,99 +101,99 @@ SortExec: [o_year@0 ASC NULLS LAST] RaySQL Plan =========== -Query Stage #0: +Query Stage #0 (4 -> 4): ShuffleWriterExec(stage_id=0, output_partitioning=Hash([Column { name: "p_partkey", index: 0 }], 4)) CoalesceBatchesExec: target_batch_size=8192 FilterExec: p_type@1 = LARGE PLATED STEEL ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/part.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/part.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/part.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/part.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-3.parquet]]}, predicate=p_type = Utf8("LARGE PLATED STEEL"), pruning_predicate=p_type_min@0 <= LARGE PLATED STEEL AND LARGE PLATED STEEL <= p_type_max@1, projection=[p_partkey, p_type] -Query Stage #1: +Query Stage #1 (4 -> 4): ShuffleWriterExec(stage_id=1, output_partitioning=Hash([Column { name: "l_partkey", index: 1 }], 4)) ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-3.parquet]]}, projection=[l_orderkey, l_partkey, l_suppkey, l_extendedprice, l_discount] -Query Stage #2: +Query Stage #2 (4 -> 4): ShuffleWriterExec(stage_id=2, output_partitioning=Hash([Column { name: "l_suppkey", index: 4 }], 4)) CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "p_partkey", index: 0 }, Column { name: "l_partkey", index: 1 })] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=0, input_partitions=4) + ShuffleReaderExec(stage_id=0, input_partitioning=Hash([Column { name: "p_partkey", index: 0 }], 4)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=1, input_partitions=4) + ShuffleReaderExec(stage_id=1, input_partitioning=Hash([Column { name: "l_partkey", index: 1 }], 4)) -Query Stage #3: +Query Stage #3 (4 -> 4): ShuffleWriterExec(stage_id=3, output_partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 4)) ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-3.parquet]]}, projection=[s_suppkey, s_nationkey] -Query Stage #4: +Query Stage #4 (4 -> 4): ShuffleWriterExec(stage_id=4, output_partitioning=Hash([Column { name: "l_orderkey", index: 2 }], 4)) CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "l_suppkey", index: 4 }, Column { name: "s_suppkey", index: 0 })] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=2, input_partitions=4) + ShuffleReaderExec(stage_id=2, input_partitioning=Hash([Column { name: "l_suppkey", index: 4 }], 4)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=3, input_partitions=4) + ShuffleReaderExec(stage_id=3, input_partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 4)) -Query Stage #5: +Query Stage #5 (4 -> 4): ShuffleWriterExec(stage_id=5, output_partitioning=Hash([Column { name: "o_orderkey", index: 0 }], 4)) CoalesceBatchesExec: target_batch_size=8192 FilterExec: o_orderdate@2 >= 9131 AND o_orderdate@2 <= 9861 ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-3.parquet]]}, predicate=o_orderdate >= Date32("9131") AND o_orderdate <= Date32("9861"), pruning_predicate=o_orderdate_max@0 >= 9131 AND o_orderdate_min@1 <= 9861, projection=[o_orderkey, o_custkey, o_orderdate] -Query Stage #6: +Query Stage #6 (4 -> 4): ShuffleWriterExec(stage_id=6, output_partitioning=Hash([Column { name: "o_custkey", index: 10 }], 4)) CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "l_orderkey", index: 2 }, Column { name: "o_orderkey", index: 0 })] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=4, input_partitions=4) + ShuffleReaderExec(stage_id=4, input_partitioning=Hash([Column { name: "l_orderkey", index: 2 }], 4)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=5, input_partitions=4) + ShuffleReaderExec(stage_id=5, input_partitioning=Hash([Column { name: "o_orderkey", index: 0 }], 4)) -Query Stage #7: +Query Stage #7 (4 -> 4): ShuffleWriterExec(stage_id=7, output_partitioning=Hash([Column { name: "c_custkey", index: 0 }], 4)) ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/customer.parquet/part-3.parquet]]}, projection=[c_custkey, c_nationkey] -Query Stage #8: +Query Stage #8 (4 -> 4): ShuffleWriterExec(stage_id=8, output_partitioning=Hash([Column { name: "c_nationkey", index: 13 }], 4)) CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "o_custkey", index: 10 }, Column { name: "c_custkey", index: 0 })] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=6, input_partitions=4) + ShuffleReaderExec(stage_id=6, input_partitioning=Hash([Column { name: "o_custkey", index: 10 }], 4)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=7, input_partitions=4) + ShuffleReaderExec(stage_id=7, input_partitioning=Hash([Column { name: "c_custkey", index: 0 }], 4)) -Query Stage #9: +Query Stage #9 (1 -> 4): ShuffleWriterExec(stage_id=9, output_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 4)) ParquetExec: limit=None, partitions={1 group: [[mnt/bigdata/tpch/sf10-parquet/nation.parquet/part-0.parquet]]}, projection=[n_nationkey, n_regionkey] -Query Stage #10: +Query Stage #10 (4 -> 4): ShuffleWriterExec(stage_id=10, output_partitioning=Hash([Column { name: "s_nationkey", index: 8 }], 4)) CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "c_nationkey", index: 13 }, Column { name: "n_nationkey", index: 0 })] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=8, input_partitions=4) + ShuffleReaderExec(stage_id=8, input_partitioning=Hash([Column { name: "c_nationkey", index: 13 }], 4)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=9, input_partitions=4) + ShuffleReaderExec(stage_id=9, input_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 4)) -Query Stage #11: +Query Stage #11 (1 -> 4): ShuffleWriterExec(stage_id=11, output_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 4)) ParquetExec: limit=None, partitions={1 group: [[mnt/bigdata/tpch/sf10-parquet/nation.parquet/part-0.parquet]]}, projection=[n_nationkey, n_name] -Query Stage #12: +Query Stage #12 (4 -> 4): ShuffleWriterExec(stage_id=12, output_partitioning=Hash([Column { name: "n_regionkey", index: 15 }], 4)) CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "s_nationkey", index: 8 }, Column { name: "n_nationkey", index: 0 })] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=10, input_partitions=4) + ShuffleReaderExec(stage_id=10, input_partitioning=Hash([Column { name: "s_nationkey", index: 8 }], 4)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=11, input_partitions=4) + ShuffleReaderExec(stage_id=11, input_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 4)) -Query Stage #13: +Query Stage #13 (1 -> 4): ShuffleWriterExec(stage_id=13, output_partitioning=Hash([Column { name: "r_regionkey", index: 0 }], 4)) CoalesceBatchesExec: target_batch_size=8192 FilterExec: r_name@1 = MIDDLE EAST ParquetExec: limit=None, partitions={1 group: [[mnt/bigdata/tpch/sf10-parquet/region.parquet/part-0.parquet]]}, predicate=r_name = Utf8("MIDDLE EAST"), pruning_predicate=r_name_min@0 <= MIDDLE EAST AND MIDDLE EAST <= r_name_max@1, projection=[r_regionkey, r_name] -Query Stage #14: +Query Stage #14 (4 -> 4): ShuffleWriterExec(stage_id=14, output_partitioning=Hash([Column { name: "o_year", index: 0 }], 4)) AggregateExec: mode=Partial, gby=[o_year@0 as o_year], aggr=[SUM(CASE WHEN all_nations.nation = Utf8("IRAQ") THEN all_nations.volume ELSE Int64(0) END), SUM(all_nations.volume)] ProjectionExec: expr=[datepart(YEAR, o_orderdate@2) as o_year, CAST(l_extendedprice@0 AS Decimal128(35, 4)) * CAST(Some(100),23,2 - CAST(l_discount@1 AS Decimal128(23, 2)) AS Decimal128(35, 4)) as volume, n_name@3 as nation] @@ -201,18 +201,19 @@ ShuffleWriterExec(stage_id=14, output_partitioning=Hash([Column { name: "o_year" CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "n_regionkey", index: 15 }, Column { name: "r_regionkey", index: 0 })] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=12, input_partitions=4) + ShuffleReaderExec(stage_id=12, input_partitioning=Hash([Column { name: "n_regionkey", index: 15 }], 4)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=13, input_partitions=4) + ShuffleReaderExec(stage_id=13, input_partitioning=Hash([Column { name: "r_regionkey", index: 0 }], 4)) -Query Stage #15: -ShuffleWriterExec(stage_id=15, output_partitioning=UnknownPartitioning(1)) +Query Stage #15 (4 -> 4): +ShuffleWriterExec(stage_id=15, output_partitioning=Hash([Column { name: "o_year", index: 0 }], 4)) ProjectionExec: expr=[o_year@0 as o_year, SUM(CASE WHEN all_nations.nation = Utf8("IRAQ") THEN all_nations.volume ELSE Int64(0) END)@1 / SUM(all_nations.volume)@2 as mkt_share] AggregateExec: mode=FinalPartitioned, gby=[o_year@0 as o_year], aggr=[SUM(CASE WHEN all_nations.nation = Utf8("IRAQ") THEN all_nations.volume ELSE Int64(0) END), SUM(all_nations.volume)] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=14, input_partitions=4) + ShuffleReaderExec(stage_id=14, input_partitioning=Hash([Column { name: "o_year", index: 0 }], 4)) -Query Stage #16: +Query Stage #16 (1 -> 1): SortExec: [o_year@0 ASC NULLS LAST] - ShuffleReaderExec(stage_id=15, input_partitions=1) + CoalescePartitionsExec + ShuffleReaderExec(stage_id=15, input_partitioning=Hash([Column { name: "o_year", index: 0 }], 4)) diff --git a/testdata/expected-plans/q9.txt b/testdata/expected-plans/q9.txt index 0c47db9..d19a8bc 100644 --- a/testdata/expected-plans/q9.txt +++ b/testdata/expected-plans/q9.txt @@ -74,69 +74,69 @@ SortExec: [nation@0 ASC NULLS LAST,o_year@1 DESC] RaySQL Plan =========== -Query Stage #0: +Query Stage #0 (4 -> 4): ShuffleWriterExec(stage_id=0, output_partitioning=Hash([Column { name: "p_partkey", index: 0 }], 4)) CoalesceBatchesExec: target_batch_size=8192 FilterExec: p_name@1 LIKE %moccasin% ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/part.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/part.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/part.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/part.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/part.parquet/part-3.parquet]]}, predicate=p_name LIKE Utf8("%moccasin%"), projection=[p_partkey, p_name] -Query Stage #1: +Query Stage #1 (4 -> 4): ShuffleWriterExec(stage_id=1, output_partitioning=Hash([Column { name: "l_partkey", index: 1 }], 4)) ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/lineitem.parquet/part-3.parquet]]}, projection=[l_orderkey, l_partkey, l_suppkey, l_quantity, l_extendedprice, l_discount] -Query Stage #2: +Query Stage #2 (4 -> 4): ShuffleWriterExec(stage_id=2, output_partitioning=Hash([Column { name: "l_suppkey", index: 4 }], 4)) CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "p_partkey", index: 0 }, Column { name: "l_partkey", index: 1 })] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=0, input_partitions=4) + ShuffleReaderExec(stage_id=0, input_partitioning=Hash([Column { name: "p_partkey", index: 0 }], 4)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=1, input_partitions=4) + ShuffleReaderExec(stage_id=1, input_partitioning=Hash([Column { name: "l_partkey", index: 1 }], 4)) -Query Stage #3: +Query Stage #3 (4 -> 4): ShuffleWriterExec(stage_id=3, output_partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 4)) ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/supplier.parquet/part-3.parquet]]}, projection=[s_suppkey, s_nationkey] -Query Stage #4: +Query Stage #4 (4 -> 4): ShuffleWriterExec(stage_id=4, output_partitioning=Hash([Column { name: "l_suppkey", index: 4 }, Column { name: "l_partkey", index: 3 }], 4)) CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "l_suppkey", index: 4 }, Column { name: "s_suppkey", index: 0 })] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=2, input_partitions=4) + ShuffleReaderExec(stage_id=2, input_partitioning=Hash([Column { name: "l_suppkey", index: 4 }], 4)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=3, input_partitions=4) + ShuffleReaderExec(stage_id=3, input_partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 4)) -Query Stage #5: +Query Stage #5 (4 -> 4): ShuffleWriterExec(stage_id=5, output_partitioning=Hash([Column { name: "ps_suppkey", index: 1 }, Column { name: "ps_partkey", index: 0 }], 4)) ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/partsupp.parquet/part-3.parquet]]}, projection=[ps_partkey, ps_suppkey, ps_supplycost] -Query Stage #6: +Query Stage #6 (4 -> 4): ShuffleWriterExec(stage_id=6, output_partitioning=Hash([Column { name: "l_orderkey", index: 2 }], 4)) CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "l_suppkey", index: 4 }, Column { name: "ps_suppkey", index: 1 }), (Column { name: "l_partkey", index: 3 }, Column { name: "ps_partkey", index: 0 })] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=4, input_partitions=4) + ShuffleReaderExec(stage_id=4, input_partitioning=Hash([Column { name: "l_suppkey", index: 4 }, Column { name: "l_partkey", index: 3 }], 4)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=5, input_partitions=4) + ShuffleReaderExec(stage_id=5, input_partitioning=Hash([Column { name: "ps_suppkey", index: 1 }, Column { name: "ps_partkey", index: 0 }], 4)) -Query Stage #7: +Query Stage #7 (4 -> 4): ShuffleWriterExec(stage_id=7, output_partitioning=Hash([Column { name: "o_orderkey", index: 0 }], 4)) ParquetExec: limit=None, partitions={4 groups: [[mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-5.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-10.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-11.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-13.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-6.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-14.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-20.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-2.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-22.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-19.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-0.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-16.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-21.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-23.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-4.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-17.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-9.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-1.parquet], [mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-18.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-8.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-12.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-15.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-7.parquet, mnt/bigdata/tpch/sf10-parquet/orders.parquet/part-3.parquet]]}, projection=[o_orderkey, o_orderdate] -Query Stage #8: +Query Stage #8 (4 -> 4): ShuffleWriterExec(stage_id=8, output_partitioning=Hash([Column { name: "s_nationkey", index: 9 }], 4)) CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "l_orderkey", index: 2 }, Column { name: "o_orderkey", index: 0 })] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=6, input_partitions=4) + ShuffleReaderExec(stage_id=6, input_partitioning=Hash([Column { name: "l_orderkey", index: 2 }], 4)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=7, input_partitions=4) + ShuffleReaderExec(stage_id=7, input_partitioning=Hash([Column { name: "o_orderkey", index: 0 }], 4)) -Query Stage #9: +Query Stage #9 (1 -> 4): ShuffleWriterExec(stage_id=9, output_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 4)) ParquetExec: limit=None, partitions={1 group: [[mnt/bigdata/tpch/sf10-parquet/nation.parquet/part-0.parquet]]}, projection=[n_nationkey, n_name] -Query Stage #10: +Query Stage #10 (4 -> 4): ShuffleWriterExec(stage_id=10, output_partitioning=Hash([Column { name: "nation", index: 0 }, Column { name: "o_year", index: 1 }], 4)) AggregateExec: mode=Partial, gby=[nation@0 as nation, o_year@1 as o_year], aggr=[SUM(profit.amount)] ProjectionExec: expr=[n_name@5 as nation, datepart(YEAR, o_orderdate@4) as o_year, CAST(CAST(l_extendedprice@1 AS Decimal128(35, 4)) * CAST(Some(100),23,2 - CAST(l_discount@2 AS Decimal128(23, 2)) AS Decimal128(35, 4)) AS Decimal128(36, 4)) - CAST(ps_supplycost@3 * l_quantity@0 AS Decimal128(36, 4)) as amount] @@ -144,18 +144,19 @@ ShuffleWriterExec(stage_id=10, output_partitioning=Hash([Column { name: "nation" CoalesceBatchesExec: target_batch_size=8192 HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "s_nationkey", index: 9 }, Column { name: "n_nationkey", index: 0 })] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=8, input_partitions=4) + ShuffleReaderExec(stage_id=8, input_partitioning=Hash([Column { name: "s_nationkey", index: 9 }], 4)) CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=9, input_partitions=4) + ShuffleReaderExec(stage_id=9, input_partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 4)) -Query Stage #11: -ShuffleWriterExec(stage_id=11, output_partitioning=UnknownPartitioning(1)) +Query Stage #11 (4 -> 4): +ShuffleWriterExec(stage_id=11, output_partitioning=Hash([Column { name: "nation", index: 0 }, Column { name: "o_year", index: 1 }], 4)) ProjectionExec: expr=[nation@0 as nation, o_year@1 as o_year, SUM(profit.amount)@2 as sum_profit] AggregateExec: mode=FinalPartitioned, gby=[nation@0 as nation, o_year@1 as o_year], aggr=[SUM(profit.amount)] CoalesceBatchesExec: target_batch_size=8192 - ShuffleReaderExec(stage_id=10, input_partitions=4) + ShuffleReaderExec(stage_id=10, input_partitioning=Hash([Column { name: "nation", index: 0 }, Column { name: "o_year", index: 1 }], 4)) -Query Stage #12: +Query Stage #12 (1 -> 1): SortExec: [nation@0 ASC NULLS LAST,o_year@1 DESC] - ShuffleReaderExec(stage_id=11, input_partitions=1) + CoalescePartitionsExec + ShuffleReaderExec(stage_id=11, input_partitioning=Hash([Column { name: "nation", index: 0 }, Column { name: "o_year", index: 1 }], 4))