DataFusion Logical Plan
=======================

Sort: shipping.supp_nation ASC NULLS LAST, shipping.cust_nation ASC NULLS LAST, shipping.l_year ASC NULLS LAST
  Projection: shipping.supp_nation, shipping.cust_nation, shipping.l_year, SUM(shipping.volume) AS revenue
    Aggregate: groupBy=[[shipping.supp_nation, shipping.cust_nation, shipping.l_year]], aggr=[[SUM(shipping.volume)]]
      SubqueryAlias: shipping
        Projection: n1.n_name AS supp_nation, n2.n_name AS cust_nation, datepart(Utf8("YEAR"), lineitem.l_shipdate) AS l_year, 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 volume
          Filter: n1.n_name = Utf8("GERMANY") AND n2.n_name = Utf8("IRAQ") OR n1.n_name = Utf8("IRAQ") AND n2.n_name = Utf8("GERMANY")
            Inner Join: customer.c_nationkey = n2.n_nationkey
              Inner Join: supplier.s_nationkey = n1.n_nationkey
                Inner Join: orders.o_custkey = customer.c_custkey
                  Inner Join: lineitem.l_orderkey = orders.o_orderkey
                    Inner Join: supplier.s_suppkey = lineitem.l_suppkey
                      TableScan: supplier projection=[s_suppkey, s_nationkey]
                      Filter: lineitem.l_shipdate >= Date32("9131") AND lineitem.l_shipdate <= Date32("9861")
                        TableScan: lineitem projection=[l_orderkey, l_suppkey, l_extendedprice, l_discount, l_shipdate], partial_filters=[lineitem.l_shipdate >= Date32("9131"), lineitem.l_shipdate <= Date32("9861")]
                    TableScan: orders projection=[o_orderkey, o_custkey]
                  TableScan: customer projection=[c_custkey, c_nationkey]
                SubqueryAlias: n1
                  Filter: nation.n_name = Utf8("GERMANY") OR nation.n_name = Utf8("IRAQ")
                    TableScan: nation projection=[n_nationkey, n_name], partial_filters=[nation.n_name = Utf8("GERMANY") OR nation.n_name = Utf8("IRAQ")]
              SubqueryAlias: n2
                Filter: nation.n_name = Utf8("IRAQ") OR nation.n_name = Utf8("GERMANY")
                  TableScan: nation projection=[n_nationkey, n_name], partial_filters=[nation.n_name = Utf8("IRAQ") OR nation.n_name = Utf8("GERMANY")]

DataFusion Physical Plan
========================

SortExec: [supp_nation@0 ASC NULLS LAST,cust_nation@1 ASC NULLS LAST,l_year@2 ASC NULLS LAST]
  CoalescePartitionsExec
    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
          RepartitionExec: partitioning=Hash([Column { name: "supp_nation", index: 0 }, Column { name: "cust_nation", index: 1 }, Column { name: "l_year", index: 2 }], 4), input_partitions=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]
                CoalesceBatchesExec: target_batch_size=8192
                  FilterExec: n_name@12 = GERMANY AND n_name@14 = IRAQ OR n_name@12 = IRAQ AND n_name@14 = GERMANY
                    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
                          RepartitionExec: partitioning=Hash([Column { name: "c_nationkey", index: 10 }], 4), input_partitions=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
                                  RepartitionExec: partitioning=Hash([Column { name: "s_nationkey", index: 1 }], 4), input_partitions=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
                                          RepartitionExec: partitioning=Hash([Column { name: "o_custkey", index: 8 }], 4), input_partitions=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
                                                  RepartitionExec: partitioning=Hash([Column { name: "l_orderkey", index: 2 }], 4), input_partitions=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
                                                          RepartitionExec: partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 4), input_partitions=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]
                                                        CoalesceBatchesExec: target_batch_size=8192
                                                          RepartitionExec: partitioning=Hash([Column { name: "l_suppkey", index: 1 }], 4), input_partitions=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]
                                                CoalesceBatchesExec: target_batch_size=8192
                                                  RepartitionExec: partitioning=Hash([Column { name: "o_orderkey", index: 0 }], 4), input_partitions=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]
                                        CoalesceBatchesExec: target_batch_size=8192
                                          RepartitionExec: partitioning=Hash([Column { name: "c_custkey", index: 0 }], 4), input_partitions=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]
                                CoalesceBatchesExec: target_batch_size=8192
                                  RepartitionExec: partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 4), input_partitions=4
                                    RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1
                                      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]
                        CoalesceBatchesExec: target_batch_size=8192
                          RepartitionExec: partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 4), input_partitions=4
                            RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1
                              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]

RaySQL Plan
===========

Query Stage #0:
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:
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:
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)
      CoalesceBatchesExec: target_batch_size=8192
        ShuffleReaderExec(stage_id=1, input_partitions=4)

Query Stage #3:
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:
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)
      CoalesceBatchesExec: target_batch_size=8192
        ShuffleReaderExec(stage_id=3, input_partitions=4)

Query Stage #5:
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:
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)
      CoalesceBatchesExec: target_batch_size=8192
        ShuffleReaderExec(stage_id=5, input_partitions=4)

Query Stage #7:
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:
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)
      CoalesceBatchesExec: target_batch_size=8192
        ShuffleReaderExec(stage_id=7, input_partitions=4)

Query Stage #9:
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:
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]
      CoalesceBatchesExec: target_batch_size=8192
        FilterExec: n_name@12 = GERMANY AND n_name@14 = IRAQ OR n_name@12 = IRAQ AND n_name@14 = GERMANY
          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)
              CoalesceBatchesExec: target_batch_size=8192
                ShuffleReaderExec(stage_id=9, input_partitions=4)

Query Stage #11:
ShuffleWriterExec(stage_id=11, output_partitioning=UnknownPartitioning(1))
  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)

Query Stage #12:
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)

